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

[impala] 02/03: expr-test: use gtest parameterization

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

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

commit e2ead7f8572c6005f23f8654305ea8eb389c7c5a
Author: Todd Lipcon <to...@apache.org>
AuthorDate: Wed May 8 16:14:01 2019 -0700

    expr-test: use gtest parameterization
    
    Instead of running the tests three times with different flags from
    main(), this uses gtest's parameterization feature to accomplish the
    same.
    
    The advantage here is that we end up with different test names for each
    of the runs. Additionally, this moves the setup code into a proper setup
    method so that executing expr-test --gtest_list_tests doesn't waste time
    starting a cluster.
    
    This is prep work towards adding multi-threaded test execution for
    long-running tests. expr-test seems to currently be one of the worst
    offenders.
    
    Change-Id: Idc9fb24ad62b4aa2e120a99d74ae04bb221c034b
    Reviewed-on: http://gerrit.cloudera.org:8080/13289
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exprs/expr-test.cc | 260 ++++++++++++++++++++++++----------------------
 1 file changed, 135 insertions(+), 125 deletions(-)

diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index dbdb8a1..4e32ce5 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -94,8 +94,6 @@ const char* TEST_TZ_WITHOUT_DST = "America/Anguilla";
 ImpaladQueryExecutor* executor_;
 scoped_ptr<MetricGroup> statestore_metrics(new MetricGroup("statestore_metrics"));
 Statestore* statestore;
-bool disable_codegen_;
-bool enable_expr_rewrites_;
 
 template <typename ORIGINAL_TYPE, typename VAL_TYPE>
 string LiteralToString(VAL_TYPE val) {
@@ -189,7 +187,50 @@ class ScopedLocalUnixTimestampConversionOverride {
   }
 };
 
