You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2024/01/16 23:20:58 UTC

(impala) branch master updated: IMPALA-12054: Lazily check Kudu flags in tests

This is an automated email from the ASF dual-hosted git repository.

stigahuang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git


The following commit(s) were added to refs/heads/master by this push:
     new 9ecf0cbfc IMPALA-12054: Lazily check Kudu flags in tests
9ecf0cbfc is described below

commit 9ecf0cbfc79cd4ee0b1b0ff9143f3eda20cbdcb4
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Tue Jan 16 09:41:04 2024 +0800

    IMPALA-12054: Lazily check Kudu flags in tests
    
    I usually shutdown Kudu in my dev env to save some resources. However,
    tests that import skip.py will fail if Kudu cluster is not running
    locally, even if the tests are unrelated to Kudu. The cause is that Kudu
    web pages are accessed when the module is imported, and it fails if Kudu
    cluster is not running.
    
    This patch exposes the decorators of SkipIfKudu as methods just like
    what we did in SkipIfCatalogV2, so Kudu web pages can be checked lazily
    when needed.
    
    Tests:
     - Ran Kudu tests.
     - Ran some Kudu unrelated tests without lauching the Kudu cluster.
    
    Change-Id: Ic7a8282b59d72322085c21c70a5019c51b586a52
    Reviewed-on: http://gerrit.cloudera.org:8080/20904
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 tests/common/skip.py              | 23 +++++++---
 tests/custom_cluster/test_kudu.py | 56 +++++++++++------------
 tests/metadata/test_ddl.py        |  2 +-
 tests/query_test/test_kudu.py     | 96 +++++++++++++++++++--------------------
 4 files changed, 94 insertions(+), 83 deletions(-)

diff --git a/tests/common/skip.py b/tests/common/skip.py
index 31aa4a272..0963739c6 100644
--- a/tests/common/skip.py
+++ b/tests/common/skip.py
@@ -89,13 +89,24 @@ class SkipIfFS:
   eventually_consistent = pytest.mark.skipif(IS_ADLS or IS_COS or IS_OSS or IS_OBS,
       reason="The client is slow to realize changes to file metadata")
 
+
 class SkipIfKudu:
