You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jb...@apache.org on 2017/01/27 00:54:13 UTC

[5/6] incubator-impala git commit: IMPALA-4750: Rename test infra classes so they don't mimic test classes.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_hash_join_timer.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_hash_join_timer.py b/tests/query_test/test_hash_join_timer.py
index d570657..f3d4d79 100644
--- a/tests/query_test/test_hash_join_timer.py
+++ b/tests/query_test/test_hash_join_timer.py
@@ -19,7 +19,7 @@ import pytest
 import re
 
 from tests.common.impala_test_suite import ImpalaTestSuite
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 
 class TestHashJoinTimer(ImpalaTestSuite):
   """Tests that the local time in hash join is correct in the ExecSummary, average
@@ -74,8 +74,9 @@ class TestHashJoinTimer(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestHashJoinTimer, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(TestDimension('test cases', *cls.TEST_CASES))
-    cls.TestMatrix.add_constraint(lambda v: cls.__is_valid_test_vector(v))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('test cases', *cls.TEST_CASES))
+    cls.ImpalaTestMatrix.add_constraint(lambda v: cls.__is_valid_test_vector(v))
 
   @classmethod
   def __is_valid_test_vector(cls, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_hbase_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_hbase_queries.py b/tests/query_test/test_hbase_queries.py
index 7ed1533..8fbdfb0 100644
--- a/tests/query_test/test_hbase_queries.py
+++ b/tests/query_test/test_hbase_queries.py
@@ -29,7 +29,7 @@ class TestHBaseQueries(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestHBaseQueries, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(\
+    cls.ImpalaTestMatrix.add_constraint(\
         lambda v: v.get_value('table_format').file_format == 'hbase')
 
   def test_hbase_scan_node(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_hdfs_caching.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_hdfs_caching.py b/tests/query_test/test_hdfs_caching.py
index 5119625..c0f9cfb 100644
--- a/tests/query_test/test_hdfs_caching.py
+++ b/tests/query_test/test_hdfs_caching.py
@@ -41,9 +41,9 @@ class TestHdfsCaching(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestHdfsCaching, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('exec_option')['batch_size'] == 0)
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == "text")
 
   # The tpch nation table is cached as part of data loading. We'll issue a query
@@ -174,9 +174,9 @@ class TestHdfsCachingDdl(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestHdfsCachingDdl, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
 
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text' and \
         v.get_value('table_format').compression_codec == 'none')
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_hdfs_fd_caching.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_hdfs_fd_caching.py b/tests/query_test/test_hdfs_fd_caching.py
index 83eeab4..4d72714 100644
--- a/tests/query_test/test_hdfs_fd_caching.py
+++ b/tests/query_test/test_hdfs_fd_caching.py
@@ -39,7 +39,7 @@ class TestHdfsFdCaching(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestHdfsFdCaching, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(cls.file_format_constraint)
+    cls.ImpalaTestMatrix.add_constraint(cls.file_format_constraint)
 
   @classmethod
   def get_workload(cls):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_insert.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_insert.py b/tests/query_test/test_insert.py
index d195050..2c14312 100644
--- a/tests/query_test/test_insert.py
+++ b/tests/query_test/test_insert.py
@@ -30,7 +30,7 @@ from tests.common.test_result_verifier import (
     parse_column_labels,
     QueryTestResult,
     parse_result_rows)
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 
 # TODO: Add Gzip back.  IMPALA-424
 PARQUET_CODECS = ['none', 'snappy']
@@ -49,28 +49,30 @@ class TestInsertQueries(ImpalaTestSuite):
     # TODO: When we do decide to run these tests in parallel we could create unique temp
     # tables for each test case to resolve the concurrency problems.
     if cls.exploration_strategy() == 'core':
-      cls.TestMatrix.add_dimension(create_exec_option_dimension(
+      cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
           cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0],
           sync_ddl=[0]))
-      cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
+      cls.ImpalaTestMatrix.add_dimension(
+          create_uncompressed_text_dimension(cls.get_workload()))
     else:
-      cls.TestMatrix.add_dimension(create_exec_option_dimension(
+      cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
           cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0, 1, 16],
           sync_ddl=[0, 1]))
-      cls.TestMatrix.add_dimension(TestDimension("compression_codec", *PARQUET_CODECS));
+      cls.ImpalaTestMatrix.add_dimension(
+          ImpalaTestDimension("compression_codec", *PARQUET_CODECS));
       # Insert is currently only supported for text and parquet
       # For parquet, we want to iterate through all the compression codecs
       # TODO: each column in parquet can have a different codec.  We could
       # test all the codecs in one table/file with some additional flags.
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').file_format == 'parquet' or \
             (v.get_value('table_format').file_format == 'text' and \
             v.get_value('compression_codec') == 'none'))
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').compression_codec == 'none')
       # Only test other batch sizes for uncompressed parquet to keep the execution time
       # within reasonable bounds.
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('exec_option')['batch_size'] == 0 or \
             (v.get_value('table_format').file_format == 'parquet' and \
             v.get_value('compression_codec') == 'none'))
@@ -131,21 +133,21 @@ class TestInsertWideTable(ImpalaTestSuite):
     super(TestInsertWideTable, cls).add_test_dimensions()
 
     # Only vary codegen
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[True, False], batch_sizes=[0]))
 
     # Inserts only supported on text and parquet
     # TODO: Enable 'text'/codec once the compressed text writers are in.
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'parquet' or \
         v.get_value('table_format').file_format == 'text')
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').compression_codec == 'none')
 
     # Don't run on core. This test is very slow (IMPALA-864) and we are unlikely to
     # regress here.
     if cls.exploration_strategy() == 'core':
-      cls.TestMatrix.add_constraint(lambda v: False);
+      cls.ImpalaTestMatrix.add_constraint(lambda v: False);
 
   @SkipIfLocal.parquet_file_size
   def test_insert_wide_table(self, vector, unique_database):
@@ -189,13 +191,13 @@ class TestInsertPartKey(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestInsertPartKey, cls).add_test_dimensions()
     # Only run for a single table type
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0],
         sync_ddl=[1]))
 
-    cls.TestMatrix.add_constraint(lambda v:
+    cls.ImpalaTestMatrix.add_constraint(lambda v:
         (v.get_value('table_format').file_format == 'text'))
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').compression_codec == 'none')
 
   @pytest.mark.execute_serially
@@ -217,12 +219,12 @@ class TestInsertNullQueries(ImpalaTestSuite):
     # into the same table at the same time for the same file format).
     # TODO: When we do decide to run these tests in parallel we could create unique temp
     # tables for each test case to resolve the concurrency problems.
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0]))
 
     # These tests only make sense for inserting into a text table with special
     # logic to handle all the possible ways NULL needs to be written as ascii
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
           (v.get_value('table_format').file_format == 'text' and \
            v.get_value('table_format').compression_codec == 'none'))
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_insert_parquet.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_insert_parquet.py b/tests/query_test/test_insert_parquet.py
index d5b8c17..b7cb285 100644
--- a/tests/query_test/test_insert_parquet.py
+++ b/tests/query_test/test_insert_parquet.py
@@ -29,7 +29,7 @@ from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.parametrize import UniqueDatabase
 from tests.common.skip import SkipIfIsilon, SkipIfLocal
 from tests.common.test_dimensions import create_exec_option_dimension
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 from tests.util.filesystem_utils import get_fs_path, WAREHOUSE
 
 # TODO: Add Gzip back.  IMPALA-424
@@ -52,16 +52,18 @@ class TestInsertParquetQueries(ImpalaTestSuite):
     # into the same table at the same time for the same file format).
     # TODO: When we do decide to run these tests in parallel we could create unique temp
     # tables for each test case to resolve the concurrency problems.
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0],
         sync_ddl=[1]))
 
-    cls.TestMatrix.add_dimension(TestDimension("compression_codec", *PARQUET_CODECS));
-    cls.TestMatrix.add_dimension(TestDimension("file_size", *PARQUET_FILE_SIZES));
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension("compression_codec", *PARQUET_CODECS));
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension("file_size", *PARQUET_FILE_SIZES));
 
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'parquet')
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').compression_codec == 'none')
 
   @SkipIfLocal.multiple_impalad
@@ -82,13 +84,14 @@ class TestInsertParquetInvalidCodec(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestInsertParquetInvalidCodec, cls).add_test_dimensions()
     # Fix the exec_option vector to have a single value.
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0],
         sync_ddl=[1]))
-    cls.TestMatrix.add_dimension(TestDimension("compression_codec", 'bzip2'));
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension("compression_codec", 'bzip2'));
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'parquet')
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').compression_codec == 'none')
 
   @SkipIfLocal.multiple_impalad
@@ -108,14 +111,15 @@ class TestInsertParquetVerifySize(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestInsertParquetVerifySize, cls).add_test_dimensions()
     # Fix the exec_option vector to have a single value.
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0],
         sync_ddl=[1]))
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'parquet')
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').compression_codec == 'none')
-    cls.TestMatrix.add_dimension(TestDimension("compression_codec", *PARQUET_CODECS));
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension("compression_codec", *PARQUET_CODECS));
 
   @SkipIfIsilon.hdfs_block_size
   @SkipIfLocal.hdfs_client
@@ -156,7 +160,7 @@ class TestHdfsParquetTableWriter(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestHdfsParquetTableWriter, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(
+    cls.ImpalaTestMatrix.add_constraint(
         lambda v: v.get_value('table_format').file_format == 'parquet')
 
   def test_def_level_encoding(self, vector, unique_database):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_insert_permutation.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_insert_permutation.py b/tests/query_test/test_insert_permutation.py
index 4e9dda8..46d1090 100644
--- a/tests/query_test/test_insert_permutation.py
+++ b/tests/query_test/test_insert_permutation.py
@@ -40,9 +40,10 @@ class TestInsertQueriesWithPermutation(ImpalaTestSuite):
     # tables for each test case to resolve the concurrency problems.
     # TODO: do we need to run with multiple file formats? This seems to be really
     # targeting FE behavior.
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0]))
-    cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
+    cls.ImpalaTestMatrix.add_dimension(
+        create_uncompressed_text_dimension(cls.get_workload()))
 
   def test_insert_permutation(self, vector):
     map(self.cleanup_db, ["insert_permutation_test"])

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_invalid_test_header.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_invalid_test_header.py b/tests/query_test/test_invalid_test_header.py
index 2ebeb8a..6b7b3bd 100644
--- a/tests/query_test/test_invalid_test_header.py
+++ b/tests/query_test/test_invalid_test_header.py
@@ -25,7 +25,7 @@ class TestInvalidTestHeader(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestInvalidTestHeader, cls).add_test_dimensions()
     # It's sufficient to run this test once.
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text' and\
         v.get_value('table_format').compression_codec == 'none')
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_join_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_join_queries.py b/tests/query_test/test_join_queries.py
index 93fcba5..631db00 100644
--- a/tests/query_test/test_join_queries.py
+++ b/tests/query_test/test_join_queries.py
@@ -27,7 +27,7 @@ from tests.common.skip import (
     SkipIfLocal,
     SkipIfOldAggsJoins,
     SkipIfS3)
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 
 class TestJoinQueries(ImpalaTestSuite):
   BATCH_SIZES = [0, 1]
@@ -39,16 +39,16 @@ class TestJoinQueries(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestJoinQueries, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(
-        TestDimension('batch_size', *TestJoinQueries.BATCH_SIZES))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('batch_size', *TestJoinQueries.BATCH_SIZES))
     # TODO: Look into splitting up join tests to accomodate hbase.
     # Joins with hbase tables produce drastically different results.
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format in ['parquet'])
 
     if cls.exploration_strategy() != 'exhaustive':
       # Cut down on execution time when not running in exhaustive mode.
-      cls.TestMatrix.add_constraint(lambda v: v.get_value('batch_size') != 1)
+      cls.ImpalaTestMatrix.add_constraint(lambda v: v.get_value('batch_size') != 1)
 
   def test_basic_joins(self, vector):
     new_vector = copy(vector)
@@ -102,14 +102,14 @@ class TestTPCHJoinQueries(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestTPCHJoinQueries, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(
-        TestDimension('batch_size', *TestJoinQueries.BATCH_SIZES))
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('batch_size', *TestJoinQueries.BATCH_SIZES))
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format in ['parquet'])
 
     if cls.exploration_strategy() != 'exhaustive':
       # Cut down on execution time when not running in exhaustive mode.
-      cls.TestMatrix.add_constraint(lambda v: v.get_value('batch_size') != 1)
+      cls.ImpalaTestMatrix.add_constraint(lambda v: v.get_value('batch_size') != 1)
 
   @classmethod
   def teardown_class(cls):
@@ -129,15 +129,15 @@ class TestSemiJoinQueries(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestSemiJoinQueries, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(
-        TestDimension('batch_size', *TestJoinQueries.BATCH_SIZES))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('batch_size', *TestJoinQueries.BATCH_SIZES))
     # Joins with hbase tables produce drastically different results.
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format in ['parquet'])
 
     if cls.exploration_strategy() != 'exhaustive':
       # Cut down on execution time when not running in exhaustive mode.
-      cls.TestMatrix.add_constraint(lambda v: v.get_value('batch_size') != 1)
+      cls.ImpalaTestMatrix.add_constraint(lambda v: v.get_value('batch_size') != 1)
 
   def __load_semi_join_tables(self, db_name):
     # Create and load fresh test tables for semi/anti-join tests

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_limit.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_limit.py b/tests/query_test/test_limit.py
index fe1ff5b..3224299 100644
--- a/tests/query_test/test_limit.py
+++ b/tests/query_test/test_limit.py
@@ -19,7 +19,7 @@
 
 from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
 from tests.common.impala_test_suite import ImpalaTestSuite
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 
 class TestLimit(ImpalaTestSuite):
   LIMIT_VALUES = [1, 2, 3, 4, 5, 10, 100, 5000]
@@ -40,20 +40,20 @@ class TestLimit(ImpalaTestSuite):
 
     # Add two more dimensions
     if cls.exploration_strategy() == 'core':
-      cls.TestMatrix.add_dimension(
-          TestDimension('limit_value', *TestLimit.LIMIT_VALUES_CORE))
+      cls.ImpalaTestMatrix.add_dimension(
+          ImpalaTestDimension('limit_value', *TestLimit.LIMIT_VALUES_CORE))
     else:
-      cls.TestMatrix.add_dimension(
-          TestDimension('limit_value', *TestLimit.LIMIT_VALUES))
-    cls.TestMatrix.add_dimension(TestDimension('query', *TestLimit.QUERIES))
+      cls.ImpalaTestMatrix.add_dimension(
+          ImpalaTestDimension('limit_value', *TestLimit.LIMIT_VALUES))
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('query', *TestLimit.QUERIES))
 
     # Don't run with large limits and tiny batch sizes.  This generates excessive
     # network traffic and makes the machine run very slowly.
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('limit_value') < 100 or v.get_value('exec_option')['batch_size'] == 0)
     # TPCH is not generated in hbase format.
     # TODO: Add test coverage for hbase.
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format != "hbase")
 
   def test_limit(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_local_fs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_local_fs.py b/tests/query_test/test_local_fs.py
index 5123290..0eb218b 100644
--- a/tests/query_test/test_local_fs.py
+++ b/tests/query_test/test_local_fs.py
@@ -28,9 +28,9 @@ class TestLocalFileSystem(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestLocalFileSystem, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
 
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text' and \
         v.get_value('table_format').compression_codec == 'none')
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_mem_usage_scaling.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_mem_usage_scaling.py b/tests/query_test/test_mem_usage_scaling.py
index df41d38..65c86e6 100644
--- a/tests/query_test/test_mem_usage_scaling.py
+++ b/tests/query_test/test_mem_usage_scaling.py
@@ -22,7 +22,7 @@ from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfLocal
 from tests.common.test_dimensions import create_single_exec_option_dimension
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 
 class TestQueryMemLimitScaling(ImpalaTestSuite):
   """Test class to do functional validation of per query memory limits. """
@@ -41,10 +41,11 @@ class TestQueryMemLimitScaling(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestQueryMemLimitScaling, cls).add_test_dimensions()
     # add mem_limit as a test dimension.
-    new_dimension = TestDimension('mem_limit', *TestQueryMemLimitScaling.MEM_LIMITS)
-    cls.TestMatrix.add_dimension(new_dimension)
+    new_dimension = ImpalaTestDimension('mem_limit',
+                                        *TestQueryMemLimitScaling.MEM_LIMITS)
+    cls.ImpalaTestMatrix.add_dimension(new_dimension)
     if cls.exploration_strategy() != 'exhaustive':
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').file_format in ['parquet'])
 
   # Test running with different mem limits to exercise the dynamic memory
@@ -65,9 +66,9 @@ class TestExprMemUsage(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestExprMemUsage, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
     if cls.exploration_strategy() != 'exhaustive':
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').file_format in ['parquet'])
 
   def test_scanner_mem_usage(self, vector):
@@ -123,10 +124,10 @@ class TestTpchMemLimitError(TestLowMemoryLimits):
   def add_test_dimensions(cls):
     super(TestTpchMemLimitError, cls).add_test_dimensions()
 
-    cls.TestMatrix.add_dimension(
-      TestDimension('mem_limit', *TestTpchMemLimitError.MEM_IN_MB))
+    cls.ImpalaTestMatrix.add_dimension(
+      ImpalaTestDimension('mem_limit', *TestTpchMemLimitError.MEM_IN_MB))
 
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format in ['parquet'])
 
   def test_low_mem_limit_q1(self, vector):
@@ -215,10 +216,10 @@ class TestTpcdsMemLimitError(TestLowMemoryLimits):
   def add_test_dimensions(cls):
     super(TestTpcdsMemLimitError, cls).add_test_dimensions()
 
-    cls.TestMatrix.add_dimension(
-      TestDimension('mem_limit', *TestTpcdsMemLimitError.MEM_IN_MB))
+    cls.ImpalaTestMatrix.add_dimension(
+      ImpalaTestDimension('mem_limit', *TestTpcdsMemLimitError.MEM_IN_MB))
 
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format in ['parquet'])
 
   def test_low_mem_limit_q53(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_mt_dop.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_mt_dop.py b/tests/query_test/test_mt_dop.py
index 237e039..5dddfc0 100644
--- a/tests/query_test/test_mt_dop.py
+++ b/tests/query_test/test_mt_dop.py
@@ -22,8 +22,8 @@ import pytest
 from copy import deepcopy
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfOldAggsJoins
-from tests.common.test_vector import TestDimension
-from tests.common.test_vector import TestVector
+from tests.common.test_vector import ImpalaTestDimension
+from tests.common.test_vector import ImpalaTestVector
 
 # COMPUTE STATS on Parquet tables automatically sets MT_DOP=4, so include
 # the value 0 to cover the non-MT path as well.
@@ -33,7 +33,7 @@ class TestMtDop(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestMtDop, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(TestDimension('mt_dop', *MT_DOP_VALUES))
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('mt_dop', *MT_DOP_VALUES))
 
   @classmethod
   def get_workload(cls):
@@ -84,8 +84,8 @@ class TestMtDopParquet(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestMtDopParquet, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(TestDimension('mt_dop', *MT_DOP_VALUES))
-    cls.TestMatrix.add_constraint(
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('mt_dop', *MT_DOP_VALUES))
+    cls.ImpalaTestMatrix.add_constraint(
         lambda v: v.get_value('table_format').file_format == 'parquet')
 
   def test_parquet(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_multiple_filesystems.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_multiple_filesystems.py b/tests/query_test/test_multiple_filesystems.py
index 7ede766..2e97821 100644
--- a/tests/query_test/test_multiple_filesystems.py
+++ b/tests/query_test/test_multiple_filesystems.py
@@ -38,9 +38,9 @@ class TestMultipleFilesystems(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestMultipleFilesystems, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
 
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text' and \
         v.get_value('table_format').compression_codec == 'none')
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_nested_types.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_nested_types.py b/tests/query_test/test_nested_types.py
index 55be260..59fcced 100644
--- a/tests/query_test/test_nested_types.py
+++ b/tests/query_test/test_nested_types.py
@@ -34,7 +34,7 @@ class TestNestedTypes(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestNestedTypes, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:
+    cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format == 'parquet')
 
   def test_scanner_basic(self, vector):
@@ -87,7 +87,7 @@ class TestParquetArrayEncodings(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestParquetArrayEncodings, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:
+    cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format == 'parquet')
 
   # $ parquet-tools schema SingleFieldGroupInList.parquet
@@ -450,7 +450,7 @@ class TestMaxNestingDepth(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestMaxNestingDepth, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:
+    cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format == 'parquet')
 
   def test_max_nesting_depth(self, vector, unique_database):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_partitioning.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_partitioning.py b/tests/query_test/test_partitioning.py
index 08dcbdd..04d6c3a 100644
--- a/tests/query_test/test_partitioning.py
+++ b/tests/query_test/test_partitioning.py
@@ -31,10 +31,10 @@ class TestPartitioning(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestPartitioning, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
 
     # There is no reason to run these tests using all dimensions.
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text' and\
         v.get_value('table_format').compression_codec == 'none')
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_queries.py b/tests/query_test/test_queries.py
index 2218398..d8e1f65 100644
--- a/tests/query_test/test_queries.py
+++ b/tests/query_test/test_queries.py
@@ -22,14 +22,14 @@ import pytest
 
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.test_dimensions import create_uncompressed_text_dimension
-from tests.common.test_vector import TestVector
+from tests.common.test_vector import ImpalaTestVector
 
 class TestQueries(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestQueries, cls).add_test_dimensions()
     if cls.exploration_strategy() == 'core':
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').file_format == 'parquet')
 
     # Manually adding a test dimension here to test the small query opt
@@ -37,13 +37,13 @@ class TestQueries(ImpalaTestSuite):
     # TODO Cleanup required, allow adding values to dimensions without having to
     # manually explode them
     if cls.exploration_strategy() == 'exhaustive':
-      dim = cls.TestMatrix.dimensions["exec_option"]
+      dim = cls.ImpalaTestMatrix.dimensions["exec_option"]
       new_value = []
       for v in dim:
-        new_value.append(TestVector.Value(v.name, copy.copy(v.value)))
+        new_value.append(ImpalaTestVector.Value(v.name, copy.copy(v.value)))
         new_value[-1].value["exec_single_node_rows_threshold"] = 100
       dim.extend(new_value)
-      cls.TestMatrix.add_dimension(dim)
+      cls.ImpalaTestMatrix.add_dimension(dim)
 
   @classmethod
   def get_workload(cls):
@@ -134,7 +134,8 @@ class TestQueriesTextTables(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestQueriesTextTables, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
+    cls.ImpalaTestMatrix.add_dimension(
+        create_uncompressed_text_dimension(cls.get_workload()))
 
   @classmethod
   def get_workload(cls):
@@ -172,7 +173,7 @@ class TestQueriesParquetTables(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestQueriesParquetTables, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'parquet')
 
   @classmethod
@@ -201,7 +202,7 @@ class TestHdfsQueries(TestQueries):
   def add_test_dimensions(cls):
     super(TestHdfsQueries, cls).add_test_dimensions()
     # Kudu doesn't support AllTypesAggMultiFilesNoPart (KUDU-1271, KUDU-1570).
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format != 'kudu')
 
   def test_hdfs_scan_node(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_query_mem_limit.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_query_mem_limit.py b/tests/query_test/test_query_mem_limit.py
index 161f714..2fdd6eb 100644
--- a/tests/query_test/test_query_mem_limit.py
+++ b/tests/query_test/test_query_mem_limit.py
@@ -25,7 +25,7 @@ from copy import copy
 from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.test_dimensions import (
-    TestDimension,
+    ImpalaTestDimension,
     create_single_exec_option_dimension,
     create_uncompressed_text_dimension)
 
@@ -68,21 +68,24 @@ class TestQueryMemLimit(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestQueryMemLimit, cls).add_test_dimensions()
     # Only run the query for text
-    cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
+    cls.ImpalaTestMatrix.add_dimension(
+        create_uncompressed_text_dimension(cls.get_workload()))
 
     # add mem_limit as a test dimension.
     if cls.exploration_strategy() == 'core':
-      cls.TestMatrix.add_dimension(\
-          TestDimension('mem_limit', *TestQueryMemLimit.MEM_LIMITS_CORE))
+      cls.ImpalaTestMatrix.add_dimension(\
+          ImpalaTestDimension('mem_limit', *TestQueryMemLimit.MEM_LIMITS_CORE))
     else:
-      cls.TestMatrix.add_dimension(\
-          TestDimension('mem_limit', *TestQueryMemLimit.MEM_LIMITS))
+      cls.ImpalaTestMatrix.add_dimension(\
+          ImpalaTestDimension('mem_limit', *TestQueryMemLimit.MEM_LIMITS))
 
     # Make query a test dimension so we can support more queries.
-    cls.TestMatrix.add_dimension(TestDimension('query', *TestQueryMemLimit.QUERIES))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('query', *TestQueryMemLimit.QUERIES))
     # This query takes a very long time to finish with a bound on the batch_size.
     # Remove the bound on the batch size.
-    cls.TestMatrix.add_constraint(lambda v: v.get_value('exec_option')['batch_size'] == 0)
+    cls.ImpalaTestMatrix.add_constraint(
+        lambda v: v.get_value('exec_option')['batch_size'] == 0)
 
   @pytest.mark.execute_serially
   def test_mem_limit(self, vector):
@@ -115,9 +118,9 @@ class TestCodegenMemLimit(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestCodegenMemLimit, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
     # Only run the query for parquet
-    cls.TestMatrix.add_constraint(
+    cls.ImpalaTestMatrix.add_constraint(
       lambda v: v.get_value('table_format').file_format == 'parquet')
 
   def test_codegen_mem_limit(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_query_opts.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_query_opts.py b/tests/query_test/test_query_opts.py
index fefe475..f9ef1d5 100644
--- a/tests/query_test/test_query_opts.py
+++ b/tests/query_test/test_query_opts.py
@@ -34,9 +34,9 @@ class TestQueryOptions(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestQueryOptions, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text')
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0]))
 
   def test_set_invalid_query_option(self, vector):
@@ -47,9 +47,9 @@ class TestQueryOptionsHS2(HS2TestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestQueryOptions, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text')
-    cls.TestMatrix.add_dimension(create_exec_option_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_exec_option_dimension(
         cluster_sizes=[0], disable_codegen_options=[False], batch_sizes=[0]))
 
   @needs_session()

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_rows_availability.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_rows_availability.py b/tests/query_test/test_rows_availability.py
index 5959b5c..fa1ecb8 100644
--- a/tests/query_test/test_rows_availability.py
+++ b/tests/query_test/test_rows_availability.py
@@ -18,7 +18,7 @@
 import pytest
 import re
 from tests.common.impala_test_suite import ImpalaTestSuite
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 
 class TestRowsAvailability(ImpalaTestSuite):
   """Tests that the 'Rows available' timeline event is marked only after rows are
