You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vo...@apache.org on 2019/01/25 16:49:10 UTC

[drill] 06/08: DRILL-6977: Improve Hive tests configuration

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

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

commit a00f1800b95a007b0890f97fd260a6085911438b
Author: Igor Guzenko <ih...@gmail.com>
AuthorDate: Fri Jan 18 20:27:54 2019 +0200

    DRILL-6977: Improve Hive tests configuration
    
    1. HiveTestBase data initialization moved to static block
       to be initialized once for all derivatives.
    2. Extracted Hive driver and storage plugin management from HiveTestDataGenerator
       to HiveTestFixture class. This increased cohesion of generator and
       added loose coupling between hive test configuration and data generation
       tasks.
    3. Replaced usage of Guava ImmutableLists with TestBaseViewSupport
       helper methods by using standard JDK collections.
    
    closes #1613
---
 .../org/apache/drill/exec/hive/HiveTestBase.java   |  44 ++-
 .../apache/drill/exec/hive/HiveTestFixture.java    | 295 +++++++++++++++++++++
 .../apache/drill/exec/hive/HiveTestUtilities.java  |  36 +++
 .../apache/drill/exec/hive/TestHiveStorage.java    |  34 +--
 .../hive/BaseTestHiveImpersonation.java            |  18 +-
 .../exec/sql/hive/TestViewSupportOnHiveTables.java |  47 ++--
 .../exec/store/hive/HiveTestDataGenerator.java     | 137 ++--------
 .../apache/drill/exec/sql/TestBaseViewSupport.java |  38 +++
 .../org/apache/drill/exec/sql/TestViewSupport.java | 106 +++-----
 9 files changed, 510 insertions(+), 245 deletions(-)

diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java
index 5758eec..c3acdb0 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java
@@ -17,28 +17,58 @@
  */
 package org.apache.drill.exec.hive;
 