-class ExprTest : public testing::Test {
+class ExprTest : public testing::TestWithParam<std::tuple<bool, bool>> {
+ public:
+  // Run once (independent of parameter values).
+  static void SetUpTestCase() {
+    InitFeSupport(false);
+    ABORT_IF_ERROR(impala::LlvmCodeGen::InitializeLlvm());
+
+    // The host running this test might have an out-of-date tzdata package installed.
+    // To avoid tzdata related issues, we will load time-zone db from the testdata
+    // directory.
+    FLAGS_hdfs_zone_info_zip = Substitute("file://$0/testdata/tzdb/2017c.zip",
+        getenv("IMPALA_HOME"));
+    ABORT_IF_ERROR(TimezoneDatabase::Initialize());
+
+    // Disable llvm optimization passes if the env var is no set to true. Running without
+    // the optimizations makes the tests run much faster.
+    char* optimizations = getenv("EXPR_TEST_ENABLE_OPTIMIZATIONS");
+    if (optimizations != NULL && strcmp(optimizations, "true") == 0) {
+      cout << "Running with optimization passes." << endl;
+      FLAGS_disable_optimization_passes = false;
+    } else {
+      cout << "Running without optimization passes." << endl;
+      FLAGS_disable_optimization_passes = true;
+    }
+
+    // Create an in-process Impala server and in-process backends for test environment
+    // without any startup validation check
+    FLAGS_abort_on_config_error = false;
+    VLOG_CONNECTION << "creating test env";
+    VLOG_CONNECTION << "starting backends";
+    statestore = new Statestore(statestore_metrics.get());
+    IGNORE_LEAKING_OBJECT(statestore);
+
+    // Pass in 0 to have the statestore use an ephemeral port for the service.
+    ABORT_IF_ERROR(statestore->Init(0));
+    InProcessImpalaServer* impala_server;
+    ABORT_IF_ERROR(InProcessImpalaServer::StartWithEphemeralPorts(
+        FLAGS_hostname, statestore->port(), &impala_server));
+    IGNORE_LEAKING_OBJECT(impala_server);
+
+    executor_ = new ImpaladQueryExecutor(FLAGS_hostname, impala_server->GetBeeswaxPort());
+    ABORT_IF_ERROR(executor_->Setup());
+  }
+
  protected:
   // Pool for objects to be destroyed during test teardown.
   ObjectPool pool_;
@@ -222,7 +263,27 @@ class ExprTest : public testing::Test {
   TimestampValue default_timestamp_val_;
   DateValue default_date_val_;
 
+  bool disable_codegen_;
+  bool enable_expr_rewrites_;
+
   virtual void SetUp() {
+    disable_codegen_ = std::get<0>(GetParam());
+    enable_expr_rewrites_ = std::get<1>(GetParam());
+    LOG(INFO) << Substitute(
+      "Test case: disable_codegen=$0  enable_expr_rewrites=$1",
+      disable_codegen_, enable_expr_rewrites_);
+    executor_->ClearExecOptions();
+    executor_->PushExecOption(Substitute("DISABLE_CODEGEN=$0",
+        disable_codegen_ ? 1 : 0));
+    executor_->PushExecOption(Substitute("ENABLE_EXPR_REWRITES=$0",
+        enable_expr_rewrites_ ? 1 : 0));
+
+    // The following have no effect when codegen is disabled, but don't
+    // harm anything either. They generally prevent the planner from doing
+    // anything clever here.
+    executor_->PushExecOption("EXEC_SINGLE_NODE_ROWS_THRESHOLD=0");
+    executor_->PushExecOption("DISABLE_CODEGEN_ROWS_THRESHOLD=0");
+
     min_int_values_[TYPE_TINYINT] = 1;
     min_int_values_[TYPE_SMALLINT] =
         static_cast<int64_t>(numeric_limits<int8_t>::max()) + 1;
@@ -1333,7 +1394,7 @@ void ExprTest::TestSingleLiteralConstruction(
   state.ReleaseResources();
 }
 
-TEST_F(ExprTest, NullLiteral) {
+TEST_P(ExprTest, NullLiteral) {
   for (int type = TYPE_BOOLEAN; type != TYPE_DATE; ++type) {
     RuntimeState state(TQueryCtx(), ExecEnv::GetInstance());
     ObjectPool pool;
@@ -1351,7 +1412,7 @@ TEST_F(ExprTest, NullLiteral) {
   }
 }
 
-TEST_F(ExprTest, LiteralConstruction) {
+TEST_P(ExprTest, LiteralConstruction) {
   bool b_val = true;
   int8_t c_val = 'f';
   int16_t s_val = 123;
@@ -1409,7 +1470,7 @@ TEST_F(ExprTest, LiteralConstruction) {
 }
 
 
-TEST_F(ExprTest, LiteralExprs) {
+TEST_P(ExprTest, LiteralExprs) {
   TestFixedPointLimits<int8_t>(TYPE_TINYINT);
   TestFixedPointLimits<int16_t>(TYPE_SMALLINT);
   TestFixedPointLimits<int32_t>(TYPE_INT);
@@ -1426,7 +1487,7 @@ TEST_F(ExprTest, LiteralExprs) {
 }
 
 // IMPALA-3942: Test escaping string literal for single/double quotes
-TEST_F(ExprTest, EscapeStringLiteral) {
+TEST_P(ExprTest, EscapeStringLiteral) {
   TestStringValue(R"('"')", R"(")");
   TestStringValue(R"("'")", R"(')");
   TestStringValue(R"("\"")", R"(")");
@@ -1461,7 +1522,7 @@ TEST_F(ExprTest, EscapeStringLiteral) {
   TestStringValue(R"(concat('a\\\'b', "c\\\"d"))", R"(a\'bc\"d)");
 }
 
-TEST_F(ExprTest, ArithmeticExprs) {
+TEST_P(ExprTest, ArithmeticExprs) {
   // Test float ops.
   TestFixedResultTypeOps<float, float, double>(min_float_values_[TYPE_FLOAT],
       min_float_values_[TYPE_FLOAT], TYPE_DOUBLE);
@@ -2968,7 +3029,7 @@ void TestScaleBy() {
   }
 }
 
-TEST_F(ExprTest, DecimalArithmeticExprs) {
+TEST_P(ExprTest, DecimalArithmeticExprs) {
   // Test with both decimal_v2={false, true}
   for (int v2: { 0, 1 }) {
     string opt = "DECIMAL_V2=" + lexical_cast<string>(v2);
@@ -3004,7 +3065,7 @@ TEST_F(ExprTest, DecimalArithmeticExprs) {
 }
 
 // Tests for expressions that mix decimal and non-decimal arguments with DECIMAL_V2=false.
-TEST_F(ExprTest, DecimalV1MixedArithmeticExprs) {
+TEST_P(ExprTest, DecimalV1MixedArithmeticExprs) {
   executor_->PushExecOption("DECIMAL_V2=false");
   // IMPALA-3437: decimal constants are implicitly converted to double.
   TestValue("10.0 + 3", TYPE_DOUBLE, 13.0);
@@ -3022,7 +3083,7 @@ TEST_F(ExprTest, DecimalV1MixedArithmeticExprs) {
 }
 
 // Tests the same expressions as above with DECIMAL_V2=true.
-TEST_F(ExprTest, DecimalV2MixedArithmeticExprs) {
+TEST_P(ExprTest, DecimalV2MixedArithmeticExprs) {
   executor_->PushExecOption("DECIMAL_V2=true");
   // IMPALA-3437: decimal constants remain decimal.
   TestDecimalValue(
@@ -3046,7 +3107,7 @@ TEST_F(ExprTest, DecimalV2MixedArithmeticExprs) {
 
 // There are two tests of ranges, the second of which requires a cast
 // of the second operand to a higher-resolution type.
-TEST_F(ExprTest, BinaryPredicates) {
+TEST_P(ExprTest, BinaryPredicates) {
   TestComparison("false", "true", false);
   TestBinaryPredicates("false", false);
   TestBinaryPredicates("true", false);
@@ -3063,7 +3124,7 @@ TEST_F(ExprTest, BinaryPredicates) {
 }
 
 // Test casting from all types to all other types
-TEST_F(ExprTest, CastExprs) {
+TEST_P(ExprTest, CastExprs) {
   // From tinyint
   TestCast("cast(0 as tinyint)", 0);
   TestCast("cast(5 as tinyint)", 5);
@@ -3384,7 +3445,7 @@ TEST_F(ExprTest, CastExprs) {
 }
 
 // Test casting from/to Date.
-TEST_F(ExprTest, CastDateExprs) {
+TEST_P(ExprTest, CastDateExprs) {
   // From Date to Date
   TestStringValue("cast(cast(cast('2012-01-01' as date) as date) as string)",
       "2012-01-01");
@@ -3510,7 +3571,7 @@ TEST_F(ExprTest, CastDateExprs) {
   TestIsNull("cast(cast(null as date) as timestamp)", TYPE_TIMESTAMP);
 }
 
-TEST_F(ExprTest, CompoundPredicates) {
+TEST_P(ExprTest, CompoundPredicates) {
   TestValue("TRUE AND TRUE", TYPE_BOOLEAN, true);
   TestValue("TRUE AND FALSE", TYPE_BOOLEAN, false);
   TestValue("FALSE AND TRUE", TYPE_BOOLEAN, false);
@@ -3561,14 +3622,14 @@ TEST_F(ExprTest, CompoundPredicates) {
   TestIsNull("!NULL", TYPE_BOOLEAN);
 }
 
-TEST_F(ExprTest, IsNullPredicate) {
+TEST_P(ExprTest, IsNullPredicate) {
   TestValue("5 IS NULL", TYPE_BOOLEAN, false);
   TestValue("5 IS NOT NULL", TYPE_BOOLEAN, true);
   TestValue("NULL IS NULL", TYPE_BOOLEAN, true);
   TestValue("NULL IS NOT NULL", TYPE_BOOLEAN, false);
 }
 
-TEST_F(ExprTest, BoolTestExpr) {
+TEST_P(ExprTest, BoolTestExpr) {
   // Tests against constants.
   TestValue("TRUE IS TRUE", TYPE_BOOLEAN, true);
   TestValue("TRUE IS FALSE", TYPE_BOOLEAN, false);
@@ -3610,7 +3671,7 @@ TEST_F(ExprTest, BoolTestExpr) {
   TestValue("(NULL = 1) IS NOT UNKNOWN", TYPE_BOOLEAN, false);
 }
 
-TEST_F(ExprTest, LikePredicate) {
+TEST_P(ExprTest, LikePredicate) {
   TestValue("'a' LIKE '%a%'", TYPE_BOOLEAN, true);
   TestValue("'a' LIKE '%abcde'", TYPE_BOOLEAN, false);
   TestValue("'a' LIKE 'abcde%'", TYPE_BOOLEAN, false);
@@ -3759,7 +3820,7 @@ TEST_F(ExprTest, LikePredicate) {
   TestValue("'engLish' IREGEXP 'lIsh$'", TYPE_BOOLEAN, true);
 }
 
-TEST_F(ExprTest, BetweenPredicate) {
+TEST_P(ExprTest, BetweenPredicate) {
   // Between is rewritten into a conjunctive compound predicate.
   // Compound predicates are also tested elsewere, so we just do basic testing here.
   TestValue("5 between 0 and 10", TYPE_BOOLEAN, true);
@@ -3798,7 +3859,7 @@ TEST_F(ExprTest, BetweenPredicate) {
 }
 
 // Tests with NULLs are in the FE QueryTest.
-TEST_F(ExprTest, InPredicate) {
+TEST_P(ExprTest, InPredicate) {
   // Test integers.
   IntValMap::iterator int_iter;
   for(int_iter = min_int_values_.begin(); int_iter != min_int_values_.end();
@@ -3908,7 +3969,7 @@ TEST_F(ExprTest, InPredicate) {
   TestIsNull("NULL in (NULL, NULL)", TYPE_BOOLEAN);
 }
 
-TEST_F(ExprTest, StringFunctions) {
+TEST_P(ExprTest, StringFunctions) {
   TestValue("levenshtein('levenshtein', 'frankenstein')", TYPE_INT, 6);
   TestValue("levenshtein('example', 'samples')", TYPE_INT, 3);
   TestValue("levenshtein('sturgeon', 'urgently')", TYPE_INT, 6);
@@ -4533,7 +4594,7 @@ TEST_F(ExprTest, StringFunctions) {
   TestStringValue(query, big_str);
 }
 
-TEST_F(ExprTest, StringBase64Coding) {
+TEST_P(ExprTest, StringBase64Coding) {
   // Test some known values of base64{en,de}code
   TestIsNull("base64encode(NULL)", TYPE_STRING);
   TestIsNull("base64decode(NULL)", TYPE_STRING);
@@ -4563,7 +4624,7 @@ TEST_F(ExprTest, StringBase64Coding) {
   }
 }
 
-TEST_F(ExprTest, LongReverse) {
+TEST_P(ExprTest, LongReverse) {
   static const int MAX_LEN = 2048;
   string to_reverse(MAX_LEN, ' '), reversed(MAX_LEN, ' ');
   // Pick some 'interesting' (i.e. random, but include some powers of two, some primes,
@@ -4575,7 +4636,7 @@ TEST_F(ExprTest, LongReverse) {
   }
 }
 
-TEST_F(ExprTest, StringRegexpFunctions) {
+TEST_P(ExprTest, StringRegexpFunctions) {
   // Single group.
   TestStringValue("regexp_extract('abxcy1234a', 'a.x', 0)", "abx");
   TestStringValue("regexp_extract('abxcy1234a', 'a.x.*a', 0)", "abxcy1234a");
@@ -4747,7 +4808,8 @@ TEST_F(ExprTest, StringRegexpFunctions) {
       R"('([[:alpha:]]+)(\\\\\\\\)([[:alpha:]]+)', 3))", "world");
 }
 
-TEST_F(ExprTest, StringParseUrlFunction) { // TODO: For now, our parse_url my not behave exactly like Hive
+TEST_P(ExprTest, StringParseUrlFunction) {
+  // TODO: For now, our parse_url my not behave exactly like Hive
   // when given malformed URLs.
   // If necessary, we can closely follow Java's URL implementation
   // to behave exactly like Hive.
@@ -5075,7 +5137,7 @@ TEST_F(ExprTest, StringParseUrlFunction) { // TODO: For now, our parse_url my no
       "index.html?test=true&name=networking&op=true', 'XYZ', 'name')");
 }
 
-TEST_F(ExprTest, UtilityFunctions) {
+TEST_P(ExprTest, UtilityFunctions) {
   TestStringValue("current_database()", "default");
   TestStringValue("current_catalog()", "default");
   TestStringValue("user()", "impala_test_user");
@@ -5170,7 +5232,7 @@ TEST_F(ExprTest, UtilityFunctions) {
 }
 
 // Test that UtilityFunctions::Coordinator() will return null if coord_address is unset
-TEST_F(ExprTest, CoordinatorFunction) {
+TEST_P(ExprTest, CoordinatorFunction) {
   // Make a RuntimeState where the query context does not have coord_address set.
   // Note that this should never happen in a real impalad.
   RuntimeState state(TQueryCtx(), ExecEnv::GetInstance());
@@ -5189,7 +5251,7 @@ TEST_F(ExprTest, CoordinatorFunction) {
   state.ReleaseResources();
 }
 
-TEST_F(ExprTest, MurmurHashFunction) {
+TEST_P(ExprTest, MurmurHashFunction) {
   string s("hello world");
   int64_t expected = HashUtil::MurmurHash2_64(s.data(), s.size(),
       HashUtil::MURMUR_DEFAULT_SEED);
@@ -5239,7 +5301,7 @@ TEST_F(ExprTest, MurmurHashFunction) {
   TestIsNull("murmur_hash(NULL)", TYPE_BIGINT);
 }
 
-TEST_F(ExprTest, SessionFunctions) {
+TEST_P(ExprTest, SessionFunctions) {
   enum Session {S1, S2};
   enum Query {Q1, Q2};
 
@@ -5257,7 +5319,7 @@ TEST_F(ExprTest, SessionFunctions) {
   EXPECT_NE(results[S1][Q1], results[S2][Q1]);
 }
 
-TEST_F(ExprTest, NonFiniteFloats) {
+TEST_P(ExprTest, NonFiniteFloats) {
   TestValue("is_inf(0.0)", TYPE_BOOLEAN, false);
   TestValue("is_inf(-1/0)", TYPE_BOOLEAN, true);
   TestValue("is_inf(1/0)", TYPE_BOOLEAN, true);
@@ -5330,7 +5392,7 @@ TEST_F(ExprTest, NonFiniteFloats) {
   TestStringValue("CAST(0/0 AS STRING)", string("nan"));
 }
 
-TEST_F(ExprTest, InvalidFloats) {
+TEST_P(ExprTest, InvalidFloats) {
   // IMPALA-1731: Test that leading/trailing garbage is not allowed when parsing inf.
   TestIsNull("CAST('1.23inf' AS FLOAT)", TYPE_FLOAT);
   TestIsNull("CAST('1.23inf' AS DOUBLE)", TYPE_DOUBLE);
@@ -5374,7 +5436,7 @@ TEST_F(ExprTest, InvalidFloats) {
   TestIsNull("CAST('' AS FLOAT)", TYPE_FLOAT);
 }
 
-TEST_F(ExprTest, MathTrigonometricFunctions) {
+TEST_P(ExprTest, MathTrigonometricFunctions) {
   // It is important to calculate the expected values
   // using math functions, and not simply use constants.
   // Otherwise, floating point imprecisions may lead to failed tests.
@@ -5422,7 +5484,7 @@ TEST_F(ExprTest, MathTrigonometricFunctions) {
   TestIsNull("degrees(NULL)", TYPE_DOUBLE);
 }
 
-TEST_F(ExprTest, MathConversionFunctions) {
+TEST_P(ExprTest, MathConversionFunctions) {
   TestStringValue("bin(0)", "0");
   TestStringValue("bin(1)", "1");
   TestStringValue("bin(12)", "1100");
@@ -5525,7 +5587,7 @@ TEST_F(ExprTest, MathConversionFunctions) {
   TestIsNull("conv(NULL, NULL, NULL)", TYPE_STRING);
 }
 
-TEST_F(ExprTest, MathFunctions) {
+TEST_P(ExprTest, MathFunctions) {
   TestValue("pi()", TYPE_DOUBLE, M_PI);
   TestValue("e()", TYPE_DOUBLE, M_E);
   TestValue("abs(cast(-1.0 as double))", TYPE_DOUBLE, 1.0);
@@ -5949,7 +6011,7 @@ TEST_F(ExprTest, MathFunctions) {
   TestIsNull("greatest(cast(NULL as date))", TYPE_DATE);
 }
 
-TEST_F(ExprTest, MathRoundingFunctions) {
+TEST_P(ExprTest, MathRoundingFunctions) {
   TestValue("ceil(cast(0.1 as double))", TYPE_DOUBLE, 1);
   TestValue("ceil(cast(-10.05 as double))", TYPE_DOUBLE, -10);
   TestValue("ceil(cast(23.6 as double))", TYPE_DOUBLE, 24);
@@ -6005,7 +6067,7 @@ TEST_F(ExprTest, MathRoundingFunctions) {
   TestIsNull("round(cast(NULL as double), NULL)", TYPE_DOUBLE);
 }
 
-TEST_F(ExprTest, UnaryOperators) {
+TEST_P(ExprTest, UnaryOperators) {
   TestValue("+1", TYPE_TINYINT, 1);
   TestValue("-1", TYPE_TINYINT, -1);
   TestValue("- -1", TYPE_TINYINT, 1);
@@ -6024,7 +6086,7 @@ TEST_F(ExprTest, UnaryOperators) {
   TestValue("-1 & 8", TYPE_TINYINT, 8);
 }
 
-TEST_F(ExprTest, MoscowTimezoneConversion) {
+TEST_P(ExprTest, MoscowTimezoneConversion) {
 #pragma push_macro("UTC_TO_MSC")
 #pragma push_macro("MSC_TO_UTC")
 #define UTC_TO_MSC(X) ("cast(from_utc_timestamp('" X "', 'Europe/Moscow') as string)")
@@ -6114,7 +6176,7 @@ void ExprTest::TestAusDSTEndingForCentralTimeZone(const string& time_zone) {
 }
 
 // IMPALA-6699: Fix DST end time for Australian time-zones
-TEST_F(ExprTest, AusDSTEndingTests) {
+TEST_P(ExprTest, AusDSTEndingTests) {
   TestAusDSTEndingForEastTimeZone("AET");
   TestAusDSTEndingForEastTimeZone("Australia/ACT");
   TestAusDSTEndingForCentralTimeZone("Australia/Adelaide");
@@ -6131,7 +6193,7 @@ TEST_F(ExprTest, AusDSTEndingTests) {
   TestAusDSTEndingForCentralTimeZone("Australia/Yancowinna");
 }
 
-TEST_F(ExprTest, TimestampFunctions) {
+TEST_P(ExprTest, TimestampFunctions) {
   // Regression for IMPALA-1105
   TestIsNull("cast(cast('NOTATIMESTAMP' as timestamp) as string)", TYPE_STRING);
 
@@ -7357,7 +7419,7 @@ TEST_F(ExprTest, TimestampFunctions) {
   TestIsNull("unix_micros_to_utc_timestamp(253402300800000000)", TYPE_TIMESTAMP);
 }
 
-TEST_F(ExprTest, ConditionalFunctions) {
+TEST_P(ExprTest, ConditionalFunctions) {
   // If first param evaluates to true, should return second parameter,
   // false or NULL should return the third.
   TestValue("if(TRUE, FALSE, TRUE)", TYPE_BOOLEAN, false);
@@ -7653,7 +7715,7 @@ TEST_F(ExprTest, ConditionalFunctions) {
   TestIsNull("decode(NULL, 1, 2)", TYPE_TINYINT);
 }
 
-TEST_F(ExprTest, ConditionalFunctionIsTrue) {
+TEST_P(ExprTest, ConditionalFunctionIsTrue) {
   TestValue("istrue(cast(false as boolean))", TYPE_BOOLEAN, false);
   TestValue("istrue(cast(true as boolean))", TYPE_BOOLEAN, true);
   TestValue("istrue(cast(NULL as boolean))", TYPE_BOOLEAN, false);
@@ -7680,7 +7742,7 @@ TEST_F(ExprTest, ConditionalFunctionIsTrue) {
   TestError("istrue(-9999999999999999999999999999999999999.9)");
 }
 
-TEST_F(ExprTest, ConditionalFunctionIsFalse) {
+TEST_P(ExprTest, ConditionalFunctionIsFalse) {
   TestValue("isfalse(cast(false as boolean))", TYPE_BOOLEAN, true);
   TestValue("isfalse(cast(true as boolean))", TYPE_BOOLEAN, false);
   TestValue("isfalse(cast(NULL as boolean))", TYPE_BOOLEAN, false);
@@ -7709,7 +7771,7 @@ TEST_F(ExprTest, ConditionalFunctionIsFalse) {
   TestError("isfalse(-9999999999999999999999999999999999999.9)");
 }
 
-TEST_F(ExprTest, ConditionalFunctionIsNotTrue) {
+TEST_P(ExprTest, ConditionalFunctionIsNotTrue) {
   TestValue("isnottrue(cast(false as boolean))", TYPE_BOOLEAN, true);
   TestValue("isnottrue(cast(true as boolean))", TYPE_BOOLEAN, false);
   TestValue("isnottrue(cast(NULL as boolean))", TYPE_BOOLEAN, true);
@@ -7738,7 +7800,7 @@ TEST_F(ExprTest, ConditionalFunctionIsNotTrue) {
   TestError("isnottrue(-9999999999999999999999999999999999999.9)");
 }
 
-TEST_F(ExprTest, ConditionalFunctionIsNotFalse) {
+TEST_P(ExprTest, ConditionalFunctionIsNotFalse) {
   TestValue("isnotfalse(cast(false as boolean))", TYPE_BOOLEAN, false);
   TestValue("isnotfalse(cast(true as boolean))", TYPE_BOOLEAN, true);
   TestValue("isnotfalse(cast(NULL as boolean))", TYPE_BOOLEAN, true);
@@ -7798,7 +7860,7 @@ void ValidateLayout(const vector<ScalarExpr*>& exprs, int expected_byte_size,
   }
 }
 
-TEST_F(ExprTest, ResultsLayoutTest) {
+TEST_P(ExprTest, ResultsLayoutTest) {
   vector<ScalarExpr*> exprs;
   map<int, set<int>> expected_offsets;
 
@@ -7932,7 +7994,7 @@ TEST_F(ExprTest, ResultsLayoutTest) {
 }
 
 // TODO: is there an easy way to templatize/parametrize these tests?
-TEST_F(ExprTest, DecimalFunctions) {
+TEST_P(ExprTest, DecimalFunctions) {
   TestValue("precision(cast (1 as decimal(10,2)))", TYPE_INT, 10);
   TestValue("scale(cast(1 as decimal(10,2)))", TYPE_INT, 2);
 
@@ -8455,7 +8517,7 @@ TEST_F(ExprTest, DecimalFunctions) {
 
 // Sanity check some overflow casting. We have a random test framework that covers
 // this more thoroughly.
-TEST_F(ExprTest, DecimalOverflowCastsDecimalV1) {
+TEST_P(ExprTest, DecimalOverflowCastsDecimalV1) {
   executor_->PushExecOption("DECIMAL_V2=0");
 
   TestDecimalValue("cast(123.456 as decimal(6,3))",
@@ -8530,7 +8592,7 @@ TEST_F(ExprTest, DecimalOverflowCastsDecimalV1) {
   executor_->PopExecOption();
 }
 
-TEST_F(ExprTest, DecimalOverflowCastsDecimalV2) {
+TEST_P(ExprTest, DecimalOverflowCastsDecimalV2) {
   executor_->PushExecOption("DECIMAL_V2=1");
 
   TestDecimalValue("cast(123.456 as decimal(6,3))",
@@ -8599,7 +8661,7 @@ TEST_F(ExprTest, DecimalOverflowCastsDecimalV2) {
   executor_->PopExecOption();
 }
 
-TEST_F(ExprTest, NullValueFunction) {
+TEST_P(ExprTest, NullValueFunction) {
   TestValue("nullvalue(cast(NULL as boolean))", TYPE_BOOLEAN, true);
   TestValue("nullvalue(cast(0 as boolean))", TYPE_BOOLEAN, false);
   TestValue("nullvalue(cast(5 as boolean))", TYPE_BOOLEAN, false);
@@ -8669,7 +8731,7 @@ TEST_F(ExprTest, NullValueFunction) {
   TestValue("nullvalue(-99999999999999999999999999999999999.9)", TYPE_BOOLEAN, false);
 }
 
-TEST_F(ExprTest, NonNullValueFunction) {
+TEST_P(ExprTest, NonNullValueFunction) {
   TestValue("nonnullvalue(cast(NULL as boolean))", TYPE_BOOLEAN, false);
   TestValue("nonnullvalue(cast(0 as boolean))", TYPE_BOOLEAN, true);
   TestValue("nonnullvalue(cast(5 as boolean))", TYPE_BOOLEAN, true);
@@ -8739,7 +8801,7 @@ TEST_F(ExprTest, NonNullValueFunction) {
   TestValue("nonnullvalue(-99999999999999999999999999999999999.9)", TYPE_BOOLEAN, true);
 }
 
-TEST_F(ExprTest, UdfInterfaceBuiltins) {
+TEST_P(ExprTest, UdfInterfaceBuiltins) {
   TestValue("udf_pi()", TYPE_DOUBLE, M_PI);
   TestValue("udf_abs(-1)", TYPE_DOUBLE, 1.0);
   TestStringValue("udf_lower('Hello_WORLD')", "hello_world");
@@ -8755,7 +8817,7 @@ TEST_F(ExprTest, UdfInterfaceBuiltins) {
   TestValue("min_bigint()", TYPE_BIGINT, numeric_limits<int64_t>::min());
 }
 
-TEST_F(ExprTest, MADlib) {
+TEST_P(ExprTest, MADlib) {
   TestStringValue("madlib_encode_vector(madlib_vector(1.0, 2.0, 3.0))",
                   "aaaaaipdaaaaaaaeaaaaaeae");
   TestStringValue("madlib_print_vector(madlib_vector(1, 2, 3))", "<1, 2, 3>");
@@ -8774,7 +8836,7 @@ TEST_F(ExprTest, MADlib) {
     TYPE_DOUBLE, 3.0);
 }
 
-TEST_F(ExprTest, BitByteBuiltins) {
+TEST_P(ExprTest, BitByteBuiltins) {
   TestIsNull("bitand(1,NULL)", TYPE_TINYINT);
   TestIsNull("bitand(NULL,3)", TYPE_TINYINT);
   // And of numbers differing in 2nd bit position gives min of two numbers
@@ -8945,7 +9007,7 @@ TEST_F(ExprTest, BitByteBuiltins) {
   TestValue("rotateright(256, -2)", TYPE_SMALLINT, 1024);
 }
 
-TEST_F(ExprTest, UuidTest) {
+TEST_P(ExprTest, UuidTest) {
   boost::unordered_set<string> string_set;
   const unsigned int NUM_UUIDS = 10;
   const string regex(
@@ -8957,7 +9019,7 @@ TEST_F(ExprTest, UuidTest) {
   EXPECT_TRUE(string_set.size() == NUM_UUIDS);
 }
 
-TEST_F(ExprTest, DateTruncTest) {
+TEST_P(ExprTest, DateTruncTest) {
   TestTimestampValue("date_trunc('MILLENNIUM', '2016-05-08 10:30:00')",
       TimestampValue::Parse("2001-01-01 00:00:00"));
   TestTimestampValue("date_trunc('MILLENNIUM', '3000-12-31 23:59:59.999999999')",
@@ -9081,7 +9143,7 @@ TEST_F(ExprTest, DateTruncTest) {
   TestError("date_trunc('2017-01-09 10:00:00', 'HOUR')");
 }
 
-TEST_F(ExprTest, JsonTest) {
+TEST_P(ExprTest, JsonTest) {
   TestStringValue("get_json_object('{\"a\":1, \"b\":2, \"c\":3}', '$.b')", "2");
   TestStringValue(
       "get_json_object('{\"a\":true, \"b\":false, \"c\":true}', '$.b')", "false");
@@ -9225,77 +9287,25 @@ TEST_F(ExprTest, JsonTest) {
 }
 } // namespace impala
 
+INSTANTIATE_TEST_CASE_P(Instantiations, ExprTest, ::testing::Values(
+  //              disable_codegen  enable_expr_rewrites
+  std::make_tuple(true,            false),
+  std::make_tuple(false,           false),
+  std::make_tuple(true,            true)));
+  // Note: the false/true case is not tested because it provides very little
+  // incremental coverage but adds a lot to test runtime (this test is quite
+  // slow when codegen is enabled).
+  //
+  // Mostly we get the best backend codegened/interpreted coverage from running
+  // the unmodified (i.e. more complex) expr trees enabled. Running the trees
+  // in interpreted mode should be enough to validate correctness of the
+  // rewrites. So enabling the remaining combination might provide some
+  // additional incidental coverage from codegening some additional expr tree
+  // shapes but mostly it isn't that interesting since the majority of
+  // expressions get folded to a constant anyway.
+
 int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
   InitCommonRuntime(argc, argv, true, TestInfo::BE_TEST);
-  InitFeSupport(false);
-  ABORT_IF_ERROR(impala::LlvmCodeGen::InitializeLlvm());
-
-  // The host running this test might have an out-of-date tzdata package installed.
-  // To avoid tzdata related issues, we will load time-zone db from the testdata
-  // directory.
-  FLAGS_hdfs_zone_info_zip = Substitute("file://$0/testdata/tzdb/2017c.zip",
-      getenv("IMPALA_HOME"));
-  ABORT_IF_ERROR(TimezoneDatabase::Initialize());
-
-  // Disable llvm optimization passes if the env var is no set to true. Running without
-  // the optimizations makes the tests run much faster.
-  char* optimizations = getenv("EXPR_TEST_ENABLE_OPTIMIZATIONS");
-  if (optimizations != NULL && strcmp(optimizations, "true") == 0) {
-    cout << "Running with optimization passes." << endl;
-    FLAGS_disable_optimization_passes = false;
-  } else {
-    cout << "Running without optimization passes." << endl;
-    FLAGS_disable_optimization_passes = true;
-  }
-
-  // Create an in-process Impala server and in-process backends for test environment
-  // without any startup validation check
-  FLAGS_abort_on_config_error = false;
-  VLOG_CONNECTION << "creating test env";
-  VLOG_CONNECTION << "starting backends";
-  statestore = new Statestore(statestore_metrics.get());
-  IGNORE_LEAKING_OBJECT(statestore);
-
-  // Pass in 0 to have the statestore use an ephemeral port for the service.
-  ABORT_IF_ERROR(statestore->Init(0));
-  InProcessImpalaServer* impala_server;
-  ABORT_IF_ERROR(InProcessImpalaServer::StartWithEphemeralPorts(
-      FLAGS_hostname, statestore->port(), &impala_server));
-  IGNORE_LEAKING_OBJECT(impala_server);
-
-  executor_ = new ImpaladQueryExecutor(FLAGS_hostname, impala_server->GetBeeswaxPort());
-  ABORT_IF_ERROR(executor_->Setup());
-
-  // Disable FE expr rewrites to make sure the Exprs get executed exactly as specified
-  // in the tests here.
-  int ret;
-  disable_codegen_ = true;
-  enable_expr_rewrites_ = false;
-  executor_->ClearExecOptions();
-  executor_->PushExecOption("ENABLE_EXPR_REWRITES=0");
-  executor_->PushExecOption("DISABLE_CODEGEN=1");
-  cout << "Running without codegen and without expr rewrites" << endl;
-  ret = RUN_ALL_TESTS();
-  if (ret != 0) return ret;
-
-  disable_codegen_ = false;
-  enable_expr_rewrites_ = false;
-  executor_->ClearExecOptions();
-  executor_->PushExecOption("ENABLE_EXPR_REWRITES=0");
-  executor_->PushExecOption("DISABLE_CODEGEN=0");
-  executor_->PushExecOption("EXEC_SINGLE_NODE_ROWS_THRESHOLD=0");
-  executor_->PushExecOption("DISABLE_CODEGEN_ROWS_THRESHOLD=0");
-  cout << endl << "Running with codegen and without expr rewrites" << endl;
-  ret = RUN_ALL_TESTS();
-  if (ret != 0) return ret;
-
-  // Enable FE expr rewrites to get test for constant folding over all exprs.
-  disable_codegen_ = true;
-  enable_expr_rewrites_ = true;
-  executor_->ClearExecOptions();
-  executor_->PushExecOption("ENABLE_EXPR_REWRITES=1");
-  executor_->PushExecOption("DISABLE_CODEGEN=1");
-  cout << endl << "Running without codegen and with expr rewrites" << endl;
   return RUN_ALL_TESTS();
 }