IMPALA-13929: Make 'functional-query' the default workload in tests

This change adds get_workload() to ImpalaTestSuite and removes it
from all test suites that already returned 'functional-query'.
get_workload() is also removed from CustomClusterTestSuite which
used to return 'tpch'.

All other changes besides impala_test_suite.py and
custom_cluster_test_suite.py are just mass removals of
get_workload() functions.

The behavior is only changed in custom cluster tests that didn't
override get_workload(). By returning 'functional-query' instead
of 'tpch', exploration_strategy() will no longer return 'core' in
'exhaustive' test runs. See IMPALA-3947 on why workload affected
exploration_strategy. An example for affected test is
TestCatalogHMSFailures which was skipped both in core and exhaustive
runs before this change.

get_workload() functions that return a different workload than
'functional-query' are not changed - it is possible that some of
these also don't handle exploration_strategy() as expected, but
individually checking these tests is out of scope in this patch.

Change-Id: I9ec6c41ffb3a30e1ea2de773626d1485c69fe115
Reviewed-on: http://gerrit.cloudera.org:8080/22726
Reviewed-by: Riza Suminto <riza.suminto@cloudera.com>
Reviewed-by: Daniel Becker <daniel.becker@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This commit is contained in:
Csaba Ringhofer
2025-04-03 12:40:05 +02:00
parent bad56af62c
commit f98b697c7b
189 changed files with 12 additions and 1282 deletions

View File

@@ -69,10 +69,6 @@ class TestRanger(CustomClusterTestSuite):
Tests for Apache Ranger integration with Apache Impala.
"""
@classmethod
def get_workload(cls):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
impala_log_dir=tempfile.mkdtemp(prefix="ranger_audit_xff", dir=os.getenv("LOG_DIR")),
@@ -3164,10 +3160,6 @@ class TestRangerColumnMaskingComplexTypesInSelectList(CustomClusterTestSuite):
while some tests in TestRanger needs Beeswax interface otherwise some of them fails.
"""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestRangerColumnMaskingComplexTypesInSelectList, cls).add_test_dimensions()

View File

@@ -41,10 +41,6 @@ LOG = logging.getLogger('test_catalog_service_client')
# on a stable catalog Thrift API.
class TestCatalogServiceClient(ImpalaTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestCatalogServiceClient, cls).add_test_dimensions()

View File

@@ -23,10 +23,6 @@ from tests.common.impala_test_suite import ImpalaTestSuite
from tests.common.test_dimensions import create_single_exec_option_dimension
class TestLargeNumPartitions(ImpalaTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestLargeNumPartitions, cls).add_test_dimensions()

View File

@@ -114,10 +114,6 @@ class CustomClusterTestSuite(ImpalaTestSuite):
# Args for cluster startup/teardown when sharing a single cluster for the entire class.
SHARED_CLUSTER_ARGS = {}
@classmethod
def get_workload(cls):
return 'tpch'
@classmethod
def add_test_dimensions(cls):
super(CustomClusterTestSuite, cls).add_test_dimensions()

View File