+import java.io.File;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.exec.store.hive.HiveTestDataGenerator;
+import org.apache.drill.test.BaseDirTestWatcher;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.runner.Description;
 
 /**
  * Base class for Hive test. Takes care of generating and adding Hive test plugin before tests and deleting the
  * plugin after tests.
  */
 public class HiveTestBase extends PlanTestBase {
-  protected static HiveTestDataGenerator hiveTest;
+
+  public static final HiveTestFixture HIVE_TEST_FIXTURE;
+
+  static {
+    // generate hive data common for all test classes using own dirWatcher
+    BaseDirTestWatcher generalDirWatcher = new BaseDirTestWatcher() {
+      {
+        /*
+           Below protected method invoked to create directory DirWatcher.dir with path like:
+           ./target/org.apache.drill.exec.hive.HiveTestBase123e4567-e89b-12d3-a456-556642440000.
+           Then subdirectory with name 'root' will be used to hold metastore_db and other data shared between
+           all derivatives of the class. Note that UUID suffix is necessary to avoid conflicts between forked JVMs.
+        */
+        starting(Description.createSuiteDescription(HiveTestBase.class.getName().concat(UUID.randomUUID().toString())));
+      }
+    };
+    File baseDir = generalDirWatcher.getRootDir();
+    HIVE_TEST_FIXTURE = HiveTestFixture.builder(baseDir).build();
+    HiveTestDataGenerator dataGenerator = new HiveTestDataGenerator(generalDirWatcher, baseDir,
+        HIVE_TEST_FIXTURE.getWarehouseDir());
+    HIVE_TEST_FIXTURE.getDriverManager().runWithinSession(dataGenerator::generateData);
+
+    // set hook for clearing watcher's dir on JVM shutdown
+    Runtime.getRuntime().addShutdownHook(new Thread(() -> FileUtils.deleteQuietly(generalDirWatcher.getDir())));
+  }
 
   @BeforeClass
-  public static void generateHive() throws Exception {
-    hiveTest = HiveTestDataGenerator.getInstance(dirTestWatcher);
-    hiveTest.addHiveTestPlugin(getDrillbitContext().getStorage());
+  public static void setUp() {
+    HIVE_TEST_FIXTURE.getPluginManager().addHivePluginTo(bits);
   }
 
   @AfterClass
-  public static void cleanupHiveTestData() throws Exception{
-    if (hiveTest != null) {
-      hiveTest.deleteHiveTestPlugin(getDrillbitContext().getStorage());
+  public static void tearDown() {
+    if (HIVE_TEST_FIXTURE != null) {
+      HIVE_TEST_FIXTURE.getPluginManager().removeHivePluginFrom(bits);
     }
   }
+
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestFixture.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestFixture.java
new file mode 100644
index 0000000..5078272
--- /dev/null
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestFixture.java
@@ -0,0 +1,295 @@
+/*
+ * 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.drill.exec.hive;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Consumer;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.hive.HiveStoragePlugin;
+import org.apache.drill.exec.store.hive.HiveStoragePluginConfig;
+import org.apache.drill.test.BaseDirTestWatcher;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import static java.util.Objects.nonNull;
+import static java.util.Objects.requireNonNull;
+import static org.apache.drill.exec.hive.HiveTestUtilities.createDirWithPosixPermissions;
+
+
+/**
+ * Test fixture for configuration of Hive tests, which
+ * allows granular control over initialization of test data
+ * and hive storage plugin.
+ * <p>
+ * Below is example of usage HiveTestFixture along with ClusterFixture:
+ * <p>
+ * <pre><code>
+ *   // Note that HiveTestFixture doesn't require extension of ClusterTest,
+ *   // this is just the simplest way for configuring test drillbit
+ *   public class HiveTestExample extends ClusterTest {
+ *
+ *       private static HiveTestFixture hiveTestFixture;
+ *
+ *       {@literal @}BeforeClass
+ *       public static void setUp() throws Exception {
+ *            startCluster(ClusterFixture.builder(dirTestWatcher));
+ *
+ *            // Below is minimal config which uses defaults from HiveTestFixture.Builder
+ *            // constructor, but any option for driver or storage plugin may be
+ *            // overridden using builder's methods
+ *            hiveTestFixture = HiveTestFixture.builder(dirTestWatcher).build();
+ *
+ *            // Use driver manager to configure test data in Hive metastore
+ *            hiveTestFixture.getDriverManager().runWithinSession(HiveTestExample::generateData);
+ *
+ *            // Use plugin manager to add, remove, update hive storage plugin of one or many test drillbits
+ *            hiveTestFixture.getPluginManager().addHivePluginTo(cluster.drillbits());
+ *       }
+ *
+ *       private static void generateData(Driver driver) {
+ *            // Set up data using HiveTestUtilities.executeQuery(driver, sql)
+ *       }
+ *
+ *       {@literal @}AfterClass
+ *       public static void tearDown() throws Exception {
+ *            if (nonNull(hiveTestFixture)) {
+ *                hiveTestFixture.getPluginManager().removeHivePluginFrom(cluster.drillbits());
+ *            }
+ *       }
+ * }
+ * </code></pre>
+ */
+public class HiveTestFixture {
+
+  private final Map<String, String> pluginConf;
+
+  private final Map<String, String> driverConf;
+
+  private final String pluginName;
+
+  private final HivePluginManager pluginManager;
+
+  private final HiveDriverManager driverManager;
+
+  private HiveTestFixture(Builder builder) {
+    this.pluginConf = new HashMap<>(builder.pluginConf);
+    this.driverConf = new HashMap<>(builder.driverConf);
+    this.pluginName = builder.pluginName;
+    this.pluginManager = new HivePluginManager();
+    this.driverManager = new HiveDriverManager();
+  }
+
+  public static Builder builder(BaseDirTestWatcher dirWatcher) {
+    return builder(requireNonNull(dirWatcher, "Parameter 'dirWatcher' can't be null!").getRootDir());
+  }
+
+  public static Builder builder(File baseDir) {
+    return new Builder(requireNonNull(baseDir, "Parameter 'baseDir' can't be null!"));
+  }
+
+  public HivePluginManager getPluginManager() {
+    return pluginManager;
+  }
+
+  public HiveDriverManager getDriverManager() {
+    return driverManager;
+  }
+
+  /**
+   * Returns current value of 'hive.metastore.warehouse.dir' option
+   * which expected to represent location of metastore warehouse directory.
+   * Builder's user can override any option either of pluginConf or driverConf.
+   * Since setting of the option is not enforced, this method just tries to
+   * find it in any of the conf maps.
+   *
+   * @return current value of 'hive.metastore.warehouse.dir' option
+   *         from pluginConf or driverConf
+   */
+  public String getWarehouseDir() {
+    String warehouseDir = pluginConf.get(ConfVars.METASTOREWAREHOUSE.varname);
+    return nonNull(warehouseDir) ? warehouseDir : driverConf.get(ConfVars.METASTOREWAREHOUSE.varname);
+  }
+
+  public static class Builder {
+
+    private final Map<String, String> pluginConf;
+
+    private final Map<String, String> driverConf;
+
+    private String pluginName;
+
+    private Builder(File baseDir) {
+      this.pluginConf = new HashMap<>();
+      this.driverConf = new HashMap<>();
+      String jdbcUrl = String.format("jdbc:derby:;databaseName=%s;create=true",
+          new File(baseDir, "metastore_db").getAbsolutePath());
+      String warehouseDir = new File(baseDir, "warehouse").getAbsolutePath();
+      // Drill Hive Storage plugin defaults
+      pluginName("hive");
+      pluginOption(ConfVars.METASTOREURIS, "");
+      pluginOption(ConfVars.METASTORECONNECTURLKEY, jdbcUrl);
+      pluginOption(ConfVars.METASTOREWAREHOUSE, warehouseDir);
+      pluginOption(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
+      // Hive Driver defaults
+      driverOption(ConfVars.METASTORECONNECTURLKEY, jdbcUrl);
+      driverOption(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
+      driverOption(ConfVars.METASTOREWAREHOUSE, warehouseDir);
+      driverOption("mapred.job.tracker", "local");
+      driverOption(ConfVars.SCRATCHDIR, createDirWithPosixPermissions(baseDir, "scratch_dir").getAbsolutePath());
+      driverOption(ConfVars.LOCALSCRATCHDIR, createDirWithPosixPermissions(baseDir, "local_scratch_dir").getAbsolutePath());
+      driverOption(ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict");
+      driverOption(ConfVars.METASTORE_AUTO_CREATE_ALL, Boolean.toString(true));
+      driverOption(ConfVars.METASTORE_SCHEMA_VERIFICATION, Boolean.toString(false));
+      driverOption(ConfVars.HIVE_CBO_ENABLED, Boolean.toString(false));
+    }
+
+    public Builder pluginOption(ConfVars option, String value) {
+      return pluginOption(option.varname, value);
+    }
+
+    public Builder pluginOption(String option, String value) {
+      return put(pluginConf, option, value);
+    }
+
+    public Builder driverOption(ConfVars option, String value) {
+      return driverOption(option.varname, value);
+    }
+
+    public Builder driverOption(String option, String value) {
+      return put(driverConf, option, value);
+    }
+
+    public Builder pluginName(String name) {
+      this.pluginName = Objects.requireNonNull(name, "Hive plugin name can't be null!");
+      return this;
+    }
+
+    private Builder put(Map<String, String> map, String key, String value) {
+      map.put(key, value);
+      return this;
+    }
+
+    public HiveTestFixture build() {
+      return new HiveTestFixture(this);
+    }
+
+  }
+
+  /**
+   * Implements addition, update and deletion of
+   * Hive storage plugin for drillbits passed from outside.
+   * The class was made inner because it uses pluginName and pluginConf
+   * of enclosing fixture instance.
+   */
+  public class HivePluginManager {
+
+    /**
+     *  {@link HiveTestFixture}'s constructor will create instance,
+     *  and API users will get it via {@link HiveTestFixture#getPluginManager()}.
+     */
+    private HivePluginManager() {
+    }
+
+    public void addHivePluginTo(Drillbit... drillbits) {
+      addHivePluginTo(Arrays.asList(drillbits));
+    }
+
+    public void addHivePluginTo(Iterable<Drillbit> drillbits) {
+      try {
+        for (Drillbit drillbit : drillbits) {
+          HiveStoragePluginConfig pluginConfig = new HiveStoragePluginConfig(new HashMap<>(pluginConf));
+          pluginConfig.setEnabled(true);
+          drillbit.getContext().getStorage().createOrUpdate(pluginName, pluginConfig, true);
+        }
+      } catch (ExecutionSetupException e) {
+        throw new RuntimeException("Failed to add Hive storage plugin to drillbits", e);
+      }
+    }
+
+    public void removeHivePluginFrom(Drillbit... drillbits) {
+      removeHivePluginFrom(Arrays.asList(drillbits));
+    }
+
+    public void removeHivePluginFrom(Iterable<Drillbit> drillbits) {
+      drillbits.forEach(bit -> bit.getContext().getStorage().deletePlugin(pluginName));
+    }
+
+    public void updateHivePlugin(Iterable<Drillbit> drillbits,
+                                 Map<String, String> configOverride) {
+      try {
+        for (Drillbit drillbit : drillbits) {
+          StoragePluginRegistry pluginRegistry = drillbit.getContext().getStorage();
+          HiveStoragePlugin storagePlugin = Objects.requireNonNull(
+              (HiveStoragePlugin) pluginRegistry.getPlugin(pluginName),
+              String.format("Hive storage plugin with name '%s' doesn't exist.", pluginName));
+
+          HiveStoragePluginConfig newPluginConfig = storagePlugin.getConfig();
+          newPluginConfig.getConfigProps().putAll(configOverride);
+          pluginRegistry.createOrUpdate(pluginName, newPluginConfig, true);
+        }
+      } catch (ExecutionSetupException e) {
+        throw new RuntimeException("Failed to update Hive storage plugin for drillbits", e);
+      }
+    }
+
+  }
+
+
+  /**
+   * Implements method for initialization and passing
+   * of Hive to consumer instances in order to be used
+   * for test data generation within session.
+   * The class was made inner because it uses driverConf
+   * of enclosing fixture instance.
+   */
+  public class HiveDriverManager {
+
+    /**
+     *  {@link HiveTestFixture}'s constructor will create instance,
+     *  and API users will get it via {@link HiveTestFixture#getDriverManager()}.
+     */
+    private HiveDriverManager() {
+    }
+
+    public void runWithinSession(Consumer<Driver> driverConsumer) {
+      final HiveConf hiveConf = new HiveConf(SessionState.class);
+      driverConf.forEach(hiveConf::set);
+      SessionState ss = new SessionState(hiveConf);
+      try (Closeable ssClose = ss::close) {
+        SessionState.start(ss);
+        driverConsumer.accept(new Driver(hiveConf));
+      } catch (IOException e) {
+        throw new RuntimeException("Exception was thrown while closing SessionState", e);
+      }
+    }
+
+  }
+
+}
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestUtilities.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestUtilities.java
index 0556c93..7457511 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestUtilities.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestUtilities.java
@@ -17,6 +17,14 @@
  */
 package org.apache.drill.exec.hive;
 
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.attribute.PosixFilePermission;
+import java.util.EnumSet;
+import java.util.Set;
+
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -24,6 +32,12 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 public class HiveTestUtilities {
 
   /**
+   * Set of all posix permissions to be assigned to newly created file in
+   * {@link HiveTestUtilities#createDirWithPosixPermissions(File, String)}
+   */
+  private static final Set<PosixFilePermission> ALL_POSIX_PERMISSIONS = EnumSet.allOf(PosixFilePermission.class);
+
+  /**
    * Execute the give <i>query</i> on given <i>hiveDriver</i> instance. If a {@link CommandNeedRetryException}
    * exception is thrown, it tries upto 3 times before returning failure.
    * @param hiveDriver
@@ -47,4 +61,26 @@ public class HiveTestUtilities {
           query, (response != null ? response.getErrorMessage() : "")));
     }
   }
+
+  /**
+   * Creates desired directory structure and
+   * adds all posix permissions to created directory.
+   *
+   * @param parentDir parent directory
+   * @param dirName directory name
+   * @return file representing created dir with all posix permissions
+   */
+  public static File createDirWithPosixPermissions(File parentDir, String dirName) {
+    File dir = new File(parentDir, dirName);
+    dir.mkdirs();
+    Path path = dir.toPath();
+    try {
+      Files.setPosixFilePermissions(path, ALL_POSIX_PERMISSIONS);
+    } catch (IOException e) {
+      throw new RuntimeException(
+          String.format("Failed to set all posix permissions for directory [%s]", dir), e);
+    }
+    return dir;
+  }
+
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
index 0685906..7355de7 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
@@ -17,12 +17,9 @@
  */
 package org.apache.drill.exec.hive;
 
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
 import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -43,8 +40,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
 
 @Category({SlowTest.class, HiveStorageTest.class})
 public class TestHiveStorage extends HiveTestBase {
@@ -211,8 +211,10 @@ public class TestHiveStorage extends HiveTestBase {
   @Test
   public void queryingTablesInNonDefaultFS() throws Exception {
     // Update the default FS settings in Hive test storage plugin to non-local FS
-    hiveTest.updatePluginConfig(getDrillbitContext().getStorage(),
-        ImmutableMap.of(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost:9001"));
+
+    HIVE_TEST_FIXTURE.getPluginManager().updateHivePlugin(
+        Collections.singleton(bits[0]),
+        Collections.singletonMap(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost:9001"));
 
     testBuilder()
         .sqlQuery("SELECT * FROM hive.`default`.kv LIMIT 1")
@@ -224,8 +226,8 @@ public class TestHiveStorage extends HiveTestBase {
 
   @Test // DRILL-745
   public void queryingHiveAvroTable() throws Exception {
-      testBuilder()
-          .sqlQuery("SELECT * FROM hive.db1.avro ORDER BY key DESC LIMIT 1")
+    testBuilder()
+        .sqlQuery("SELECT * FROM hive.db1.avro ORDER BY key DESC LIMIT 1")
         .unOrdered()
         .baselineColumns("key", "value")
         .baselineValues(5, " key_5")
@@ -274,7 +276,6 @@ public class TestHiveStorage extends HiveTestBase {
   }
 
 
-
   @Test // DRILL-3739
   public void readingFromStorageHandleBasedTable() throws Exception {
     testBuilder()
@@ -287,7 +288,7 @@ public class TestHiveStorage extends HiveTestBase {
 
   @Test // DRILL-3688
   public void readingFromSmallTableWithSkipHeaderAndFooter() throws Exception {
-   testBuilder()
+    testBuilder()
         .sqlQuery("select key, `value` from hive.skipper.kv_text_small order by key asc")
         .ordered()
         .baselineColumns("key", "value")
@@ -312,7 +313,7 @@ public class TestHiveStorage extends HiveTestBase {
         .sqlQuery("select sum(key) as sum_keys from hive.skipper.kv_text_large")
         .unOrdered()
         .baselineColumns("sum_keys")
-        .baselineValues((long)(5000*(5000 + 1)/2))
+        .baselineValues((long) (5000 * (5000 + 1) / 2))
         .go();
 
     testBuilder()
@@ -383,7 +384,7 @@ public class TestHiveStorage extends HiveTestBase {
   public void testStringColumnsMetadata() throws Exception {
     String query = "select varchar_field, char_field, string_field from hive.readtest";
 
-    Map<String, Integer> expectedResult = Maps.newHashMap();
+    Map<String, Integer> expectedResult = new HashMap<>();
     expectedResult.put("varchar_field", 50);
     expectedResult.put("char_field", 10);
     expectedResult.put("string_field", HiveVarchar.MAX_VARCHAR_LENGTH);
@@ -394,7 +395,7 @@ public class TestHiveStorage extends HiveTestBase {
     try {
       test("alter session set `%s` = true", ExecConstants.EARLY_LIMIT0_OPT_KEY);
       verifyColumnsMetadata(client.createPreparedStatement(String.format("select * from (%s) t limit 0", query)).get()
-              .getPreparedStatement().getColumnsList(), expectedResult);
+          .getPreparedStatement().getColumnsList(), expectedResult);
     } finally {
       test("alter session reset `%s`", ExecConstants.EARLY_LIMIT0_OPT_KEY);
     }
@@ -450,4 +451,5 @@ public class TestHiveStorage extends HiveTestBase {
       assertTrue("Column should be nullable", columnMetadata.getIsNullable());
     }
   }
+
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
index 53088ed..39f8655 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
@@ -17,10 +17,15 @@
  */
 package org.apache.drill.exec.impersonation.hive;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.calcite.schema.Schema.TableType;
-import org.apache.drill.test.TestBuilder;
 import org.apache.drill.exec.impersonation.BaseTestImpersonation;
 import org.apache.drill.exec.store.hive.HiveStoragePluginConfig;
+import org.apache.drill.test.TestBuilder;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -30,13 +35,8 @@ import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.shims.ShimLoader;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
+import static org.apache.drill.exec.hive.HiveTestUtilities.createDirWithPosixPermissions;
 import static org.apache.drill.exec.hive.HiveTestUtilities.executeQuery;
-import static org.apache.drill.exec.store.hive.HiveTestDataGenerator.createFileWithPermissions;
 import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
 
@@ -65,8 +65,8 @@ public class BaseTestHiveImpersonation extends BaseTestImpersonation {
   protected static void prepHiveConfAndData() throws Exception {
     hiveConf = new HiveConf();
 
-    File scratchDir = createFileWithPermissions(dirTestWatcher.getRootDir(), "scratch_dir");
-    File localScratchDir = createFileWithPermissions(dirTestWatcher.getRootDir(), "local_scratch_dir");
+    File scratchDir = createDirWithPosixPermissions(dirTestWatcher.getRootDir(), "scratch_dir");
+    File localScratchDir = createDirWithPosixPermissions(dirTestWatcher.getRootDir(), "local_scratch_dir");
     File metaStoreDBDir = new File(dirTestWatcher.getRootDir(), "metastore_db");
 
     // Configure metastore persistence db location on local filesystem
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
index 821200d..5b53113 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
@@ -17,26 +17,34 @@
  */
 package org.apache.drill.exec.sql.hive;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import java.util.Objects;
+
 import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.exec.sql.TestBaseViewSupport;
-import org.apache.drill.exec.store.hive.HiveTestDataGenerator;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.apache.drill.exec.hive.HiveTestBase.HIVE_TEST_FIXTURE;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.DFS_TMP_SCHEMA;
 
 @Category({SlowTest.class, HiveStorageTest.class})
 public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
-  protected static HiveTestDataGenerator hiveTest;
 
   @BeforeClass
-  public static void generateHive() throws Exception{
-    hiveTest = HiveTestDataGenerator.getInstance(dirTestWatcher);
-    hiveTest.addHiveTestPlugin(getDrillbitContext().getStorage());
+  public static void setUp() throws Exception {
+    Objects.requireNonNull(HIVE_TEST_FIXTURE, "Failed to configure Hive storage plugin, " +
+        "because HiveTestBase.HIVE_TEST_FIXTURE isn't initialized!")
+        .getPluginManager().addHivePluginTo(bits);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (HIVE_TEST_FIXTURE != null) {
+      HIVE_TEST_FIXTURE.getPluginManager().removeHivePluginFrom(bits);
+    }
   }
 
   @Test
@@ -46,8 +54,8 @@ public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
         null,
         "SELECT * FROM hive.kv",
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
-        new String[] { "key", "value"},
-        ImmutableList.of(new Object[] { 1, " key_1" })
+        baselineColumns("key", "value"),
+        baselineRows(row( 1, " key_1" ))
     );
   }
 
@@ -58,8 +66,8 @@ public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
         null,
         "SELECT * FROM hive.kv",
         "SELECT key, `value` FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
-        new String[] { "key", "value" },
-        ImmutableList.of(new Object[] { 1, " key_1" })
+        baselineColumns("key", "value"),
+        baselineRows(row(1, " key_1"))
     );
   }
 
@@ -70,8 +78,8 @@ public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
         null,
         "SELECT * FROM hive.kv",
         "SELECT `value` FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
-        new String[] { "value" },
-        ImmutableList.of(new Object[] { " key_1" })
+        baselineColumns("value"),
+        baselineRows(row(" key_1"))
     );
   }
 
@@ -82,8 +90,8 @@ public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
         null,
         "SELECT key, `value` FROM hive.kv",
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
-        new String[] { "key", "value" },
-        ImmutableList.of(new Object[] { 1, " key_1" })
+        baselineColumns("key", "value"),
+        baselineRows(row(1, " key_1"))
     );
   }
 
@@ -94,8 +102,8 @@ public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
         null,
         "SELECT key, `value` FROM hive.kv",
         "SELECT key, `value` FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
-        new String[] { "key", "value" },
-        ImmutableList.of(new Object[] { 1, " key_1" })
+        baselineColumns("key", "value"),
+        baselineRows(row(1, " key_1"))
     );
   }
 
@@ -110,11 +118,4 @@ public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
         .go();
   }
 
-  @AfterClass
-  public static void cleanupHiveTestData() throws Exception{
-    if (hiveTest != null) {
-      hiveTest.deleteHiveTestPlugin(getDrillbitContext().getStorage());
-    }
-  }
-
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
index 0b9cd36..a60f0a6 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
@@ -18,152 +18,46 @@
 package org.apache.drill.exec.store.hive;
 
 import java.io.File;
-import java.io.IOException;
 import java.io.PrintWriter;
-import java.nio.file.Files;
 import java.nio.file.Paths;
-import java.nio.file.attribute.PosixFilePermission;
 import java.sql.Date;
 import java.sql.Timestamp;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
-import org.apache.drill.shaded.guava.com.google.common.io.Resources;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.drill.shaded.guava.com.google.common.io.Resources;
 import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.BaseTestQuery;
-import org.apache.drill.common.exceptions.DrillException;
-import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.Driver;
-import org.apache.hadoop.hive.ql.session.SessionState;
-
 import org.apache.hadoop.hive.serde.serdeConstants;
 
+import static org.apache.drill.exec.hive.HiveTestUtilities.createDirWithPosixPermissions;
 import static org.apache.drill.exec.hive.HiveTestUtilities.executeQuery;
 
 public class HiveTestDataGenerator {
-  private static final String HIVE_TEST_PLUGIN_NAME = "hive";
-  private static HiveTestDataGenerator instance;
-  private static File baseDir;
 
-  private final String dbDir;
-  private final String whDir;
   private final BaseDirTestWatcher dirTestWatcher;
-  private final Map<String, String> config;
-
-  public static synchronized HiveTestDataGenerator getInstance(BaseDirTestWatcher dirTestWatcher) throws Exception {
-    File baseDir = dirTestWatcher.getRootDir();
-    if (instance == null || !HiveTestDataGenerator.baseDir.equals(baseDir)) {
-      HiveTestDataGenerator.baseDir = baseDir;
-
-      File dbDirFile = new File(baseDir, "metastore_db");
-      File whDirFile = new File(baseDir, "warehouse");
-
-      final String dbDir = dbDirFile.getAbsolutePath();
-      final String whDir = whDirFile.getAbsolutePath();
 
-      instance = new HiveTestDataGenerator(dbDir, whDir, dirTestWatcher);
-      instance.generateTestData();
-    }
+  private final File baseDir;
 
-    return instance;
-  }
+  private final String warehouseDir;
 
-  private HiveTestDataGenerator(final String dbDir, final String whDir, final BaseDirTestWatcher dirTestWatcher) {
-    this.dbDir = dbDir;
-    this.whDir = whDir;
+  public HiveTestDataGenerator(BaseDirTestWatcher dirTestWatcher, File baseDir, String warehouseDir) {
     this.dirTestWatcher = dirTestWatcher;
-
-    config = new HashMap<>();
-    config.put(ConfVars.METASTOREURIS.toString(), "");
-    config.put("javax.jdo.option.ConnectionURL", String.format("jdbc:derby:;databaseName=%s;create=true", dbDir));
-    config.put("hive.metastore.warehouse.dir", whDir);
-    config.put(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-  }
-
-  /**
-   * Add Hive test storage plugin to the given plugin registry.
-   *
-   * @param pluginRegistry storage plugin registry
-   * @throws Exception in case if unable to update Hive storage plugin
-   */
-  public void addHiveTestPlugin(final StoragePluginRegistry pluginRegistry) throws Exception {
-    HiveStoragePluginConfig pluginConfig = new HiveStoragePluginConfig(config);
-    pluginConfig.setEnabled(true);
-
-    pluginRegistry.createOrUpdate(HIVE_TEST_PLUGIN_NAME, pluginConfig, true);
-  }
-
-  /**
-   * Update the current HiveStoragePlugin in given plugin registry with given <i>configOverride</i>.
-   *
-   * @param pluginRegistry storage plugin registry
-   * @param configOverride config properties to be overridden
-   * @throws DrillException if fails to update or no Hive plugin currently exists in given plugin registry.
-   */
-  public void updatePluginConfig(final StoragePluginRegistry pluginRegistry, Map<String, String> configOverride)
-      throws DrillException {
-    HiveStoragePlugin storagePlugin = (HiveStoragePlugin) pluginRegistry.getPlugin(HIVE_TEST_PLUGIN_NAME);
-    if (storagePlugin == null) {
-      throw new DrillException(
-          "Hive test storage plugin doesn't exist. Add a plugin using addHiveTestPlugin()");
-    }
-
-    HiveStoragePluginConfig newPluginConfig = storagePlugin.getConfig();
-    newPluginConfig.getConfigProps().putAll(configOverride);
-
-    pluginRegistry.createOrUpdate(HIVE_TEST_PLUGIN_NAME, newPluginConfig, true);
-  }
-
-  /**
-   * Delete the Hive test plugin from registry.
-   */
-  public void deleteHiveTestPlugin(final StoragePluginRegistry pluginRegistry) {
-    pluginRegistry.deletePlugin(HIVE_TEST_PLUGIN_NAME);
+    this.baseDir = baseDir;
+    this.warehouseDir = warehouseDir;
   }
 
-  public static File createFileWithPermissions(File baseDir, String name) {
-    Set<PosixFilePermission> perms = Sets.newHashSet(PosixFilePermission.values());
-    File dir = new File(baseDir, name);
-    dir.mkdirs();
-
+  public void generateData(Driver hiveDriver) {
     try {
-      Files.setPosixFilePermissions(dir.toPath(), perms);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
+      generateDataInternal(hiveDriver);
+    } catch (Exception e) {
+      throw new RuntimeException("Exception was thrown while generating test data", e);
     }
-
-    return dir;
   }
 
-  private void generateTestData() throws Exception {
-    HiveConf conf = new HiveConf(SessionState.class);
-
-    File scratchDir = createFileWithPermissions(baseDir, "scratch_dir");
-    File localScratchDir = createFileWithPermissions(baseDir, "local_scratch_dir");
-    File part1Dir = createFileWithPermissions(baseDir, "part1");
-
-    conf.set("javax.jdo.option.ConnectionURL", String.format("jdbc:derby:;databaseName=%s;create=true", dbDir));
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-    conf.set("hive.metastore.warehouse.dir", whDir);
-    conf.set("mapred.job.tracker", "local");
-    conf.set(ConfVars.SCRATCHDIR.varname,  scratchDir.getAbsolutePath());
-    conf.set(ConfVars.LOCALSCRATCHDIR.varname, localScratchDir.getAbsolutePath());
-    conf.set(ConfVars.DYNAMICPARTITIONINGMODE.varname, "nonstrict");
-    conf.set(ConfVars.METASTORE_AUTO_CREATE_ALL.varname, "true");
-    conf.set(ConfVars.METASTORE_SCHEMA_VERIFICATION.varname, "false");
-    conf.set(ConfVars.HIVE_CBO_ENABLED.varname, "false");
-
-    SessionState ss = new SessionState(conf);
-    SessionState.start(ss);
-    Driver hiveDriver = new Driver(conf);
-
+  private void generateDataInternal(Driver hiveDriver) throws Exception {
+    File part1Dir = createDirWithPosixPermissions(baseDir, "part1");
     // generate (key, value) test data
     String testDataFile = generateTestDataFile();
 
@@ -199,7 +93,7 @@ public class HiveTestDataGenerator {
     // create a table with no data
     executeQuery(hiveDriver, "CREATE TABLE IF NOT EXISTS empty_table(a INT, b STRING)");
     // delete the table location of empty table
-    File emptyTableLocation = new File(whDir, "empty_table");
+    File emptyTableLocation = new File(warehouseDir, "empty_table");
     if (emptyTableLocation.exists()) {
       FileUtils.forceDelete(emptyTableLocation);
     }
@@ -504,7 +398,7 @@ public class HiveTestDataGenerator {
         "'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'");
     // Insert fails if the table directory already exists for tables with DefaultStorageHandlers. Its a known
     // issue in Hive. So delete the table directory created as part of the CREATE TABLE
-    FileUtils.deleteQuietly(new File(whDir, "kv_sh"));
+    FileUtils.deleteQuietly(new File(warehouseDir, "kv_sh"));
     //executeQuery(hiveDriver, "INSERT OVERWRITE TABLE kv_sh SELECT * FROM kv");
 
     // Create text tables with skip header and footer table property
@@ -552,7 +446,6 @@ public class HiveTestDataGenerator {
     executeQuery(hiveDriver, "CREATE OR REPLACE VIEW view_over_hive_view AS SELECT * FROM hive_view WHERE key BETWEEN 2 AND 3");
     executeQuery(hiveDriver, "CREATE OR REPLACE VIEW db1.two_table_view AS SELECT COUNT(dk.key) dk_key_count FROM db1.avro dk " +
         "INNER JOIN kv ON kv.key = dk.key");
-    ss.close();
   }
 
   private void createTestDataForDrillNativeParquetReaderTests(Driver hiveDriver) {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
index e9f7aab..8569d2f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
@@ -21,6 +21,8 @@ import org.apache.drill.PlanTestBase;
 import org.apache.drill.shaded.guava.com.google.common.base.Strings;
 import org.apache.drill.test.TestBuilder;
 
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -209,4 +211,40 @@ public class TestBaseViewSupport extends PlanTestBase {
       dropViewHelper(finalSchema, viewName, finalSchema);
     }
   }
+
+  /**
+   * Convenient method for defining baselineColumns to be passed into
+   * view helper methods.
+   *
+   * @param names column names varargs
+   * @return column names array
+   */
+  protected static String[] baselineColumns(String... names) {
+    return names;
+  }
+
+  /**
+   * Convenient method for grouping of expected rows into
+   * list of Object arrays, where each array represents concrete
+   * row. This method is used for defining baselineValues and passing it to
+   * view helper methods.
+   *
+   * @param rows rows in form of Object[] varargs
+   * @return list of rows
+   */
+  protected static List<Object[]> baselineRows(Object[]... rows) {
+    return Collections.unmodifiableList(Arrays.asList(rows));
+  }
+
+  /**
+   * Helper method for conversion of Object varargs into
+   * array of objects. Used for passing rows into
+   * {@link TestBaseViewSupport#baselineRows(Object[]...)}
+   *
+   * @param columns Object varargs
+   * @return array of passed objects
+   */
+  protected static Object[] row(Object... columns) {
+    return columns;
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
index a2eefcc..b8d5495 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
@@ -24,7 +24,6 @@ import java.util.List;
 import org.apache.commons.io.FileUtils;
 import org.apache.drill.categories.SqlTest;
 import org.apache.drill.categories.UnlikelyTest;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -104,9 +103,10 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         "SELECT * FROM cp.`region.json` ORDER BY `region_id`",
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
-        new String[] { "region_id", "sales_city", "sales_state_province", "sales_district", "sales_region",
-            "sales_country", "sales_district_id" },
-        ImmutableList.of(new Object[] { 0L, "None", "None", "No District", "No Region", "No Country", 0L })
+        baselineColumns("region_id", "sales_city", "sales_state_province", "sales_district",
+                        "sales_region", "sales_country", "sales_district_id"),
+        baselineRows(row(0L, "None", "None", "No District",
+                        "No Region", "No Country", 0L))
     );
   }
 
@@ -117,11 +117,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`",
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[] { "region_id", "sales_city" },
-        ImmutableList.of(
-            new Object[] { 0L, "None" },
-            new Object[] { 1L, "San Francisco" }
-        )
+        baselineColumns("region_id", "sales_city"),
+        baselineRows(row(0L, "None"), row(1L, "San Francisco"))
     );
   }
 
@@ -132,11 +129,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         "(regionid, salescity)",
         "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`",
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[] { "regionid", "salescity" },
-        ImmutableList.of(
-            new Object[] { 0L, "None" },
-            new Object[] { 1L, "San Francisco" }
-        )
+        baselineColumns("regionid", "salescity"),
+        baselineRows(row(0L, "None"), row(1L, "San Francisco"))
     );
   }
 
@@ -147,11 +141,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         "SELECT * FROM cp.`region.json` ORDER BY `region_id`",
         "SELECT region_id, sales_city FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[] { "region_id", "sales_city" },
-        ImmutableList.of(
-            new Object[] { 0L, "None" },
-            new Object[] { 1L, "San Francisco" }
-        )
+        baselineColumns("region_id", "sales_city"),
+        baselineRows(row(0L, "None"), row(1L, "San Francisco"))
     );
   }
 
@@ -162,11 +153,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`",
         "SELECT region_id, sales_city FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[] { "region_id", "sales_city" },
-        ImmutableList.of(
-            new Object[] { 0L, "None" },
-            new Object[] { 1L, "San Francisco" }
-        )
+        baselineColumns("region_id", "sales_city"),
+        baselineRows(row(0L, "None"), row(1L, "San Francisco"))
     );
   }
 
@@ -177,11 +165,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`",
         "SELECT sales_city FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[] { "sales_city" },
-        ImmutableList.of(
-            new Object[] { "None" },
-            new Object[] { "San Francisco" }
-        )
+        baselineColumns("sales_city"),
+        baselineRows(row("None"), row("San Francisco"))
     );
   }
 
@@ -192,11 +177,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         "(regionid, salescity)",
         "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id` LIMIT 2",
         "SELECT regionid, salescity FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[] { "regionid", "salescity" },
-        ImmutableList.of(
-            new Object[] { 0L, "None" },
-            new Object[] { 1L, "San Francisco" }
-        )
+        baselineColumns("regionid", "salescity"),
+        baselineRows(row(0L, "None"), row(1L, "San Francisco"))
     );
   }
 
@@ -207,11 +189,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         "(regionid, salescity)",
         "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id` DESC",
         "SELECT regionid FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[]{"regionid"},
-        ImmutableList.of(
-            new Object[]{109L},
-            new Object[]{108L}
-        )
+        baselineColumns("regionid"),
+        baselineRows(row(109L), row(108L))
     );
   }
 
@@ -223,11 +202,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         "SELECT region_id FROM cp.`region.json` UNION SELECT employee_id FROM cp.`employee.json`",
         "SELECT regionid FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[]{"regionid"},
-        ImmutableList.of(
-            new Object[]{110L},
-            new Object[]{108L}
-        )
+        baselineColumns("regionid"),
+        baselineRows(row(110L), row(108L))
     );
   }
 
@@ -245,8 +221,8 @@ public class TestViewSupport extends TestBaseViewSupport {
 
       queryViewHelper(
           String.format("SELECT region_id FROM %s.`%s` LIMIT 1", DFS_TMP_SCHEMA, outerView),
-          new String[] { "region_id" },
-          ImmutableList.of(new Object[] { 0L })
+          baselineColumns("region_id"),
+          baselineRows(row(0L))
       );
     } finally {
       dropViewHelper(DFS_TMP_SCHEMA, outerView, DFS_TMP_SCHEMA);
@@ -269,10 +245,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         viewDef,
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
-        new String[]{"n_nationkey", "n_name", "n_regionkey", "n_comment"},
-        ImmutableList.of(
-            new Object[]{0, "ALGERIA", 0, " haggle. carefully final deposits detect slyly agai"}
-        )
+        baselineColumns("n_nationkey", "n_name", "n_regionkey", "n_comment"),
+        baselineRows(row(0, "ALGERIA", 0, " haggle. carefully final deposits detect slyly agai"))
     );
   }
 
@@ -304,8 +278,8 @@ public class TestViewSupport extends TestBaseViewSupport {
 
       // Make sure the new view created returns the data expected.
       queryViewHelper(String.format("SELECT * FROM %s.`%s` LIMIT 1", DFS_TMP_SCHEMA, viewName),
-          new String[]{"sales_state_province"},
-          ImmutableList.of(new Object[]{"None"})
+          baselineColumns("sales_state_province"),
+          baselineRows(row("None"))
       );
     } finally {
       dropViewHelper(DFS_TMP_SCHEMA, viewName, DFS_TMP_SCHEMA);
@@ -361,8 +335,8 @@ public class TestViewSupport extends TestBaseViewSupport {
 
       // Make sure the view created returns the data expected.
       queryViewHelper(String.format("SELECT * FROM %s.`%s` LIMIT 1", DFS_TMP_SCHEMA, viewName),
-        new String[]{"region_id", "sales_city"},
-        ImmutableList.of(new Object[]{0L, "None"})
+          baselineColumns("region_id", "sales_city"),
+          baselineRows(row(0L, "None"))
       );
     } finally {
       dropViewHelper(DFS_TMP_SCHEMA, viewName, DFS_TMP_SCHEMA);
@@ -382,8 +356,8 @@ public class TestViewSupport extends TestBaseViewSupport {
 
       // Make sure the view created returns the data expected.
       queryViewHelper(String.format("SELECT * FROM %s.`%s` LIMIT 1", DFS_TMP_SCHEMA, viewName),
-        new String[]{"region_id", "sales_city"},
-        ImmutableList.of(new Object[]{0L, "None"})
+          baselineColumns("region_id", "sales_city"),
+          baselineRows(row(0L, "None"))
       );
     } finally {
       dropViewHelper(DFS_TMP_SCHEMA, viewName, DFS_TMP_SCHEMA);
@@ -484,8 +458,8 @@ public class TestViewSupport extends TestBaseViewSupport {
       createViewHelper("tmp", viewName, DFS_TMP_SCHEMA, null,
           "SELECT CAST(`employee_id` AS INTEGER) AS `employeeid`\n" + "FROM `cp`.`employee.json`");
 
-      final String[] baselineColumns = new String[] { "employeeid" };
-      final List<Object[]> baselineValues = ImmutableList.of(new Object[] { 1156 });
+      final String[] baselineColumns = baselineColumns("employeeid");
+      final List<Object[]> baselineValues = baselineRows(row(1156));
 
       // Query view from current schema "dfs" by referring to the view using "tmp.viewName"
       queryViewHelper(
@@ -521,8 +495,8 @@ public class TestViewSupport extends TestBaseViewSupport {
       // Create a view with full schema identifier and refer the "region.json" as without schema.
       createViewHelper(DFS_TMP_SCHEMA, viewName, DFS_TMP_SCHEMA, null, "SELECT region_id, sales_city FROM `region.json`");
 
-      final String[] baselineColumns = new String[] { "region_id", "sales_city" };
-      final List<Object[]> baselineValues = ImmutableList.of(new Object[]{109L, "Santa Fe"});
+      final String[] baselineColumns = baselineColumns("region_id", "sales_city");
+      final List<Object[]> baselineValues = baselineRows(row(109L, "Santa Fe"));
 
       // Query the view
       queryViewHelper(
@@ -635,10 +609,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         "(regionid1, regionid2)",
         "SELECT region_id, region_id FROM cp.`region.json` LIMIT 1",
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME",
-        new String[]{"regionid1", "regionid2"},
-        ImmutableList.of(
-            new Object[]{0L, 0L}
-        )
+        baselineColumns("regionid1", "regionid2"),
+        baselineRows(row(0L, 0L))
     );
   }
 
@@ -651,10 +623,8 @@ public class TestViewSupport extends TestBaseViewSupport {
         "SELECT t1.region_id, t2.region_id FROM cp.`region.json` t1 JOIN cp.`region.json` t2 " +
             "ON t1.region_id = t2.region_id LIMIT 1",
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME",
-        new String[]{"regionid1", "regionid2"},
-        ImmutableList.of(
-            new Object[]{0L, 0L}
-        )
+        baselineColumns("regionid1", "regionid2"),
+        baselineRows(row(0L, 0L))
     );
   }