@@ -49,8 +49,8 @@ class TestRowsAvailability(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestRowsAvailability, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(TestDimension('query', *cls.QUERIES))
-    cls.TestMatrix.add_constraint(lambda v: cls.__is_valid_test_vector(v))
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('query', *cls.QUERIES))
+    cls.ImpalaTestMatrix.add_constraint(lambda v: cls.__is_valid_test_vector(v))
 
   @classmethod
   def __is_valid_test_vector(cls, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_runtime_filters.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_runtime_filters.py b/tests/query_test/test_runtime_filters.py
index f3946d4..6edfd82 100644
--- a/tests/query_test/test_runtime_filters.py
+++ b/tests/query_test/test_runtime_filters.py
@@ -32,7 +32,7 @@ class TestRuntimeFilters(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestRuntimeFilters, cls).add_test_dimensions()
     # Runtime filters are disabled on HBase, Kudu
-    cls.TestMatrix.add_constraint(
+    cls.ImpalaTestMatrix.add_constraint(
       lambda v: v.get_value('table_format').file_format not in ['hbase', 'kudu'])
 
   def test_basic_filters(self, vector):
@@ -57,7 +57,7 @@ class TestRuntimeRowFilters(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestRuntimeRowFilters, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:
+    cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format in ['parquet'])
 
   def test_row_filters(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_scanners.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py
index cd1b94a..21c0d9d 100644
--- a/tests/query_test/test_scanners.py
+++ b/tests/query_test/test_scanners.py
@@ -39,7 +39,7 @@ from tests.common.test_result_verifier import (
     parse_column_labels,
     QueryTestResult,
     parse_result_rows)
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 from tests.util.filesystem_utils import WAREHOUSE, get_fs_path
 from tests.util.get_parquet_metadata import get_parquet_metadata
 from tests.util.test_file_parser import QueryTestSectionReader
@@ -58,9 +58,9 @@ class TestScannersAllTableFormats(ImpalaTestSuite):
     if cls.exploration_strategy() == 'core':
       # The purpose of this test is to get some base coverage of all the file formats.
       # Even in 'core', we'll test each format by using the pairwise strategy.
-      cls.TestMatrix.add_dimension(cls.create_table_info_dimension('pairwise'))
-    cls.TestMatrix.add_dimension(
-        TestDimension('batch_size', *TestScannersAllTableFormats.BATCH_SIZES))
+      cls.ImpalaTestMatrix.add_dimension(cls.create_table_info_dimension('pairwise'))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('batch_size', *TestScannersAllTableFormats.BATCH_SIZES))
 
   def test_scanners(self, vector):
     new_vector = deepcopy(vector)
@@ -111,10 +111,10 @@ class TestUnmatchedSchema(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestUnmatchedSchema, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
     # Avro has a more advanced schema evolution process which is covered in more depth
     # in the test_avro_schema_evolution test suite.
-    cls.TestMatrix.add_constraint(\
+    cls.ImpalaTestMatrix.add_constraint(\
         lambda v: v.get_value('table_format').file_format != 'avro')
 
   def _create_test_table(self, vector):
@@ -160,7 +160,7 @@ class TestWideRow(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestWideRow, cls).add_test_dimensions()
     # I can't figure out how to load a huge row into hbase
-    cls.TestMatrix.add_constraint(
+    cls.ImpalaTestMatrix.add_constraint(
       lambda v: v.get_value('table_format').file_format != 'hbase')
 
   def test_wide_row(self, vector):
@@ -190,10 +190,10 @@ class TestWideTable(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestWideTable, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(TestDimension("num_cols", *cls.NUM_COLS))
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension("num_cols", *cls.NUM_COLS))
     # To cut down on test execution time, only run in exhaustive.
     if cls.exploration_strategy() != 'exhaustive':
-      cls.TestMatrix.add_constraint(lambda v: False)
+      cls.ImpalaTestMatrix.add_constraint(lambda v: False)
 
   def test_wide_table(self, vector):
     if vector.get_value('table_format').file_format == 'kudu':
@@ -232,7 +232,7 @@ class TestParquet(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestParquet, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(
+    cls.ImpalaTestMatrix.add_constraint(
       lambda v: v.get_value('table_format').file_format == 'parquet')
 
   def test_parquet(self, vector):
@@ -471,8 +471,8 @@ class TestScanRangeLengths(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestScanRangeLengths, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(
-        TestDimension('max_scan_range_length', *MAX_SCAN_RANGE_LENGTHS))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('max_scan_range_length', *MAX_SCAN_RANGE_LENGTHS))
 
   def test_scan_ranges(self, vector):
     vector.get_value('exec_option')['max_scan_range_length'] =\
@@ -494,9 +494,9 @@ class TestTextScanRangeLengths(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestTextScanRangeLengths, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(
-        TestDimension('max_scan_range_length', *MAX_SCAN_RANGE_LENGTHS))
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('max_scan_range_length', *MAX_SCAN_RANGE_LENGTHS))
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text' and\
         v.get_value('table_format').compression_codec == 'none')
 
@@ -527,7 +527,7 @@ class TestTextSplitDelimiters(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestTextSplitDelimiters, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text' and\
         v.get_value('table_format').compression_codec == 'none')
 
@@ -604,7 +604,7 @@ class TestTextScanRangeLengths(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestTextScanRangeLengths, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(
+    cls.ImpalaTestMatrix.add_constraint(
       lambda v: v.get_value('table_format').file_format == 'text')
 
   def test_text_scanner_with_header(self, vector, unique_database):
@@ -623,18 +623,18 @@ class TestScanTruncatedFiles(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestScanTruncatedFiles, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
 
     # This test takes about a minute to complete due to the Hive commands that are
     # executed. To cut down on runtime, limit the test to exhaustive exploration
     # strategy.
     # TODO: Test other file formats
     if cls.exploration_strategy() == 'exhaustive':
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').file_format == 'text' and\
           v.get_value('table_format').compression_codec == 'none')
     else:
-      cls.TestMatrix.add_constraint(lambda v: False)
+      cls.ImpalaTestMatrix.add_constraint(lambda v: False)
 
   def test_scan_truncated_file_empty(self, vector, unique_database):
     self.scan_truncated_file(0, unique_database)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_scanners_fuzz.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners_fuzz.py b/tests/query_test/test_scanners_fuzz.py
index 81d20a4..82b4ec9 100644
--- a/tests/query_test/test_scanners_fuzz.py
+++ b/tests/query_test/test_scanners_fuzz.py
@@ -53,13 +53,13 @@ class TestScannersFuzzing(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestScannersFuzzing, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(
+    cls.ImpalaTestMatrix.add_dimension(
         create_exec_option_dimension_from_dict({
           'abort_on_error' : cls.ABORT_ON_ERROR_VALUES,
           'num_nodes' : cls.NUM_NODES_VALUES,
           'mem_limit' : cls.MEM_LIMITS}))
     # TODO: enable for more table formats once they consistently pass the fuzz test.
-    cls.TestMatrix.add_constraint(lambda v:
+    cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format in ('avro', 'parquet') or
         (v.get_value('table_format').file_format == 'text' and
           v.get_value('table_format').compression_codec in ('none', 'lzo')))

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_scratch_limit.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scratch_limit.py b/tests/query_test/test_scratch_limit.py
index 20e8e8f..6a13318 100644
--- a/tests/query_test/test_scratch_limit.py
+++ b/tests/query_test/test_scratch_limit.py
@@ -38,8 +38,8 @@ class TestScratchLimit(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestScratchLimit, cls).add_test_dimensions()
     # There is no reason to run these tests using all dimensions.
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
-    cls.TestMatrix.add_dimension(
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(
         create_uncompressed_text_dimension(cls.get_workload()))
 
   def test_with_high_scratch_limit(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_sort.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_sort.py b/tests/query_test/test_sort.py
index 89d6d41..228d25d 100644
--- a/tests/query_test/test_sort.py
+++ b/tests/query_test/test_sort.py
@@ -35,7 +35,7 @@ class TestQueryFullSort(ImpalaTestSuite):
     super(TestQueryFullSort, cls).add_test_dimensions()
 
     if cls.exploration_strategy() == 'core':
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').file_format == 'parquet')
 
   def test_multiple_mem_limits(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_timezones.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_timezones.py b/tests/query_test/test_timezones.py
index f388f02..367ba5b 100644
--- a/tests/query_test/test_timezones.py
+++ b/tests/query_test/test_timezones.py
@@ -32,7 +32,7 @@ class TestTimeZones(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestTimeZones, cls).add_test_dimensions()
 
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format == 'text' and\
         v.get_value('table_format').compression_codec == 'none')
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_tpcds_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_tpcds_queries.py b/tests/query_test/test_tpcds_queries.py
index 7645547..eb6618f 100644
--- a/tests/query_test/test_tpcds_queries.py
+++ b/tests/query_test/test_tpcds_queries.py
@@ -32,7 +32,7 @@ class TestTpcdsQuery(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestTpcdsQuery, cls).add_test_dimensions()
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format not in ['rc', 'hbase', 'kudu'] and\
         v.get_value('table_format').compression_codec in ['none', 'snap'] and\
         v.get_value('table_format').compression_type != 'record')
@@ -40,10 +40,10 @@ class TestTpcdsQuery(ImpalaTestSuite):
     if cls.exploration_strategy() != 'exhaustive':
       # Cut down on the execution time for these tests in core by running only
       # against parquet.
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').file_format in ['parquet'])
 
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('exec_option')['batch_size'] == 0)
 
   @pytest.mark.execute_serially
@@ -135,14 +135,14 @@ class TestTpcdsInsert(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestTpcdsInsert, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         is_supported_insert_format(v.get_value('table_format')))
     if cls.exploration_strategy() == 'core' and not pytest.config.option.table_formats:
       # Don't run on core, unless the user explicitly wants to validate a specific table
       # format. Each test vector takes > 30s to complete and it doesn't add much
       # additional coverage on top of what's in the functional insert test suite
-      cls.TestMatrix.add_constraint(lambda v: False);
+      cls.ImpalaTestMatrix.add_constraint(lambda v: False);
 
   def test_tpcds_partitioned_insert(self, vector):
     self.run_test_case('partitioned-insert', vector)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_tpch_nested_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_tpch_nested_queries.py b/tests/query_test/test_tpch_nested_queries.py
index 2b87280..7a78cbe 100644
--- a/tests/query_test/test_tpch_nested_queries.py
+++ b/tests/query_test/test_tpch_nested_queries.py
@@ -30,9 +30,9 @@ class TestTpchNestedQuery(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestTpchNestedQuery, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
     # The nested tpch data is currently only available in parquet.
-    cls.TestMatrix.add_constraint(lambda v:\
+    cls.ImpalaTestMatrix.add_constraint(lambda v:\
         v.get_value('table_format').file_format in ['parquet'])
 
   def test_tpch_q1(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_tpch_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_tpch_queries.py b/tests/query_test/test_tpch_queries.py
index adcd575..55c9f3d 100644
--- a/tests/query_test/test_tpch_queries.py
+++ b/tests/query_test/test_tpch_queries.py
@@ -29,13 +29,13 @@ class TestTpchQuery(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestTpchQuery, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
 
     # The tpch tests take a long time to execute so restrict the combinations they
     # execute over.
     # TODO: the planner tests are based on text and need this.
     if cls.exploration_strategy() == 'core':
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('table_format').file_format in ['text', 'parquet', 'kudu'])
 
   def get_test_file_prefix(self, vector):

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index 56f1253..1b2a51c 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -239,12 +239,13 @@ class TestUdfExecution(TestUdfBase):
   @classmethod
   def add_test_dimensions(cls):
     super(TestUdfExecution, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(
+    cls.ImpalaTestMatrix.add_dimension(
         create_exec_option_dimension_from_dict({"disable_codegen" : [False, True],
           "exec_single_node_rows_threshold" : [0,100],
           "enable_expr_rewrites" : [False, True]}))
     # There is no reason to run these tests using all dimensions.
-    cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
+    cls.ImpalaTestMatrix.add_dimension(
+        create_uncompressed_text_dimension(cls.get_workload()))
 
   def test_native_functions(self, vector, unique_database):
     enable_expr_rewrites = vector.get_value('exec_option')['enable_expr_rewrites']
@@ -363,7 +364,8 @@ class TestUdfTargeted(TestUdfBase):
   def add_test_dimensions(cls):
     super(TestUdfTargeted, cls).add_test_dimensions()
     # There is no reason to run these tests using all dimensions.
-    cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
+    cls.ImpalaTestMatrix.add_dimension(
+        create_uncompressed_text_dimension(cls.get_workload()))
 
   def test_udf_invalid_symbol(self, vector, unique_database):
     """ IMPALA-1642: Impala crashes if the symbol for a Hive UDF doesn't exist

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/stress/test_ddl_stress.py
----------------------------------------------------------------------
diff --git a/tests/stress/test_ddl_stress.py b/tests/stress/test_ddl_stress.py
index 7bc650b..5a55f5d 100644
--- a/tests/stress/test_ddl_stress.py
+++ b/tests/stress/test_ddl_stress.py
@@ -20,7 +20,7 @@ import uuid
 
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfS3, SkipIfIsilon, SkipIfLocal
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 
 # Number of tables to create per thread
 NUM_TBLS_PER_THREAD = 10
@@ -42,10 +42,11 @@ class TestDdlStress(ImpalaTestSuite):
     if cls.exploration_strategy() != 'exhaustive':
       pytest.skip("Should only run in exhaustive due to long execution time.")
 
-    cls.TestMatrix.add_dimension(TestDimension('test_id', *TEST_IDS))
-    cls.TestMatrix.add_constraint(lambda v: v.get_value('exec_option')['batch_size'] == 0)
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('test_id', *TEST_IDS))
+    cls.ImpalaTestMatrix.add_constraint(
+        lambda v: v.get_value('exec_option')['batch_size'] == 0)
 
-    cls.TestMatrix.add_constraint(lambda v:
+    cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format == 'text' and\
         v.get_value('table_format').compression_codec == 'none')
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/stress/test_mini_stress.py
----------------------------------------------------------------------
diff --git a/tests/stress/test_mini_stress.py b/tests/stress/test_mini_stress.py
index 87a8742..d91163d 100644
--- a/tests/stress/test_mini_stress.py
+++ b/tests/stress/test_mini_stress.py
@@ -20,7 +20,7 @@ import re
 
 from tests.common.impala_cluster import ImpalaCluster
 from tests.common.impala_test_suite import ImpalaTestSuite
-from tests.common.test_vector import TestDimension
+from tests.common.test_vector import ImpalaTestDimension
 
 # Number of times to execute each query
 NUM_ITERATIONS = 5
@@ -38,12 +38,12 @@ class TestMiniStress(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestMiniStress, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(TestDimension('test_id', *TEST_IDS))
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('test_id', *TEST_IDS))
     if cls.exploration_strategy() != 'exhaustive':
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('exec_option')['batch_size'] == 0)
     else:
-      cls.TestMatrix.add_constraint(lambda v:\
+      cls.ImpalaTestMatrix.add_constraint(lambda v:\
           v.get_value('exec_option')['batch_size'] != 1)
 
   @pytest.mark.xfail(run=False, reason="IMPALA-2605: the stress tests have a history of "

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/f590bc0d/tests/unittests/test_result_verifier.py
----------------------------------------------------------------------
diff --git a/tests/unittests/test_result_verifier.py b/tests/unittests/test_result_verifier.py
index 0c6ffd1..de8515a 100644
--- a/tests/unittests/test_result_verifier.py
+++ b/tests/unittests/test_result_verifier.py
@@ -28,7 +28,8 @@ class TestResultVerifier(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestResultVerifier, cls).add_test_dimensions()
-    cls.TestMatrix.add_dimension(create_uncompressed_text_dimension(cls.get_workload()))
+    cls.ImpalaTestMatrix.add_dimension(
+        create_uncompressed_text_dimension(cls.get_workload()))
 
   def test_result_row_indexing(self, vector):
     res = create_query_result(self.client.execute("select 1 as int_col, 'A' as str_col"))