-  no_hybrid_clock = pytest.mark.skipif(
-      get_kudu_master_flag("--use_hybrid_clock") == "false",
-      reason="Test relies on --use_hybrid_clock=true in Kudu.")
-  hms_integration_enabled = pytest.mark.skipif(
-      get_kudu_master_flag("--hive_metastore_uris") != "",
-      reason="Test assumes Kudu/HMS integration is not enabled.")
+  """Expose decorators as methods so that kudu web pages can be checked lazily when
+     needed, instead of whenever this module is imported. This helps to run Kudu
+     unrelated tests without launching the Kudu cluster."""
+
+  @classmethod
+  def no_hybrid_clock(cls):
+    return pytest.mark.skipif(
+        get_kudu_master_flag("--use_hybrid_clock") == "false",
+        reason="Test relies on --use_hybrid_clock=true in Kudu.")
+
+  @classmethod
+  def hms_integration_enabled(cls):
+    return pytest.mark.skipif(
+        get_kudu_master_flag("--hive_metastore_uris") != "",
+        reason="Test assumes Kudu/HMS integration is not enabled.")
+
 
 class SkipIf:
   skip_hbase = pytest.mark.skipif(pytest.config.option.skip_hbase,
diff --git a/tests/custom_cluster/test_kudu.py b/tests/custom_cluster/test_kudu.py
index 36bf55f0f..0bf9b9615 100644
--- a/tests/custom_cluster/test_kudu.py
+++ b/tests/custom_cluster/test_kudu.py
@@ -68,8 +68,8 @@ class TestKuduOperations(CustomKuduTest):
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(impalad_args=\
       "--use_local_tz_for_unix_timestamp_conversions=true")
-  @SkipIfKudu.no_hybrid_clock
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.no_hybrid_clock()
+  @SkipIfKudu.hms_integration_enabled()
   def test_local_tz_conversion_ops(self, vector, unique_database):
     """IMPALA-5539: Test Kudu timestamp reads/writes are correct with the
        use_local_tz_for_unix_timestamp_conversions flag."""
@@ -79,7 +79,7 @@ class TestKuduOperations(CustomKuduTest):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(impalad_args="-kudu_master_hosts=")
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_kudu_master_hosts(self, cursor, kudu_client):
     """Check behavior when -kudu_master_hosts is not provided to catalogd."""
     with self.temp_kudu_table(kudu_client, [INT32]) as kudu_table:
@@ -101,7 +101,7 @@ class TestKuduOperations(CustomKuduTest):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(impalad_args="-kudu_error_buffer_size=1024")
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_error_buffer_size(self, cursor, unique_database):
     """Check that queries fail if the size of the Kudu client errors they generate is
     greater than kudu_error_buffer_size."""
@@ -133,7 +133,7 @@ class TestKuduClientTimeout(CustomKuduTest):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(impalad_args="-kudu_operation_timeout_ms=1")
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_impalad_timeout(self, vector):
     """Check impalad behavior when -kudu_operation_timeout_ms is too low."""
     self.run_test_case('QueryTest/kudu-timeouts-impalad', vector)
@@ -167,7 +167,7 @@ class TestKuduHMSIntegration(CustomKuduTest):
     super(TestKuduHMSIntegration, cls).teardown_class()
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args="-kudu_client_rpc_timeout_ms=30000")
   def test_create_managed_kudu_tables(self, vector, unique_database):
     """Tests the Create table operation when using a kudu table with Kudu's integration
@@ -178,7 +178,7 @@ class TestKuduHMSIntegration(CustomKuduTest):
     self.run_test_case('QueryTest/kudu_create', vector, use_db=unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args="-kudu_client_rpc_timeout_ms=30000 "
                                     "--lineage_event_log_dir={0}"
                                     .format(START_END_TIME_LINEAGE_LOG_DIR))
@@ -347,11 +347,11 @@ class TestKuduHMSIntegration(CustomKuduTest):
       cursor.execute("SHOW TABLES")
       assert (external_table_name,) not in cursor.fetchall()
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_alter_table(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_hms_alter', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_create_kudu_table_like(self, vector, unique_database):
     self.run_test_case(
       'QueryTest/kudu_create_table_like_table',
@@ -621,40 +621,40 @@ class TestKuduTransaction(TestKuduTransactionBase):
   _duplicate_key_error = "Kudu reported write operation errors during transaction."
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_txn_succeed(self, cursor, unique_database):
     self._test_kudu_txn_succeed(cursor, unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_txn_not_implemented(self, cursor, unique_database):
     self._test_kudu_txn_not_implemented(cursor, unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_txn_abort_dup_key(self, cursor, unique_database):
     self._test_kudu_txn_abort_dup_key(cursor, unique_database, True,
         self._duplicate_key_error)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_txn_ctas(self, cursor, unique_database):
     self._test_kudu_txn_ctas(cursor, unique_database, True, self._duplicate_key_error)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @SkipIfBuildType.not_dev_build
   def test_kudu_txn_abort_row_batch(self, cursor, unique_database):
     self._test_kudu_txn_abort_row_batch(cursor, unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @SkipIfBuildType.not_dev_build
   def test_kudu_txn_abort_partial_rows(self, cursor, unique_database):
     self._test_kudu_txn_abort_partial_rows(cursor, unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @SkipIfBuildType.not_dev_build
   def test_kudu_txn_abort_partition_lock(self, cursor, unique_database):
     self._test_kudu_txn_abort_partial_rows(cursor, unique_database)
@@ -672,46 +672,46 @@ class TestKuduTransactionNoIgnore(TestKuduTransactionBase):
   _duplicate_key_error = "Key already present in Kudu table"
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_succeed(self, cursor, unique_database):
     self._test_kudu_txn_succeed(cursor, unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_not_implemented(self, cursor, unique_database):
     self._test_kudu_txn_not_implemented(cursor, unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_abort_dup_key(self, cursor, unique_database):
     self._test_kudu_txn_abort_dup_key(cursor, unique_database, True,
         self._duplicate_key_error)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_ctas(self, cursor, unique_database):
     self._test_kudu_txn_ctas(cursor, unique_database, True, self._duplicate_key_error)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @SkipIfBuildType.not_dev_build
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_abort_row_batch(self, cursor, unique_database):
     self._test_kudu_txn_abort_row_batch(cursor, unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @SkipIfBuildType.not_dev_build
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_abort_partial_rows(self, cursor, unique_database):
     self._test_kudu_txn_abort_partial_rows(cursor, unique_database)
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @SkipIfBuildType.not_dev_build
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_abort_partition_lock(self, cursor, unique_database):
@@ -729,13 +729,13 @@ class TestKuduTransactionIgnoreConflict(TestKuduTransactionBase):
       "--kudu_ignore_conflicts_in_transaction=true"
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_dup_key(self, cursor, unique_database):
     self._test_kudu_txn_abort_dup_key(cursor, unique_database, False, "no error")
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_kudu_txn_ctas(self, cursor, unique_database):
     self._test_kudu_txn_ctas(cursor, unique_database, False, "no error")
@@ -773,7 +773,7 @@ class TestKuduTxnKeepalive(CustomKuduTest):
     super(TestKuduTxnKeepalive, cls).teardown_class()
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @SkipIfBuildType.not_dev_build
   def test_kudu_txn_heartbeat(self, cursor, unique_database):
     # Create Kudu table.
@@ -906,7 +906,7 @@ class TestKuduDmlConflictNoError(TestKuduDmlConflictBase):
   """
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_insert_update_delete(self, cursor, unique_database):
     self._test_insert_update_delete(cursor, unique_database, False)
 