@@ -190,6 +190,12 @@ class ImpalaTestSuite(BaseTestSuite):
# not disabling glog buffering (--logbuflevel=-1).
_warn_assert_log = False
# Use 'functional-query' as default workload. The workload affects exploration strategy
# if workload_exploration_strategy is set. See exploration_strategy() for more details.
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
"""
@@ -1431,6 +1437,12 @@ class ImpalaTestSuite(BaseTestSuite):
exec_single_node_option=exec_single_node_option,
disable_codegen_rows_threshold_options=[0])
# The exploration strategy is affected by both option exploration_strategy and
# workload_exploration_strategy+workload. workload_exploration_strategy is used
# by bin/run-all-test.sh - using a workload (get_workload()) that is not set in
# run-all-tests.sh can lead to returning "core" even in exhaustive test runs.
# TODO: workload handling is not used consistently in tests and it is not even
# clear how it should be used (IMPALA-13929)
@classmethod
def exploration_strategy(cls):
default_strategy = pytest.config.option.exploration_strategy

View File

@@ -81,10 +81,6 @@ class KuduTestSuite(ImpalaTestSuite):
def setup_class(cls):
super(KuduTestSuite, cls).setup_class()
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(KuduTestSuite, cls).add_test_dimensions()

View File

@@ -174,10 +174,6 @@ def wait_statestore_heartbeat(num_heartbeat=1):
class TestAdmissionControllerBase(CustomClusterTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def default_test_protocol(cls):
# Do not change this. Multiple test method has been hardcoded under this assumption.

View File

@@ -25,10 +25,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
class TestAIGenerateText(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -25,10 +25,6 @@ from tests.common.skip import SkipIfBuildType
class TestAllocFail(CustomClusterTestSuite):
"""Tests for handling malloc() failure for UDF/UDA"""
@classmethod
def get_workload(self):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args("--stress_fn_ctx_alloc=1")
def test_alloc_fail_init(self, vector):

View File

@@ -23,10 +23,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
from tests.common.skip import SkipIfBuildType
class TestAlwaysFalseFilter(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -31,10 +31,6 @@ LOG = logging.getLogger("test_auto_scaling")
TOTAL_BACKENDS_METRIC_NAME = "cluster-membership.backends.total"
class TestAutoScaling(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -42,10 +42,6 @@ class TestAutomaticCatalogInvalidation(CustomClusterTestSuite):
(not IS_HDFS and not IS_LOCAL) else 10
timeout_flag = "--invalidate_tables_timeout_s=" + str(timeout)
@classmethod
def get_workload(cls):
return 'functional-query'
def _get_catalog_object(self):
""" Return the catalog object of functional.alltypes serialized to string. """
return self.cluster.catalogd.service.read_debug_webpage(

View File

@@ -40,10 +40,6 @@ def _get_disk_write_fail_action(port):
# Note: query-retry is not enabled by default.
@SkipIfNotHdfsMinicluster.tuned_for_minicluster
class TestBlacklist(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':
@@ -190,10 +186,6 @@ class TestBlacklist(CustomClusterTestSuite):
# tests.
@SkipIfNotHdfsMinicluster.tuned_for_minicluster
class TestBlacklistFaultyDisk(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -23,10 +23,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
class TestBlacklistedDbsAndTables(CustomClusterTestSuite):
"""Test for db and table blacklist."""
@classmethod
def get_workload(self):
return 'functional-query'
def __expect_error_in_result(self, stmt, expected_err):
# Drop db/table/view statements won't fail if they contains IF EXISTS. Instead,
# the error message is returned as results.

View File

@@ -35,10 +35,6 @@ DAEMON_ARGS = ['impalad_args', 'state_store_args', 'catalogd_args']
class TestBreakpadBase(CustomClusterTestSuite):
"""Base class with utility methods for all breakpad tests."""
@classmethod
def get_workload(cls):
return 'functional-query'
def setup_method(self, method):
# Override parent
# The temporary directory gets removed in teardown_method() after each test.

View File

@@ -30,10 +30,6 @@ class TestCalcitePlanner(CustomClusterTestSuite):
def setup_class(cls):
super(TestCalcitePlanner, cls).setup_class()
@classmethod
def get_workload(cls):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(start_args="--use_calcite_planner=true")
def test_calcite_frontend(self, vector, unique_database):

View File

@@ -36,10 +36,6 @@ class TestHiveMetaStoreFailure(CustomClusterTestSuite):
"""Tests to validate the Catalog Service continues to function even if the HMS
fails."""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
super(TestHiveMetaStoreFailure, cls).setup_class()

View File

@@ -88,10 +88,6 @@ class TestCatalogStartupDelay(CustomClusterTestSuite):
different cluster setups (different metadata, exclusive coordinators). This
is not testing anything beyond successful startup."""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -52,9 +52,6 @@ class TestCatalogdHA(CustomClusterTestSuite):
SS_TEST_PORT = ["25010"]
def get_workload(self):
return 'functional-query'
# Verify port of the active catalogd of statestore is matching with the catalog
# service port of the given catalogd service.
def __verify_statestore_active_catalogd_port(self, catalogd_service):

View File

