You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2020/10/27 21:30:04 UTC

[iceberg] branch master updated: Hive: Add TestHiveShell for parameterized StorageHandler tests (#1631)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d3eb523  Hive: Add TestHiveShell for parameterized StorageHandler tests (#1631)
d3eb523 is described below

commit d3eb523800530c917899b7036c674d48206461ad
Author: Marton Bod <ma...@gmail.com>
AuthorDate: Tue Oct 27 22:29:55 2020 +0100

    Hive: Add TestHiveShell for parameterized StorageHandler tests (#1631)
    
    Co-authored-by: Marton Bod <mb...@cloudera.com>
---
 build.gradle                                       |  11 +-
 .../mr/hive/HiveIcebergStorageHandlerBaseTest.java | 151 ++++++------------
 .../org/apache/iceberg/mr/hive/TestHiveShell.java  | 176 +++++++++++++++++++++
 versions.props                                     |   1 +
 4 files changed, 226 insertions(+), 113 deletions(-)

diff --git a/build.gradle b/build.gradle
index 2b67d50..81e4a94 100644
--- a/build.gradle
+++ b/build.gradle
@@ -463,11 +463,8 @@ project(':iceberg-mr') {
     testCompile("org.apache.calcite:calcite-core")
     testCompile("com.esotericsoftware:kryo-shaded:4.0.2")
     testCompile("com.fasterxml.jackson.core:jackson-annotations:2.6.5")
-    testCompile("com.klarna:hiverunner:5.2.1") {
-      exclude group: 'javax.jms', module: 'jms'
+    testCompile("org.apache.hive:hive-service") {
       exclude group: 'org.apache.hive', module: 'hive-exec'
-      exclude group: 'org.codehaus.jettison', module: 'jettison'
-      exclude group: 'org.apache.calcite.avatica'
     }
   }
 
@@ -530,12 +527,10 @@ if (jdkVersion == '8') {
       testCompile("org.apache.calcite:calcite-core")
       testCompile("com.esotericsoftware:kryo-shaded:4.0.2")
       testCompile("com.fasterxml.jackson.core:jackson-annotations:2.6.5")
-      testCompile("com.klarna:hiverunner:6.0.1") {
-        exclude group: 'javax.jms', module: 'jms'
+      testCompile("org.apache.hive:hive-service:3.1.2") {
         exclude group: 'org.apache.hive', module: 'hive-exec'
-        exclude group: 'org.codehaus.jettison', module: 'jettison'
-        exclude group: 'org.apache.calcite.avatica'
       }
+      testCompile("org.apache.tez:tez-dag:0.9.1")
     }
 
     test {
diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerBaseTest.java b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerBaseTest.java
index b71dd1b..2912a64 100644
--- a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerBaseTest.java
+++ b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerBaseTest.java
@@ -19,9 +19,6 @@
 
 package org.apache.iceberg.mr.hive;
 
-import com.klarna.hiverunner.HiveShell;
-import com.klarna.hiverunner.StandaloneHiveRunner;
-import com.klarna.hiverunner.annotations.HiveSQL;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
@@ -33,7 +30,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.StatsSetupConst;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.iceberg.AssertHelpers;
 import org.apache.iceberg.BaseMetastoreTableOperations;
@@ -48,7 +44,6 @@ import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.exceptions.NoSuchTableException;
 import org.apache.iceberg.hadoop.Util;
-import org.apache.iceberg.hive.TestHiveMetastore;
 import org.apache.iceberg.mr.Catalogs;
 import org.apache.iceberg.mr.InputFormatConfig;
 import org.apache.iceberg.mr.TestHelper;
@@ -65,15 +60,15 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.junit.runners.Parameterized.Parameter;
+import static org.junit.runners.Parameterized.Parameters;
 
-@RunWith(StandaloneHiveRunner.class)
+@RunWith(Parameterized.class)
 public abstract class HiveIcebergStorageHandlerBaseTest {
 
-  @HiveSQL(files = {}, autoStart = false)
-  private HiveShell shell;
-
   @Rule
   public TemporaryFolder temp = new TemporaryFolder();
 
@@ -108,117 +103,62 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
       ImmutableSet.of("bucketing_version", StatsSetupConst.ROW_COUNT,
           StatsSetupConst.RAW_DATA_SIZE, StatsSetupConst.TOTAL_SIZE, StatsSetupConst.NUM_FILES, "numFilesErasureCoded");
 
-  private static final int METASTORE_POOL_SIZE = 15;
-
-  // before variables
-  protected static TestHiveMetastore metastore;
+  private static TestHiveShell shell;
 
   private TestTables testTables;
 
   public abstract TestTables testTables(Configuration conf, TemporaryFolder tmp) throws IOException;
 
+  @Parameters(name = "fileFormat={0}")
+  public static Iterable<FileFormat> fileFormats() {
+    return ImmutableList.of(FileFormat.PARQUET, FileFormat.ORC, FileFormat.AVRO);
+  }
+
+  @Parameter
+  public FileFormat fileFormat;
 
   @BeforeClass
   public static void beforeClass() {
-    metastore = new TestHiveMetastore();
-    // We need to use increased pool size in these tests. See: #1620
-    metastore.start(METASTORE_POOL_SIZE);
+    shell = new TestHiveShell();
+    shell.setHiveConfValue("hive.notification.event.poll.interval", "-1");
+    shell.start();
   }
 
   @AfterClass
   public static void afterClass() {
-    metastore.stop();
-    metastore = null;
+    shell.stop();
   }
 
   @Before
   public void before() throws IOException {
-    String metastoreUris = metastore.hiveConf().getVar(HiveConf.ConfVars.METASTOREURIS);
-
-    testTables = testTables(metastore.hiveConf(), temp);
-
+    shell.openSession();
+    testTables = testTables(shell.metastore().hiveConf(), temp);
     for (Map.Entry<String, String> property : testTables.properties().entrySet()) {
-      shell.setHiveConfValue(property.getKey(), property.getValue());
+      shell.setHiveSessionValue(property.getKey(), property.getValue());
     }
-
-    shell.setHiveConfValue(HiveConf.ConfVars.METASTOREURIS.varname, metastoreUris);
-    String metastoreWarehouse = metastore.hiveConf().getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
-    shell.setHiveConfValue(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, metastoreWarehouse);
-
-    // Notification uses another HMSClient which we do not use in the tests, so we turn this off.
-    shell.setHiveConfValue("hive.notification.event.poll.interval", "-1");
-    shell.start();
   }
 
   @After
   public void after() throws Exception {
-    metastore.reset();
+    shell.closeSession();
+    shell.metastore().reset();
     // HiveServer2 thread pools are using thread local Hive -> HMSClient objects. These are not cleaned up when the
     // HiveServer2 is stopped. Only Finalizer closes the HMS connections.
     System.gc();
   }
 
-  // PARQUET
-
-  @Test
-  public void testScanEmptyTableParquet() throws IOException {
-    testScanEmptyTable(FileFormat.PARQUET);
-  }
-
   @Test
-  public void testScanTableParquet() throws IOException {
-    testScanTable(FileFormat.PARQUET);
-  }
-
-  @Test
-  public void testJoinTablesParquet() throws IOException {
-    testJoinTables(FileFormat.PARQUET);
-  }
-
-  // ORC
-
-  @Test
-  public void testScanEmptyTableORC() throws IOException {
-    testScanEmptyTable(FileFormat.ORC);
-  }
-
-  @Test
-  public void testScanTableORC() throws IOException {
-    testScanTable(FileFormat.ORC);
-  }
-
-  @Test
-  public void testJoinTablesORC() throws IOException {
-    testJoinTables(FileFormat.ORC);
-  }
-
-  // AVRO
-
-  @Test
-  public void testScanEmptyTableAvro() throws IOException {
-    testScanEmptyTable(FileFormat.AVRO);
-  }
-
-  @Test
-  public void testScanTableAvro() throws IOException {
-    testScanTable(FileFormat.AVRO);
-  }
-
-  @Test
-  public void testJoinTablesAvro() throws IOException {
-    testJoinTables(FileFormat.AVRO);
-  }
-
-  public void testScanEmptyTable(FileFormat format) throws IOException {
+  public void testScanEmptyTable() throws IOException {
     Schema emptySchema = new Schema(required(1, "empty", Types.StringType.get()));
-    createTable("empty", emptySchema, format, ImmutableList.of());
+    createTable("empty", emptySchema, ImmutableList.of());
 
     List<Object[]> rows = shell.executeStatement("SELECT * FROM default.empty");
     Assert.assertEquals(0, rows.size());
   }
 
-  public void testScanTable(FileFormat format) throws IOException {
-    createTable("customers", CUSTOMER_SCHEMA, format, CUSTOMER_RECORDS);
+  @Test
+  public void testScanTable() throws IOException {
+    createTable("customers", CUSTOMER_SCHEMA, CUSTOMER_RECORDS);
 
     // Single fetch task: no MR job.
     List<Object[]> rows = shell.executeStatement("SELECT * FROM default.customers");
@@ -237,9 +177,10 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
     Assert.assertArrayEquals(new Object[] {0L, "Alice"}, descRows.get(2));
   }
 
-  public void testJoinTables(FileFormat format) throws IOException {
-    createTable("customers", CUSTOMER_SCHEMA, format, CUSTOMER_RECORDS);
-    createTable("orders", ORDER_SCHEMA, format, ORDER_RECORDS);
+  @Test
+  public void testJoinTables() throws IOException {
+    createTable("customers", CUSTOMER_SCHEMA, CUSTOMER_RECORDS);
+    createTable("orders", ORDER_SCHEMA, ORDER_RECORDS);
 
     List<Object[]> rows = shell.executeStatement(
             "SELECT c.customer_id, c.first_name, o.order_id, o.total " +
@@ -276,7 +217,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
 
     // Check the HMS table parameters
     org.apache.hadoop.hive.metastore.api.Table hmsTable =
-        metastore.clientPool().run(client -> client.getTable("default", "customers"));
+        shell.metastore().clientPool().run(client -> client.getTable("default", "customers"));
 
     Map<String, String> hmsParams = hmsTable.getParameters();
     IGNORED_PARAMS.forEach(hmsParams::remove);
@@ -317,7 +258,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
       Assert.assertEquals(expectedIcebergProperties, icebergTable.properties());
 
       // Check the HMS table parameters
-      hmsTable = metastore.clientPool().run(client -> client.getTable("default", "customers"));
+      hmsTable = shell.metastore().clientPool().run(client -> client.getTable("default", "customers"));
       Path hmsTableLocation = new Path(hmsTable.getSd().getLocation());
 
       // Drop the table
@@ -362,7 +303,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
 
     // Check the HMS table parameters
     org.apache.hadoop.hive.metastore.api.Table hmsTable =
-        metastore.clientPool().run(client -> client.getTable("default", "customers"));
+        shell.metastore().clientPool().run(client -> client.getTable("default", "customers"));
 
     Map<String, String> hmsParams = hmsTable.getParameters();
     IGNORED_PARAMS.forEach(hmsParams::remove);
@@ -399,7 +340,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
 
     // Check the HMS table parameters
     org.apache.hadoop.hive.metastore.api.Table hmsTable =
-        metastore.clientPool().run(client -> client.getTable("default", "customers"));
+        shell.metastore().clientPool().run(client -> client.getTable("default", "customers"));
 
     Map<String, String> hmsParams = hmsTable.getParameters();
     IGNORED_PARAMS.forEach(hmsParams::remove);
@@ -437,7 +378,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
     } else {
       // Check the HMS table parameters
       org.apache.hadoop.hive.metastore.api.Table hmsTable =
-          metastore.clientPool().run(client -> client.getTable("default", "customers"));
+          shell.metastore().clientPool().run(client -> client.getTable("default", "customers"));
       Path hmsTableLocation = new Path(hmsTable.getSd().getLocation());
 
       // Drop the table
@@ -464,7 +405,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
     // Wrong schema
     AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class,
         "Unrecognized token 'WrongSchema'", () -> {
-          shell.executeQuery("CREATE EXTERNAL TABLE withShell2 " +
+          shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " +
               "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " +
               (location != null ? "LOCATION '" + location + "' " : "") +
               "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='WrongSchema')");
@@ -474,7 +415,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
     // Missing schema, we try to get the schema from the table and fail
     AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class,
         "Please provide an existing table or a valid schema", () -> {
-          shell.executeQuery("CREATE EXTERNAL TABLE withShell2 " +
+          shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " +
               "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " +
               (location != null ? "LOCATION '" + location + "' " : ""));
         }
@@ -484,7 +425,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
       // Missing location
       AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class,
           "Table location not set", () -> {
-            shell.executeQuery("CREATE EXTERNAL TABLE withShell2 " +
+            shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " +
                 "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " +
                 "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" +
                 SchemaParser.toJson(CUSTOMER_SCHEMA) + "')");
@@ -496,14 +437,14 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
   @Test
   public void testCreateTableAboveExistingTable() throws TException, IOException, InterruptedException {
     // Create the Iceberg table
-    createIcebergTable("customers", CUSTOMER_SCHEMA, FileFormat.PARQUET, Collections.emptyList());
+    createIcebergTable("customers", CUSTOMER_SCHEMA, Collections.emptyList());
 
     if (Catalogs.hiveCatalog(shell.getHiveConf())) {
 
       // In HiveCatalog we just expect an exception since the table is already exists
       AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class,
           "customers already exists", () -> {
-            shell.executeQuery("CREATE EXTERNAL TABLE customers " +
+            shell.executeStatement("CREATE EXTERNAL TABLE customers " +
                 "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " +
                 "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" +
                 SchemaParser.toJson(CUSTOMER_SCHEMA) + "')");
@@ -525,7 +466,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
 
       // Check the HMS table parameters
       org.apache.hadoop.hive.metastore.api.Table hmsTable =
-          metastore.clientPool().run(client -> client.getTable("default", "customers"));
+          shell.metastore().clientPool().run(client -> client.getTable("default", "customers"));
 
       Map<String, String> hmsParams = hmsTable.getParameters();
       IGNORED_PARAMS.forEach(hmsParams::remove);
@@ -540,17 +481,17 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
     }
   }
 
-  protected void createTable(String tableName, Schema schema, FileFormat format, List<Record> records)
+  protected void createTable(String tableName, Schema schema, List<Record> records)
           throws IOException {
-    Table table = createIcebergTable(tableName, schema, format, records);
+    Table table = createIcebergTable(tableName, schema, records);
     createHiveTable(tableName, table.location());
   }
 
-  protected Table createIcebergTable(String tableName, Schema schema, FileFormat format, List<Record> records)
+  protected Table createIcebergTable(String tableName, Schema schema, List<Record> records)
           throws IOException {
     String identifier = testTables.identifier("default." + tableName);
     TestHelper helper = new TestHelper(
-            metastore.hiveConf(), testTables.tables(), identifier, schema, SPEC, format, temp);
+            shell.metastore().hiveConf(), testTables.tables(), identifier, schema, SPEC, fileFormat, temp);
     Table table = helper.createTable();
 
     if (!records.isEmpty()) {
@@ -561,7 +502,7 @@ public abstract class HiveIcebergStorageHandlerBaseTest {
   }
 
   protected void createHiveTable(String tableName, String location) {
-    shell.execute(String.format(
+    shell.executeStatement(String.format(
             "CREATE TABLE default.%s " +
             "STORED BY '%s' " +
             "LOCATION '%s'",
diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java
new file mode 100644
index 0000000..7be1518
--- /dev/null
+++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.mr.hive;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.cli.CLIService;
+import org.apache.hive.service.cli.HiveSQLException;
+import org.apache.hive.service.cli.OperationHandle;
+import org.apache.hive.service.cli.RowSet;
+import org.apache.hive.service.cli.SessionHandle;
+import org.apache.hive.service.cli.session.HiveSession;
+import org.apache.hive.service.server.HiveServer2;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Test class for running HiveQL queries, essentially acting like a Beeline shell in tests.
+ *
+ * It spins up both an HS2 and a Metastore instance to work with. The shell will only accept
+ * queries if it has been previously initialized via {@link #start()}, and a session has been opened via
+ * {@link #openSession()}. Prior to calling {@link #start()}, the shell should first be configured with props that apply
+ * across all test cases by calling {@link #setHiveConfValue(String, String)} ()}. On the other hand, session-level conf
+ * can be applied anytime via {@link #setHiveSessionValue(String, String)} ()}, once we've opened an active session.
+ */
+public class TestHiveShell {
+
+  private final TestHiveMetastore metastore;
+  private final HiveServer2 hs2;
+  private final HiveConf hs2Conf;
+  private CLIService client;
+  private HiveSession session;
+  private boolean started;
+
+  public TestHiveShell() {
+    metastore = new TestHiveMetastore();
+    hs2Conf = initializeConf();
+    hs2 = new HiveServer2();
+  }
+
+  public void setHiveConfValue(String key, String value) {
+    Preconditions.checkState(!started, "TestHiveShell has already been started. Cannot set Hive conf anymore.");
+    hs2Conf.verifyAndSet(key, value);
+  }
+
+  public void setHiveSessionValue(String key, String value) {
+    Preconditions.checkState(session != null, "There is no open session for setting variables.");
+    try {
+      session.getSessionConf().set(key, value);
+    } catch (Exception e) {
+      throw new RuntimeException("Unable to set Hive session variable: ", e);
+    }
+  }
+
+  public void start() {
+    metastore.start();
+    hs2Conf.setVar(HiveConf.ConfVars.METASTOREURIS, metastore.hiveConf().getVar(HiveConf.ConfVars.METASTOREURIS));
+    hs2Conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE,
+        metastore.hiveConf().getVar(HiveConf.ConfVars.METASTOREWAREHOUSE));
+
+    hs2.init(hs2Conf);
+    hs2.start();
+    client = hs2.getServices().stream()
+            .filter(CLIService.class::isInstance)
+            .findFirst()
+            .map(CLIService.class::cast)
+            .get();
+    started = true;
+  }
+
+  public void stop() {
+    if (client != null) {
+      client.stop();
+    }
+    hs2.stop();
+    metastore.stop();
+    started = false;
+  }
+
+  public TestHiveMetastore metastore() {
+    return metastore;
+  }
+
+  public void openSession() {
+    Preconditions.checkState(started, "You have to start TestHiveShell first, before opening a session.");
+    try {
+      SessionHandle sessionHandle = client.getSessionManager().openSession(
+          CLIService.SERVER_VERSION, "", "", "127.0.0.1", Collections.emptyMap());
+      session = client.getSessionManager().getSession(sessionHandle);
+    } catch (Exception e) {
+      throw new RuntimeException("Unable to open new Hive session: ", e);
+    }
+  }
+
+  public void closeSession() {
+    Preconditions.checkState(session != null, "There is no open session to be closed.");
+    try {
+      session.close();
+      session = null;
+    } catch (Exception e) {
+      throw new RuntimeException("Unable to close Hive session: ", e);
+    }
+  }
+
+  public List<Object[]> executeStatement(String statement) {
+    Preconditions.checkState(session != null,
+            "You have to start TestHiveShell and open a session first, before running a query.");
+    try {
+      OperationHandle handle = client.executeStatement(session.getSessionHandle(), statement, Collections.emptyMap());
+      List<Object[]> resultSet = new ArrayList<>();
+      if (handle.hasResultSet()) {
+        RowSet rowSet;
+        // keep fetching results until we can
+        while ((rowSet = client.fetchResults(handle)) != null && rowSet.numRows() > 0) {
+          for (Object[] row : rowSet) {
+            resultSet.add(row.clone());
+          }
+        }
+      }
+      return resultSet;
+    } catch (HiveSQLException e) {
+      throw new IllegalArgumentException("Failed to execute Hive query '" + statement + "': " + e.getMessage(), e);
+    }
+  }
+
+  public Configuration getHiveConf() {
+    if (session != null) {
+      return session.getHiveConf();
+    } else {
+      return hs2Conf;
+    }
+  }
+
+  private HiveConf initializeConf() {
+    HiveConf hiveConf = new HiveConf();
+
+    // Use ephemeral port to enable running tests in parallel
+    hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT, 0);
+    // Disable the web UI
+    hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_WEBUI_PORT, -1);
+
+    // Switch off optimizers in order to contain the map reduction within this JVM
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_CBO_ENABLED, false);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_INFER_BUCKET_SORT, false);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVEMETADATAONLYQUERIES, false);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER, false);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN, false);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVESKEWJOIN, false);
+
+    // Speed up test execution
+    hiveConf.setLongVar(HiveConf.ConfVars.HIVECOUNTERSPULLINTERVAL, 1L);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false);
+
+    return hiveConf;
+  }
+}
diff --git a/versions.props b/versions.props
index f110b43..273ae64 100644
--- a/versions.props
+++ b/versions.props
@@ -22,3 +22,4 @@ com.github.stephenc.findbugs:findbugs-annotations = 1.3.9-1
 junit:junit = 4.12
 org.mockito:mockito-core = 1.10.19
 org.apache.hive:hive-exec = 2.3.7
+org.apache.hive:hive-service = 2.3.7