@@ -920,7 +920,7 @@ class TestKuduDmlConflictLogError(TestKuduDmlConflictBase):
   _impalad_args = "--kudu_ignore_conflicts=false"
 
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @CustomClusterTestSuite.with_args(impalad_args=_impalad_args)
   def test_insert_update_delete(self, cursor, unique_database):
     self._test_insert_update_delete(cursor, unique_database, True)
diff --git a/tests/metadata/test_ddl.py b/tests/metadata/test_ddl.py
index 1cb6fa3dd..fb2730ca3 100644
--- a/tests/metadata/test_ddl.py
+++ b/tests/metadata/test_ddl.py
@@ -326,7 +326,7 @@ class TestDdlStatements(TestDdlBase):
         use_db=unique_database, multiple_impalad=self._use_multiple_impalad(vector))
 
   @UniqueDatabase.parametrize(sync_ddl=True)
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_create_kudu(self, vector, unique_database):
     vector.get_value('exec_option')['abort_on_error'] = False
     vector.get_value('exec_option')['kudu_read_mode'] = "READ_AT_SNAPSHOT"
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index 03f5842c9..ca69a8dda 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -72,23 +72,23 @@ class TestKuduBasicDML(KuduTestSuite):
     # models. E.g. see IMPALA-9782.
     add_exec_option_dimension(cls, "mt_dop", [0, 4])
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_insert(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_insert', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_update(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_update', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_upsert(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_upsert', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_delete(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_delete', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_create_table_like_table(self, vector, unique_database):
     self.run_test_case(
       'QueryTest/kudu_create_table_like_table',
@@ -106,15 +106,15 @@ class TestKuduTimestampConvert(KuduTestSuite):
     cls.ImpalaTestMatrix.add_mandatory_exec_option('convert_kudu_utc_timestamps', 'true')
     cls.ImpalaTestMatrix.add_mandatory_exec_option('timezone', '"America/Los_Angeles"')
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_timestamp_conversion(self, vector):
     self.run_test_case('QueryTest/kudu_timestamp_conversion', vector)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_predicate_with_timestamp_conversion(self, vector):
     self.run_test_case('QueryTest/kudu_predicate_with_timestamp_conversion', vector)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_runtime_filter_with_timestamp_conversion(self, vector):
     new_vector = deepcopy(vector)
     del new_vector.get_value('exec_option')['timezone']  # .test file sets timezone
@@ -135,8 +135,8 @@ class TestKuduOperations(KuduTestSuite):
     # these tests.
     add_mandatory_exec_option(cls, "kudu_read_mode", "READ_AT_SNAPSHOT")
 
-  @SkipIfKudu.no_hybrid_clock
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.no_hybrid_clock()
+  @SkipIfKudu.hms_integration_enabled()
   def test_out_of_range_timestamps(self, vector, cursor, kudu_client, unique_database):
     """Test timestamp values that are outside of Impala's supported date range."""
     cursor.execute("set kudu_read_mode=READ_AT_SNAPSHOT")
@@ -167,35 +167,35 @@ class TestKuduOperations(KuduTestSuite):
     self.run_test_case('QueryTest/kudu-overflow-ts-abort-on-error', vector,
         use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_scan_node(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu-scan-node', vector, use_db=unique_database)
   @SkipIfNotHdfsMinicluster.tuned_for_minicluster
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_insert_mem_limit(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_insert_mem_limit', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_partition_ddl(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_partition_ddl', vector, use_db=unique_database)
 
   @pytest.mark.skipif(IMPALA_TEST_CLUSTER_PROPERTIES.is_remote_cluster(),
                       reason="Test references hardcoded hostnames: IMPALA-4873")
   @pytest.mark.execute_serially
-  @SkipIfKudu.no_hybrid_clock
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.no_hybrid_clock()
+  @SkipIfKudu.hms_integration_enabled()
   def test_kudu_alter_table(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_alter', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_stats(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_stats', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_describe(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_describe', vector, use_db=unique_database)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_limit(self, vector, unique_database):
     self.run_test_case('QueryTest/kudu_limit', vector, use_db=unique_database)
 
@@ -409,8 +409,8 @@ class TestKuduOperations(KuduTestSuite):
     cursor.execute("SELECT * FROM %s.foo" % (unique_database))
     assert cursor.fetchall() == [(0, 0)]
 
-  @SkipIfKudu.no_hybrid_clock
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.no_hybrid_clock()
+  @SkipIfKudu.hms_integration_enabled()
   def test_kudu_col_removed(self, cursor, kudu_client, unique_database):
     """Test removing a Kudu column outside of Impala."""
     cursor.execute("set kudu_read_mode=READ_AT_SNAPSHOT")
@@ -471,7 +471,7 @@ class TestKuduOperations(KuduTestSuite):
       if kudu_client.table_exists(name):
         kudu_client.delete_table(name)
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_column_storage_attributes(self, cursor, unique_database):
     """Tests that for every valid combination of column type, encoding, and compression,
        we can insert a value and scan it back from Kudu."""
@@ -673,7 +673,7 @@ class TestKuduPartitioning(KuduTestSuite):
 
 class TestCreateExternalTable(KuduTestSuite):
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_external_timestamp_default_value(self, cursor, kudu_client, unique_database):
     """Checks that a Kudu table created outside Impala with a default value on a
        UNIXTIME_MICROS column can be loaded by Impala, and validates the DESCRIBE
@@ -706,7 +706,7 @@ class TestCreateExternalTable(KuduTestSuite):
       if kudu_client.table_exists(name):
         kudu_client.delete_table(name)
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_implicit_table_props(self, cursor, kudu_client):
     """Check that table properties added internally during table creation are as
        expected.
@@ -728,7 +728,7 @@ class TestCreateExternalTable(KuduTestSuite):
         assert ["", "storage_handler", "org.apache.hadoop.hive.kudu.KuduStorageHandler"] \
             in table_desc
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_col_types(self, cursor, kudu_client):
     """Check that a table can be created using all available column types."""
     # TODO: Add DECIMAL when the Kudu python client supports decimal
@@ -747,7 +747,7 @@ class TestCreateExternalTable(KuduTestSuite):
           assert col_type.upper() == \
               self.kudu_col_type_to_impala_col_type(kudu_col.type.type)
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_unsupported_binary_col(self, cursor, kudu_client):
     """Check that external tables with BINARY columns fail gracefully.
     """
@@ -763,7 +763,7 @@ class TestCreateExternalTable(KuduTestSuite):
       except Exception as e:
         assert "Kudu type 'binary' is not supported in Impala" in str(e)
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_drop_external_table(self, cursor, kudu_client):
     """Check that dropping an external table only affects the catalog and does not delete
        the table in Kudu.
@@ -783,7 +783,7 @@ class TestCreateExternalTable(KuduTestSuite):
         assert "Could not resolve table reference" in str(e)
       assert kudu_client.table_exists(kudu_table.name)
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_explicit_name(self, cursor, kudu_client):
     """Check that a Kudu table can be specified using a table property."""
     with self.temp_kudu_table(kudu_client, [INT32]) as kudu_table:
@@ -796,7 +796,7 @@ class TestCreateExternalTable(KuduTestSuite):
         cursor.execute("SELECT * FROM %s" % table_name)
         assert len(cursor.fetchall()) == 0
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_explicit_name_preference(self, cursor, kudu_client):
     """Check that the table name from a table property is used when a table of the
        implied name also exists.
@@ -815,7 +815,7 @@ class TestCreateExternalTable(KuduTestSuite):
               [("a", "bigint", "", "true", "true", "false", "", "AUTO_ENCODING",
                 "DEFAULT_COMPRESSION", "0")]
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_explicit_name_doesnt_exist(self, cursor, kudu_client):
     kudu_table_name = self.random_table_name()
     try:
@@ -828,7 +828,7 @@ class TestCreateExternalTable(KuduTestSuite):
     except Exception as e:
       assert "Table does not exist in Kudu: '%s'" % kudu_table_name in str(e)
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_explicit_name_doesnt_exist_but_implicit_does(self, cursor, kudu_client):
     """Check that when an explicit table name is given but that table doesn't exist,
        there is no fall-through to an existing implicit table.
@@ -845,8 +845,8 @@ class TestCreateExternalTable(KuduTestSuite):
       except Exception as e:
         assert "Table does not exist in Kudu: '%s'" % table_name in str(e)
 
-  @SkipIfKudu.no_hybrid_clock
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.no_hybrid_clock()
+  @SkipIfKudu.hms_integration_enabled()
   def test_table_without_partitioning(self, cursor, kudu_client, unique_database):
     """Test a Kudu table created without partitioning (i.e. equivalent to a single
        unbounded partition). It is not possible to create such a table in Impala, but
@@ -881,8 +881,8 @@ class TestCreateExternalTable(KuduTestSuite):
       if kudu_client.table_exists(name):
         kudu_client.delete_table(name)
 
-  @SkipIfKudu.no_hybrid_clock
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.no_hybrid_clock()
+  @SkipIfKudu.hms_integration_enabled()
   def test_column_name_case(self, cursor, kudu_client, unique_database):
     """IMPALA-5286: Tests that an external Kudu table that was created with a column name
        containing upper case letters is handled correctly."""
@@ -938,7 +938,7 @@ class TestCreateExternalTable(KuduTestSuite):
       if kudu_client.table_exists(table_name):
         kudu_client.delete_table(table_name)
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_conflicting_column_name(self, cursor, kudu_client, unique_database):
     """IMPALA-5283: Tests that loading an external Kudu table that was created with column
        names that differ only in case results in an error."""
@@ -1003,7 +1003,7 @@ class TestShowCreateTable(KuduTestSuite):
     assert output == \
         textwrap.dedent(show_create_sql.format(**format_args)).strip()
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_primary_key_and_distribution(self, cursor):
     # TODO: Add case with BLOCK_SIZE
     self.assert_show_create_equals(cursor,
@@ -1104,7 +1104,7 @@ class TestShowCreateTable(KuduTestSuite):
             db=cursor.conn.db_name, p=self.column_properties,
             kudu_addr=KUDU_MASTER_HOSTS))
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_timestamp_default_value(self, cursor):
     create_sql_fmt = """
         CREATE TABLE {table} (c INT, d TIMESTAMP,
@@ -1135,7 +1135,7 @@ class TestShowCreateTable(KuduTestSuite):
       create_sql_fmt % ("2009-01-01 00:00:00.000000999"),
       show_create_sql_fmt % ("1230768000000001"))
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_external_kudu_table_name_with_show_create(self, cursor, kudu_client,
       unique_database):
     """Check that the generated kudu.table_name tblproperty is present with
@@ -1169,7 +1169,7 @@ class TestShowCreateTable(KuduTestSuite):
       if kudu_client.table_exists(kudu_table_name):
         kudu_client.delete_table(kudu_table_name)
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_managed_kudu_table_name_with_show_create(self, cursor):
     """Check that the generated kudu.table_name tblproperty is not present with
        show create table with managed Kudu tables.
@@ -1234,7 +1234,7 @@ class TestShowCreateTable(KuduTestSuite):
 
 class TestDropDb(KuduTestSuite):
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_drop_non_empty_db(self, unique_cursor, kudu_client):
     """Check that an attempt to drop a database will fail if Kudu tables are present
        and that the tables remain.
@@ -1254,7 +1254,7 @@ class TestDropDb(KuduTestSuite):
       unique_cursor.execute("SELECT COUNT(*) FROM %s.%s" % (db_name, impala_table_name))
       assert unique_cursor.fetchall() == [(0, )]
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_drop_db_cascade(self, unique_cursor, kudu_client):
     """Check that an attempt to drop a database will succeed even if Kudu tables are
        present and that the managed tables are removed.
@@ -1287,7 +1287,7 @@ class TestDropDb(KuduTestSuite):
       assert kudu_client.table_exists(kudu_table.name)
       assert not kudu_client.table_exists(managed_table_name)
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_soft_drop_db_cascade(self, unique_cursor, kudu_client):
     """Check that an attempt to drop a database will succeed but the managed Kudu tables
        are not removed immediately if 'kudu_table_reserve_seconds' is greater than 0.
@@ -1322,7 +1322,7 @@ class TestDropDb(KuduTestSuite):
       assert kudu_tbl_name not in kudu_client.list_soft_deleted_tables()
 
 class TestImpalaKuduIntegration(KuduTestSuite):
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_replace_kudu_table(self, cursor, kudu_client):
     """Check that an external Kudu table is accessible if the underlying Kudu table is
         modified using the Kudu client.
@@ -1359,7 +1359,7 @@ class TestImpalaKuduIntegration(KuduTestSuite):
              ("c", "string", "", "false", "", "true", "", "AUTO_ENCODING",
               "DEFAULT_COMPRESSION", "0")]
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_delete_external_kudu_table(self, cursor, kudu_client):
     """Check that Impala can recover from the case where the underlying Kudu table of
         an external table is dropped using the Kudu client.
@@ -1386,7 +1386,7 @@ class TestImpalaKuduIntegration(KuduTestSuite):
       cursor.execute("SHOW TABLES")
       assert (impala_table_name,) not in cursor.fetchall()
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_delete_managed_kudu_table(self, cursor, kudu_client, unique_database):
     """Check that dropping a managed Kudu table works even if the underlying Kudu table
         has been dropped externally."""
@@ -1401,7 +1401,7 @@ class TestImpalaKuduIntegration(KuduTestSuite):
     cursor.execute("SHOW TABLES IN %s" % unique_database)
     assert (impala_tbl_name,) not in cursor.fetchall()
 
-  @SkipIfKudu.hms_integration_enabled
+  @SkipIfKudu.hms_integration_enabled()
   def test_soft_delete_kudu_table(self, cursor, kudu_client, unique_database):
     """Check that the query option 'kudu_table_reserve_seconds' works for managed Kudu
     table. If it is greater than 0, the underlying Kudu will not be deleted immediately.
@@ -1612,7 +1612,7 @@ class TestKuduReadTokenSplit(KuduTestSuite):
     # these tests.
     add_mandatory_exec_option(cls, "kudu_read_mode", "READ_AT_SNAPSHOT")
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   @SkipIfNotHdfsMinicluster.tuned_for_minicluster
   def test_kudu_scanner(self, vector, unique_database):
     """This runs explain query with variations of mt_dop and
@@ -1689,7 +1689,7 @@ class TestKuduInsertWithBufferedTupleDesc(KuduTestSuite):
       "on uu.big_id=i.cl_agrmt " \
       "where u.big_id is null"
 
-  @SkipIfKudu.no_hybrid_clock
+  @SkipIfKudu.no_hybrid_clock()
   def test_kudu_insert_with_buffered_tuple_desc(self, cursor, kudu_client,
       unique_database):
     # Create Kudu tables.