@@ -31,10 +31,6 @@ class TestCodegenCache(CustomClusterTestSuite):
""" This test enables the codegen cache and verfies that cache hit and miss counts
in the runtime profile and metrics are as expected.
"""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -24,10 +24,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
class TestCompactCatalogUpdates(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -26,10 +26,6 @@ HIVE_SITE_EXT_DIR = getenv('IMPALA_HOME') + '/fe/src/test/resources/hive-site-ex
class TestCustomHiveConfigs(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
super(TestCustomHiveConfigs, cls).setup_class()

View File

@@ -57,10 +57,6 @@ class TestCustomStatestore(CustomClusterTestSuite):
handle.bind(('localhost', 0))
_, port = handle.getsockname()
@classmethod
def get_workload(self):
return 'functional-query'
def __get_protocol_version(self):
request = Subscriber.TGetProtocolVersionRequest(
protocol_version=Subscriber.StatestoreServiceVersion.V2)
@@ -207,10 +203,6 @@ class TestStatestoreStartupDelay(CustomClusterTestSuite):
expected to be able to tolerate this delay with FLAGS_tolerate_statestore_startup_delay
set as true. This is not testing anything beyond successful startup."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -33,10 +33,6 @@ class TestDataCache(CustomClusterTestSuite):
this test checks the number of data cache hit counts, which implicitly relies
on the scheduler's behavior and number of HDFS blocks.
"""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -26,10 +26,6 @@ from tests.common.skip import SkipIfFS
class TestDisableFeatures(CustomClusterTestSuite):
"""Tests that involve disabling features at startup."""
@classmethod
def get_workload(self):
return 'functional-query'
@SkipIfFS.hdfs_caching
@pytest.mark.execute_serially
@UniqueDatabase.parametrize(sync_ddl=True)

View File

@@ -23,10 +23,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
class TestDiskSpillConfigurations(CustomClusterTestSuite):
"""Tests to exercise non-default disk spill configurations end-to-end."""
@classmethod
def get_workload(self):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args("--disk_spill_encryption=false")
def test_disk_spill_encryption_disabled(self, vector):

View File

@@ -32,10 +32,6 @@ class TestEventProcessingError(CustomClusterTestSuite):
Tests for verify event processor not going into error state whenever there are
runtime exceptions while processing events.
"""
@classmethod
def get_workload(self):
return 'functional-query'
@CustomClusterTestSuite.with_args(
impalad_args="--use_local_catalog=true",
catalogd_args="--catalog_topic_mode=minimal "

View File

@@ -25,10 +25,6 @@ from tests.common.skip import SkipIfBuildType
@SkipIfBuildType.not_dev_build
class TestExchangeDeferredBatches(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -33,10 +33,6 @@ if IS_ISILON:
class TestExchangeDelays(CustomClusterTestSuite):
"""Tests for handling delays in finding data stream receivers"""
@classmethod
def get_workload(self):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
"--stress_datastream_recvr_delay_ms={0}".format(DELAY_MS)

View File

@@ -31,10 +31,6 @@ from time import sleep
class TestExtDataSources(CustomClusterTestSuite):
"""Impala query tests for external data sources."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestExtDataSources, cls).add_test_dimensions()
@@ -179,10 +175,6 @@ class TestExtDataSources(CustomClusterTestSuite):
class TestHivePostgresJdbcTables(CustomClusterTestSuite):
"""Tests for hive jdbc postgres tables. """
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
super(TestHivePostgresJdbcTables, cls).setup_class()
@@ -266,10 +258,6 @@ class TestMySqlExtJdbcTables(CustomClusterTestSuite):
"""Impala query tests for external jdbc tables on MySQL server.
It also includes tests for external hive jdbc tables on mysql."""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def _setup_mysql_test_env(cls):
# Download MySQL docker image and jdbc driver, start MySQL server, create database
@@ -394,10 +382,6 @@ class TestMySqlExtJdbcTables(CustomClusterTestSuite):
class TestImpalaExtJdbcTables(CustomClusterTestSuite):
"""Impala query tests for external jdbc tables in Impala cluster."""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestImpalaExtJdbcTables, cls).add_test_dimensions()

View File

@@ -32,10 +32,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
class TestFrontendConnectionLimit(CustomClusterTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestFrontendConnectionLimit, cls).add_test_dimensions()

View File

@@ -28,10 +28,6 @@ SHOW_FUNCTIONS = "show functions in _impala_builtins"
class TestGeospatialLibrary(CustomClusterTestSuite):
"""Tests the geospatial_library backend flag"""
@classmethod
def get_workload(cls):
return 'functional-query'
@CustomClusterTestSuite.with_args(start_args='--geospatial_library=NONE')
@SkipIfApacheHive.feature_not_supported
@pytest.mark.execute_serially

View File

@@ -36,10 +36,6 @@ class TestHBaseHmsColumnOrder(CustomClusterTestSuite):
cls.ImpalaTestMatrix.add_constraint(lambda v:
v.get_value('table_format').file_format == 'hbase')
@classmethod
def get_workload(self):
return 'functional-query'
@CustomClusterTestSuite.with_args(
catalogd_args="--use_hms_column_order_for_hbase_tables=true")
def test_hbase_hms_column_order(self, vector, unique_database):

View File

@@ -40,10 +40,6 @@ class TestHdfsFdCaching(CustomClusterTestSuite):
NUM_ROWS = 100
INSERT_TPL = "insert into cachefd.simple values"
@classmethod
def get_workload(self):
return 'functional-query'
def create_n_files(self, n):
"""Creates 'n' files by performing 'n' inserts with NUM_ROWS rows."""
values = ", ".join(["({0},{0},{0})".format(x) for x in range(self.NUM_ROWS)])

View File

@@ -31,10 +31,6 @@ from tests.util.shell_util import exec_process
class TestHdfsTimeouts(CustomClusterTestSuite):
"""Test to verify that HDFS operations time out correctly."""
@classmethod
def get_workload(self):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
impalad_args="--hdfs_operation_timeout_sec=5 --max_cached_file_handles=0")

View File

@@ -33,10 +33,6 @@ PARQUET_CODECS = ['none', 'snappy', 'gzip', 'zstd', 'zstd:7', 'lz4']
class TestParquetInterop(CustomClusterTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -40,10 +40,6 @@ class TestHiveParquetTimestampConversion(CustomClusterTestSuite):
v.get_value('table_format').file_format == 'parquet' and
v.get_value('table_format').compression_codec == 'none')
@classmethod
def get_workload(self):
return 'functional-query'
def check_sanity(self, expect_converted_result):
data = self.execute_query_expect_success(self.client, """
SELECT COUNT(timestamp_col), COUNT(DISTINCT timestamp_col),

View File

@@ -26,10 +26,6 @@ from TCLIService import TCLIService
class TestHS2(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -43,10 +43,6 @@ class TestIcebergAlwaysAllowMergeOnRead(CustomClusterTestSuite):
return "AnalysisException: Unsupported '{0}': 'copy-on-write' for " \
"Iceberg table: {1}.{2}".format(operation, db, self.TABLE_NAME)
@classmethod
def get_workload(self):
return 'functional-query'
@CustomClusterTestSuite.with_args(
impalad_args='--iceberg_always_allow_merge_on_read_operations=true')
@pytest.mark.execute_serially

View File

@@ -35,9 +35,6 @@ IMPALAD_ARGS = """--use_local_catalog=true --catalogd_deployed=false
class TestIcebergRestCatalog(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
def setup_method(self, method):
# Invoke start-impala-cluster.py with '--no_catalogd'

View File

@@ -30,10 +30,6 @@ class TestIcebergStrictDataFileLocation(CustomClusterTestSuite):
EXCEPTION = "IcebergTableLoadingException: " \
"Error loading metadata for Iceberg table"
@classmethod
def get_workload(self):
return 'functional-query'
@CustomClusterTestSuite.with_args(
catalogd_args='--iceberg_allow_datafiles_in_table_location_only=true')
@pytest.mark.execute_serially

View File

@@ -34,10 +34,6 @@ class TestIcebergWithPuffinStatsStartupFlag(CustomClusterTestSuite):
"""Tests for checking the behaviour of the startup flag
'enable_reading_puffin_stats'."""
@classmethod
def get_workload(self):
return 'functional-query'
@CustomClusterTestSuite.with_args(
catalogd_args='--enable_reading_puffin_stats=false')
@pytest.mark.execute_serially
@@ -84,10 +80,6 @@ class TestIcebergTableWithPuffinStats(CustomClusterTestSuite):
TBL_NAME = "ice_puffin_tbl"
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestIcebergTableWithPuffinStats, cls).add_test_dimensions()

View File

@@ -24,10 +24,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
class TestIncrementalMetadataUpdate(CustomClusterTestSuite):
""" Validates incremental metadata updates across catalogd and coordinators."""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
super(TestIncrementalMetadataUpdate, cls).setup_class()

View File

@@ -27,10 +27,6 @@ LOG = logging.getLogger('test_jvm_mem_tracking')
class TestJvmMemTracker(CustomClusterTestSuite):
"""Test that JVM heap memory consumption is counted against process usage."""
@classmethod
def get_workload(self):
return 'functional-query'
@CustomClusterTestSuite.with_args(impalad_args="--mem_limit_includes_jvm=true \
--codegen_cache_capacity=0",
start_args="--jvm_args=-Xmx1g", cluster_size=1)

View File

@@ -32,10 +32,6 @@ class TestKrpcMemUsage(CustomClusterTestSuite):
TEST_QUERY = "select count(c2.string_col) from \
functional.alltypestiny join functional.alltypessmall c2"
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -29,10 +29,6 @@ class TestKrpcMetrics(CustomClusterTestSuite):
TEST_QUERY = 'select count(*) from tpch_parquet.lineitem l1 \
join tpch_parquet.lineitem l2 where l1.l_orderkey = l2.l_orderkey;'
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -25,10 +25,6 @@ from tests.common.impala_cluster import DEFAULT_KRPC_PORT
class TestKrpcOptions(CustomClusterTestSuite):
"""Test for different options when using KRPC."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -26,10 +26,6 @@ from tests.util.shell_util import exec_process
class TestKrpcSocket(CustomClusterTestSuite):
"""Test for different types of socket used by KRPC."""
@classmethod
def get_workload(self):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
impalad_args="--rpc_use_unix_domain_socket=false")

View File

@@ -35,10 +35,6 @@ LOG = logging.getLogger(__name__)
class CustomKuduTest(CustomClusterTestSuite, KuduTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def default_test_protocol(cls):
# run_test_case() can produce different result types between beeswax vs hs2 protocol
@@ -67,10 +63,6 @@ class CustomKuduTest(CustomClusterTestSuite, KuduTestSuite):
class TestKuduOperations(CustomKuduTest):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestKuduOperations, cls).add_test_dimensions()
@@ -145,10 +137,6 @@ class TestKuduClientTimeout(CustomKuduTest):
this turns out to be the case, specific tests may need to be re-considered or
removed."""
@classmethod
def get_workload(cls):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(impalad_args="-kudu_operation_timeout_ms=1")
@SkipIfKudu.hms_integration_enabled()
@@ -165,10 +153,6 @@ class TestKuduHMSIntegration(CustomKuduTest):
"""Tests the different DDL operations when using a kudu table with Kudu's integration
with the Hive Metastore."""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
# Restart Kudu cluster with HMS integration enabled
@@ -413,10 +397,6 @@ class TestKuduTransactionBase(CustomKuduTest):
# query to upsert a row in Kudu table.
_upsert_query = "upsert into {0} values (3, 'hello')"
@classmethod
def get_workload(cls):
return 'functional-query'
def _test_kudu_txn_succeed(self, unique_database):
# Create Kudu table.
table_name = "%s.test_kudu_txn_succeed" % unique_database
@@ -763,10 +743,6 @@ class TestKuduTxnKeepalive(CustomKuduTest):
# queries to insert rows into Kudu table.
_insert_3_rows_query = "insert into {0} values (0, 'a'), (1, 'b'), (2, 'c')"
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
# Restart Kudu cluster with txn_keepalive_interval_ms as 1000 ms.
@@ -821,10 +797,6 @@ class TestKuduDmlConflictBase(CustomKuduTest):
# query to drop all rows from Kudu table.
_delete_all_query = "delete from {0}"
@classmethod
def get_workload(cls):
return 'functional-query'
def _check_errors(self, query_profile, expect_error, error_message, num_row_erros):
"""
Check ocurrence of error_message and num_row_errors in query_profile.

View File

@@ -25,10 +25,6 @@ class TestKuduNotAvailable(CustomClusterTestSuite):
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",

View File

@@ -42,10 +42,6 @@ class TestLineage(CustomClusterTestSuite):
def setup_class(cls):
super(TestLineage, cls).setup_class()
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def teardown_class(cls):
super(TestLineage, cls).teardown_class()

View File

@@ -597,10 +597,6 @@ class TestLocalCatalogObservability(CustomClusterTestSuite):
class TestFullAcid(CustomClusterTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@SkipIfHive2.acid
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(

View File

@@ -43,10 +43,6 @@ class TestLocalTzConversion(CustomClusterTestSuite):
v.get_value('table_format').file_format == 'text' and
v.get_value('table_format').compression_codec == 'none')
@classmethod
def get_workload(self):
return 'functional-query'
@SkipIfFS.hbase
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args("--use_local_tz_for_unix_timestamp_conversions=true")

View File

@@ -28,10 +28,6 @@ LOG = logging.getLogger(__name__)
class TestLoggingCore(CustomClusterTestSuite):
"""Test existence of certain log lines under some scenario."""
@classmethod
def get_workload(cls):
return 'functional-query'
def _test_max_errors(self, max_error_logs_per_instance, max_errors, expect_downgraded):
"""Test that number of non-fatal error printed to INFO log is limited by
max_errors and max_error_logs_per_instance."""
@@ -73,10 +69,6 @@ class TestLogFlushPermissionDenied(CustomClusterTestSuite):
"""Test logging of failures to open log files with cause Permission denied."""
LOG_FLUSH_FAILURES_DIR = "log_flush_failures_dir"
@classmethod
def get_workload(cls):
return 'functional-query'
def setup_method(self, method):
# Override parent
super(TestLogFlushPermissionDenied, self).setup_method(method)

View File

@@ -28,10 +28,6 @@ from tests.verifiers.metric_verifier import MetricVerifier
class TestMemReservations(CustomClusterTestSuite):
"""Tests for memory reservations that require custom cluster arguments."""
@classmethod
def get_workload(self):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
impalad_args="--buffer_pool_limit=2g --memory_maintenance_sleep_time_ms=100")

View File

@@ -27,10 +27,6 @@ from tests.util.hive_utils import HiveDbWrapper
class TestMetadataReplicas(CustomClusterTestSuite):
""" Validates metadata content across catalogd and impalad coordinators."""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -38,10 +38,6 @@ def impalad_admission_ctrl_maxmtdop_args():
class TestMtDopFlags(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestMtDopFlags, cls).add_test_dimensions()
@@ -80,10 +76,6 @@ class TestMtDopFlags(CustomClusterTestSuite):
class TestMaxMtDop(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestMaxMtDop, cls).add_test_dimensions()

View File

@@ -25,10 +25,6 @@ from tests.util.parse_util import parse_duration_string_ms
class TestObservability(CustomClusterTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@pytest.mark.execute_serially
def test_host_profile_jvm_gc_metrics(self, unique_database):
self.execute_query_expect_success(self.client,

View File

@@ -34,10 +34,6 @@ class TestParquetMaxPageHeader(CustomClusterTestSuite):
adjust --max_page_header_size, which is the maximum bytes of header data that the
scanner reads before it bails out.
'''
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -33,10 +33,6 @@ LOG = logging.getLogger(__name__)
class TestPartition(CustomClusterTestSuite):
"""Tests to validate partitioning"""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestPartition, cls).add_test_dimensions()
@@ -85,10 +81,6 @@ class TestPartitionDeletion(CustomClusterTestSuite):
Use a normal catalog update frequency (2s) instead of the default one in custom
cluster tests (50ms) so the race conditions of IMPALA-13009 could happen."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestPartitionDeletion, cls).add_test_dimensions()
cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())

View File

@@ -39,10 +39,6 @@ class TestUdfPersistence(CustomClusterTestSuite):
JAVA_UDF_JAR = os.getenv('DEFAULT_FS') + '/test-warehouse/impala-hive-udfs.jar'
LOCAL_LIBRARY_DIR = "udf_persistence"
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -26,10 +26,6 @@ class TestPreLoadTableTypes(CustomClusterTestSuite):
types and comments loaded.
"""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
super(TestPreLoadTableTypes, cls).setup_class()

View File

@@ -60,10 +60,6 @@ class TestHooksStartupFail(CustomClusterTestSuite):
All test cases in this testsuite are expected to fail cluster startup and will
swallow exceptions thrown during setup_method().
"""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -53,10 +53,6 @@ class TestQueryLogTableBasic(WorkloadManagementTestSuite):
super(TestQueryLogTableBasic, cls).add_test_dimensions()
cls.ImpalaTestMatrix.add_dimension(hs2_client_protocol_dimension())
@classmethod
def get_workload(self):
return 'functional-query'
MAX_SQL_PLAN_LEN = 2000
LOG_DIR_MAX_WRITES = 'max_attempts_exceeded'

View File

@@ -84,10 +84,6 @@ class TestQueryRetries(CustomClusterTestSuite):
_count_query = "select count(*) from tpch_parquet.lineitem where l_orderkey < 50"
_count_query_result = "55"
@classmethod
def get_workload(cls):
return 'functional-query'
@pytest.mark.execute_serially
def test_retries_from_cancellation_pool(self):
"""Tests that queries are retried instead of cancelled if one of the nodes leaves the
@@ -1190,10 +1186,6 @@ class TestQueryRetries(CustomClusterTestSuite):
# query.
@SkipIfNotHdfsMinicluster.tuned_for_minicluster
class TestQueryRetriesFaultyDisk(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -39,10 +39,6 @@ class TestRedaction(CustomClusterTestSuite):
limited to table data and query text since queries may refer to table data.
'''
@classmethod
def get_workload(cls):
return 'functional-query'
@property
def log_dir(self):
return os.path.join(self.tmp_dir, "logs")

View File

@@ -22,9 +22,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
class TestRefreshInvalidPartition(CustomClusterTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(

View File

@@ -44,10 +44,6 @@ LOG = logging.getLogger(__name__)
class TestRestart(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@pytest.mark.execute_serially
def test_restart_statestore(self, cursor):
""" Regression test of IMPALA-6973. After the statestore restarts, the metadata should
@@ -527,10 +523,6 @@ class TestGracefulShutdown(CustomClusterTestSuite, HS2TestSuite):
IDLE_SHUTDOWN_GRACE_PERIOD_S = 1
IMPALA_SHUTDOWN_SIGNAL = signal.SIGRTMIN
@classmethod
def get_workload(cls):
return 'functional-query'
@SkipIfFS.shutdown_idle_fails
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(

View File

@@ -33,10 +33,6 @@ class TestDedicatedCoordinator(CustomClusterTestSuite):
"""A custom cluster test that tests result spooling against a cluster with a dedicated
coordinator."""
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestDedicatedCoordinator, cls).add_test_dimensions()

View File

@@ -62,10 +62,6 @@ class TestRPCException(CustomClusterTestSuite):
where t1.l_orderkey = t2.l_orderkey"
EXPECTED_RESULT = ['30012985']
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -43,10 +43,6 @@ class TestRPCTimeout(CustomClusterTestSuite):
# Designed to take approx. 30s.
SLOW_TEST_QUERY = TEST_QUERY + " where c2.int_col = sleep(1000)"
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':
@@ -238,10 +234,6 @@ class TestRPCTimeout(CustomClusterTestSuite):
class TestCatalogRPCTimeout(CustomClusterTestSuite):
""""Tests RPC timeout and retry handling for catalogd operations."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -38,10 +38,6 @@ build_runs_slowly = ImpalaTestClusterProperties.get_instance().runs_slowly()
class TestRuntimeFilterAggregation(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestRuntimeFilterAggregation, cls).add_test_dimensions()
@@ -77,10 +73,6 @@ class TestLateQueryStateInit(CustomClusterTestSuite):
_wait_time = WAIT_TIME_MS // 20
_init_delay = [100, _wait_time]
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestLateQueryStateInit, cls).add_test_dimensions()

View File

@@ -31,10 +31,6 @@ TOTAL_ASSIGNMENTS_METRIC = "simple-scheduler.assignments.total"
class TestSchedulerLocality(CustomClusterTestSuite):
"""Tests for local and remote disk scheduling."""
@classmethod
def get_workload(cls):
return 'functional-query'
@CustomClusterTestSuite.with_args(
impalad_args='--hostname=localhost', cluster_size=1)
def test_local_assignment(self, vector):

View File

@@ -34,10 +34,6 @@ from tests.verifiers.metric_verifier import MetricVerifier
class TestScratchDir(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -30,10 +30,6 @@ class TestImpala3798(CustomClusterTestSuite):
The debug flag --skip_file_runtime_filtering disables per-file filtering, mimicing the
race that leads to the hang.
"""
@classmethod
def get_workload(cls):
return 'functional-query'
@SkipIfBuildType.not_dev_build
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args("--skip_file_runtime_filtering=true")

View File

@@ -23,10 +23,6 @@ from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
class TestStatestoreRpcErrors(CustomClusterTestSuite):
"""Tests for statestore RPC handling."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':
@@ -60,10 +56,6 @@ class TestStatestoreRpcErrors(CustomClusterTestSuite):
class TestCatalogRpcErrors(CustomClusterTestSuite):
"""Tests for catalog RPC handling."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -36,10 +36,6 @@ class TestImpalaShellCommandLine(CustomClusterTestSuite):
LOG_DIR_HTTP_TRACING_OFF = "http_tracing_off"
IMPALA_ID_RE = "([0-9a-f]{16}:[0-9a-f]{16})"
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
"""Overrides all other add_dimension methods in super classes up the entire class

View File

@@ -35,10 +35,6 @@ NUM_QUERIES = 'impala-server.num-queries'
class TestShellInteractiveReconnect(CustomClusterTestSuite):
""" Check if interactive shell is using the current DB after reconnecting """
@classmethod
def get_workload(cls):
return 'functional-query'
@pytest.mark.execute_serially
def test_manual_reconnect(self):
# Iterate over test vector within test function to avoid restarting cluster.

View File

@@ -50,10 +50,6 @@ class TestImpalaShellJWTAuth(CustomClusterTestSuite):
# Name of the Impala metric containing the total count of hs2-http connections opened.
HS2_HTTP_CONNS = "impala.thrift-server.hiveserver2-http-frontend.total-connections"
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
"""Overrides all other add_dimension methods in super classes up the entire class

View File

@@ -49,10 +49,6 @@ class TestImpalaShellOAuthAuth(CustomClusterTestSuite):
# Name of the Impala metric containing the total count of hs2-http connections opened.
HS2_HTTP_CONNS = "impala.thrift-server.hiveserver2-http-frontend.total-connections"
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
"""Overrides all other add_dimension methods in super classes up the entire class

View File

@@ -28,10 +28,6 @@ PULL_TABLE_TYPES_FLAG = "--pull_table_types_and_comments=true"
class TestShowViewsStatements(CustomClusterTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@CustomClusterTestSuite.with_args(
catalogd_args=PULL_TABLE_TYPES_FLAG)
def test_show_views(self, vector):

View File

@@ -44,9 +44,6 @@ class TestStatestoredHA(CustomClusterTestSuite):
All cluster components are started with starting flag FLAGS_enable_statestored_ha
as true."""
def get_workload(self):
return 'functional-query'
def __disable_statestored_network(self,
service_port=DEFAULT_STATESTORE_SERVICE_PORT, disable_network=False):
request = Subscriber.TSetStatestoreDebugActionRequest(
@@ -817,10 +814,6 @@ class TestStatestoredHAStartupDelay(CustomClusterTestSuite):
expected to be able to tolerate this delay with FLAGS_tolerate_statestore_startup_delay
set as true. This is not testing anything beyond successful startup."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -28,10 +28,6 @@ class TestStatsExtrapolation(CustomClusterTestSuite):
primarly checks that the flag is propagated to the FE. More testing is done in FE unit
tests and metadata/test_stats_extrapolation.py."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestStatsExtrapolation, cls).add_test_dimensions()

View File

@@ -72,10 +72,6 @@ class TestThriftSocket(CustomClusterTestSuite):
pytest.skip("Python version does not support tls 1.2")
super(TestThriftSocket, cls).setup_class()
@classmethod
def get_workload(cls):
return 'functional-query'
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
impalad_args=IDLE_ARGS, cluster_size=1, disable_log_buffering=True)

View File

@@ -119,10 +119,6 @@ class TestTupleCacheBase(CustomClusterTestSuite):
os.environ["TUPLE_CACHE_DIR"] = cls.org_tuple_cache_dir
super(TestTupleCacheBase, cls).teardown_class()
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestTupleCacheBase, cls).add_test_dimensions()

View File

@@ -35,10 +35,6 @@ from tests.util.filesystem_utils import get_fs_path
class TestUdfConcurrency(CustomClusterTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -36,10 +36,6 @@ CATALOG_URL = "http://localhost:25020/catalog"
class TestWebPage(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -29,10 +29,6 @@ NUM_PARTS = 50000
@SkipIf.not_hdfs
class TestWideTableOperations(CustomClusterTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def setup_class(cls):
if cls.exploration_strategy() != 'exhaustive':

View File

@@ -49,10 +49,6 @@ class TestWorkloadManagementInitBase(CustomClusterTestSuite):
This test class does not extend WorkloadManagementTestSuite because its subclasses
define its own setup_method and teardown_method."""
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def default_test_protocol(cls):
return HS2

View File

@@ -42,10 +42,6 @@ class TestDataErrors(ImpalaTestSuite):
cls.ImpalaTestMatrix.add_dimension(
create_exec_option_dimension(batch_sizes=cls.BATCH_SIZES))
@classmethod
def get_workload(self):
return 'functional-query'
# Regression test for IMP-633. Added as a part of IMPALA-5198.
@SkipIf.not_dfs
@@ -156,9 +152,6 @@ class TestHdfsRcFileScanNodeErrors(TestHdfsScanNodeErrors):
class TestBinaryTypeInText(ImpalaTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):

View File

@@ -64,10 +64,6 @@ QUERIES = [
@SkipIf.skip_hbase # -skip_hbase argument specified
@SkipIfFS.hbase # missing coverage: failures
class TestFailpoints(ImpalaTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestFailpoints, cls).add_test_dimensions()

View File

@@ -36,10 +36,6 @@ from tests.util.filesystem_utils import IS_LOCAL
class TestStressInfra(ImpalaTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
def test_stress_binary_search_start_point(self):
"""
Test that the stress test can use EXPLAIN to find the start point for its binary

View File

@@ -25,10 +25,6 @@ from tests.common.test_dimensions import create_client_protocol_dimension
class TestColumnUnicode(ImpalaTestSuite):
@classmethod
def get_workload(cls):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestColumnUnicode, cls).add_test_dimensions()

View File

@@ -39,10 +39,6 @@ IMPALA_TEST_CLUSTER_PROPERTIES = ImpalaTestClusterProperties.get_instance()
# Tests the COMPUTE STATS command for gathering table and column stats.
class TestComputeStats(ImpalaTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestComputeStats, cls).add_test_dimensions()
@@ -333,10 +329,6 @@ class TestComputeStats(ImpalaTestSuite):
# because we want to use the existing machanism to disable running tests on hbase/none
# based on the filesystem type (S3, Isilon, etc.).
class TestHbaseComputeStats(ImpalaTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestHbaseComputeStats, cls).add_test_dimensions()
@@ -359,10 +351,6 @@ class TestHbaseComputeStats(ImpalaTestSuite):
class TestCorruptTableStats(ImpalaTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestCorruptTableStats, cls).add_test_dimensions()
@@ -379,10 +367,6 @@ class TestCorruptTableStats(ImpalaTestSuite):
class TestIncompatibleColStats(ImpalaTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestIncompatibleColStats, cls).add_test_dimensions()
@@ -425,10 +409,6 @@ class TestIncompatibleColStats(ImpalaTestSuite):
# Test column min/max stats currently enabled for Parquet tables.
class TestParquetComputeColumnMinMax(ImpalaTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestParquetComputeColumnMinMax, cls).add_test_dimensions()
@@ -441,10 +421,6 @@ class TestParquetComputeColumnMinMax(ImpalaTestSuite):
class TestInvalidStatsFromHms(ImpalaTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestInvalidStatsFromHms, cls).add_test_dimensions()

View File

@@ -933,10 +933,6 @@ class TestDdlStatements(TestDdlBase):
# IMPALA-10811: RPC to submit query getting stuck for AWS NLB forever
# Test HS2, Beeswax and HS2-HTTP three clients.
class TestAsyncDDL(TestDdlBase):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestAsyncDDL, cls).add_test_dimensions()
@@ -1010,10 +1006,6 @@ class TestAsyncDDL(TestDdlBase):
class TestAsyncDDLTiming(TestDdlBase):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestAsyncDDLTiming, cls).add_test_dimensions()
@@ -1156,10 +1148,6 @@ class TestDdlLogs(TestDdlBase):
# IMPALA-2002: Tests repeated adding/dropping of .jar and .so in the lib cache.
class TestLibCache(TestDdlBase):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestLibCache, cls).add_test_dimensions()

View File

@@ -26,10 +26,6 @@ from tests.util.filesystem_utils import WAREHOUSE
# Base class that most DDL tests inherit from. The tests have a few common functions,
# as well as test dimensions and setup/teardown.
class TestDdlBase(ImpalaTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestDdlBase, cls).add_test_dimensions()

View File

@@ -32,10 +32,6 @@ class TestExplain(ImpalaTestSuite):
# scan nodes are consistent even when run on machines with different numbers of cores.
NUM_SCANNER_THREADS = 1
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestExplain, cls).add_test_dimensions()

View File

@@ -44,10 +44,6 @@ class TestHdfsEncryption(ImpalaTestSuite):
zones with different keys, and the LOAD DATA command is executed. The
tests operate on both partitioned and non-partitioned tables.
'''
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestHdfsEncryption, cls).add_test_dimensions()

View File

@@ -31,10 +31,6 @@ TBL_LOC = '%s/%s' % (WAREHOUSE, TEST_TBL)
@SkipIfFS.hdfs_acls
@SkipIfLocal.hdfs_client
class TestHdfsPermissions(ImpalaTestSuite):
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestHdfsPermissions, cls).add_test_dimensions()

View File

@@ -33,10 +33,6 @@ class TestHiddenFiles(ImpalaTestSuite):
# The .test file run in these tests relies this table name.
TBL_NAME = "test_hidden_files"
@classmethod
def get_workload(self):
return 'functional-query'
@classmethod
def add_test_dimensions(cls):
super(TestHiddenFiles, cls).add_test_dimensions()

Some files were not shown because too many files have changed in this diff Show More