You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2018/03/06 14:27:16 UTC

hive git commit: HIVE-18051: qfiles: dataset support (Laszlo Bodor via Zoltan Haindrich)

Repository: hive
Updated Branches:
  refs/heads/master 8f93ca0b5 -> 7cb31c030


HIVE-18051: qfiles: dataset support (Laszlo Bodor via Zoltan Haindrich)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7cb31c03
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7cb31c03
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7cb31c03

Branch: refs/heads/master
Commit: 7cb31c03052b815665b3231f2e513b9e65d3ff8c
Parents: 8f93ca0
Author: Laszlo Bodor <bo...@gmail.com>
Authored: Tue Mar 6 15:03:38 2018 +0100
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Tue Mar 6 15:03:38 2018 +0100

----------------------------------------------------------------------
 data/files/datasets/testdataset/load.hive.sql   |  10 +
 data/scripts/q_test_init_src.sql                |   1 -
 .../hive/cli/control/AbstractCliConfig.java     |   3 +-
 .../control/AbstractCoreBlobstoreCliDriver.java |   4 +-
 .../hadoop/hive/cli/control/CliAdapter.java     |  10 +-
 .../hive/cli/control/CoreAccumuloCliDriver.java |   4 +-
 .../hadoop/hive/cli/control/CoreCliDriver.java  |  24 +-
 .../hive/cli/control/CoreCompareCliDriver.java  |   2 +-
 .../hive/cli/control/CoreHBaseCliDriver.java    |   4 +-
 .../cli/control/CoreHBaseNegativeCliDriver.java |   4 +-
 .../hive/cli/control/CoreNegativeCliDriver.java |   4 +-
 .../hive/cli/control/CorePerfCliDriver.java     |   6 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 217 +++++++++++++------
 .../apache/hadoop/hive/ql/dataset/Dataset.java  |  36 +++
 .../hive/ql/dataset/DatasetCollection.java      |  46 ++++
 .../hadoop/hive/ql/dataset/DatasetParser.java   |  73 +++++++
 .../hadoop/hive/ql/parse/CoreParseNegative.java |   3 +-
 .../hive/ql/dataset/TestDatasetParser.java      |  58 +++++
 .../hive/ql/hooks/EnforceReadOnlyTables.java    |  21 +-
 .../test/queries/clientpositive/testdataset.q   |   2 +
 .../test/queries/clientpositive/testdataset_2.q |   2 +
 .../results/clientpositive/testdataset.q.out    |   9 +
 .../results/clientpositive/testdataset_2.q.out  |   9 +
 23 files changed, 441 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/data/files/datasets/testdataset/load.hive.sql
----------------------------------------------------------------------
diff --git a/data/files/datasets/testdataset/load.hive.sql b/data/files/datasets/testdataset/load.hive.sql
new file mode 100644
index 0000000..3436cda
--- /dev/null
+++ b/data/files/datasets/testdataset/load.hive.sql
@@ -0,0 +1,10 @@
+--
+-- Table testdataset
+--
+CREATE TABLE testdataset (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" INTO TABLE testdataset;
+
+ANALYZE TABLE testdataset COMPUTE STATISTICS;
+
+ANALYZE TABLE testdataset COMPUTE STATISTICS FOR COLUMNS key,value;

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/data/scripts/q_test_init_src.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_init_src.sql b/data/scripts/q_test_init_src.sql
index 2a62d29..765d6c7 100644
--- a/data/scripts/q_test_init_src.sql
+++ b/data/scripts/q_test_init_src.sql
@@ -7,4 +7,3 @@ LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" OVERWRITE INTO TABLE
 ANALYZE TABLE src COMPUTE STATISTICS;
 
 ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key,value;
-

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
index 0b12928..01b9ed6 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
@@ -71,7 +71,7 @@ public abstract class AbstractCliConfig {
   private Class<? extends CliAdapter> cliAdapter;
 
   public AbstractCliConfig(Class<? extends CliAdapter> adapter) {
-    cliAdapter=adapter;
+    cliAdapter = adapter;
     clusterType = MiniClusterType.none;
     queryFile = getSysPropValue("qfile");
     queryFileRegex = getSysPropValue("qfile_regex");
@@ -135,7 +135,6 @@ public abstract class AbstractCliConfig {
     excludedQueryFileNames.add(qFile);
   }
 
-
   private static final Splitter TEST_SPLITTER =
       Splitter.onPattern("[, ]").trimResults().omitEmptyStrings();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
index eda6b69..b1d66a5 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
@@ -21,6 +21,8 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import com.google.common.base.Strings;
+
+import java.io.File;
 import java.text.SimpleDateFormat;
 import java.util.Calendar;
 import java.util.Map;
@@ -135,7 +137,7 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
         System.err.println("Test " + fname + " skipped");
         return;
       }
-      qt.cliInit(fname, false);
+      qt.cliInit(new File(fpath), false);
       int ecode = qt.executeClient(fname);
       if ((ecode == 0) ^ expectSuccess) {
         qt.failed(ecode, fname, debugHint);

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java
index 1fd295f..574a67f 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java
@@ -48,18 +48,18 @@ public abstract class CliAdapter {
     return ret;
   }
 
-  abstract public void beforeClass() throws Exception;
+  public abstract void beforeClass() throws Exception;
 
   // HIVE-14444 pending rename: before
-  abstract public void setUp();
+  public abstract void setUp();
 
   // HIVE-14444 pending rename: after
-  abstract public void tearDown();
+  public abstract void tearDown();
 
   // HIVE-14444 pending rename: afterClass
-  abstract public void shutdown() throws Exception;
+  public abstract void shutdown() throws Exception;
 
-  abstract public void runTest(String name, String name2, String absolutePath) throws Exception;
+  public abstract void runTest(String name, String fileName, String absolutePath) throws Exception;
 
   public final TestRule buildClassRule() {
     return new TestRule() {

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java
index bd133f9..d21fcaf 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreAccumuloCliDriver.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hive.cli.control;
 
 import static org.junit.Assert.assertTrue;
 
+import java.io.File;
+
 import org.apache.hadoop.hive.accumulo.AccumuloQTestUtil;
 import org.apache.hadoop.hive.accumulo.AccumuloTestSetup;
 import org.apache.hadoop.hive.ql.QTestProcessExecResult;
@@ -87,7 +89,7 @@ public class CoreAccumuloCliDriver extends CliAdapter {
         return;
       }
 
-      qt.cliInit(fname);
+      qt.cliInit(new File(fpath));
       qt.clearTestSideEffects();
       int ecode = qt.executeClient(fname);
       if (ecode != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
index 6c37391..a7ec4f3 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
@@ -20,11 +20,9 @@ package org.apache.hadoop.hive.cli.control;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.File;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.base.Stopwatch;
-import com.google.common.base.Strings;
-import org.apache.hadoop.hive.cli.control.AbstractCliConfig.MetastoreType;
 import org.apache.hadoop.hive.ql.QTestProcessExecResult;
 import org.apache.hadoop.hive.ql.QTestUtil;
 import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
@@ -36,6 +34,9 @@ import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Stopwatch;
+import com.google.common.base.Strings;
+
 public class CoreCliDriver extends CliAdapter {
 
   private static final Logger LOG = LoggerFactory.getLogger(CoreCliDriver.class);
@@ -55,6 +56,7 @@ public class CoreCliDriver extends CliAdapter {
     final String hiveConfDir = cliConfig.getHiveConfDir();
     final String initScript = cliConfig.getInitScript();
     final String cleanupScript = cliConfig.getCleanupScript();
+
     try {
       final String hadoopVer = cliConfig.getHadoopVersion();
 
@@ -148,11 +150,13 @@ public class CoreCliDriver extends CliAdapter {
     }
   }
 
-  static String debugHint = "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
-     + "or check ./ql/target/surefire-reports or ./itests/qtest/target/surefire-reports/ for specific test cases logs.";
+  private static String debugHint =
+      "\nSee ./ql/target/tmp/log/hive.log or ./itests/qtest/target/tmp/log/hive.log, "
+          + "or check ./ql/target/surefire-reports "
+          + "or ./itests/qtest/target/surefire-reports/ for specific test cases logs.";
 
   @Override
-  public void runTest(String tname, String fname, String fpath) throws Exception {
+  public void runTest(String testName, String fname, String fpath) throws Exception {
     Stopwatch sw = Stopwatch.createStarted();
     boolean skipped = false;
     boolean failed = false;
@@ -169,7 +173,7 @@ public class CoreCliDriver extends CliAdapter {
         return;
       }
 
-      qt.cliInit(fname, false);
+      qt.cliInit(new File(fpath), false);
       int ecode = qt.executeClient(fname);
       if (ecode != 0) {
         failed = true;
@@ -178,8 +182,8 @@ public class CoreCliDriver extends CliAdapter {
       QTestProcessExecResult result = qt.checkCliDriverResults(fname);
       if (result.getReturnCode() != 0) {
         failed = true;
-        String message = Strings.isNullOrEmpty(result.getCapturedOutput()) ?
-            debugHint : "\r\n" + result.getCapturedOutput();
+        String message = Strings.isNullOrEmpty(result.getCapturedOutput()) ? debugHint
+            : "\r\n" + result.getCapturedOutput();
         qt.failedDiff(result.getReturnCode(), fname, message);
       }
     }
@@ -187,7 +191,7 @@ public class CoreCliDriver extends CliAdapter {
       failed = true;
       qt.failed(e, fname, debugHint);
     } finally {
-      String message = "Done query" + fname + ". succeeded=" + !failed + ", skipped=" + skipped +
+      String message = "Done query " + fname + ". succeeded=" + !failed + ", skipped=" + skipped +
           ". ElapsedTime(ms)=" + sw.stop().elapsed(TimeUnit.MILLISECONDS);
       LOG.info(message);
       System.err.println(message);

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
index 3cc55a9..fbca424 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
@@ -137,7 +137,7 @@ public class CoreCompareCliDriver extends CliAdapter{
       for (String versionFile : versionFiles) {
         // 1 for "_" after tname; 3 for ".qv" at the end. Version is in between.
         String versionStr = versionFile.substring(tname.length() + 1, versionFile.length() - 3);
-        outputs.add(qt.cliInit(tname + "." + versionStr, false));
+        outputs.add(qt.cliInit(new File(queryDirectory, tname + "." + versionStr), false));
         // TODO: will this work?
         ecode = qt.executeClient(versionFile, fname);
         if (ecode != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
index c38cb23..24bdc58 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
@@ -21,6 +21,8 @@ import static org.apache.hadoop.hive.cli.control.AbstractCliConfig.HIVE_ROOT;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.File;
+
 import org.apache.hadoop.hive.hbase.HBaseQTestUtil;
 import org.apache.hadoop.hive.hbase.HBaseTestSetup;
 import org.apache.hadoop.hive.ql.QTestProcessExecResult;
@@ -114,7 +116,7 @@ public class CoreHBaseCliDriver extends CliAdapter {
         return;
       }
 
-      qt.cliInit(fname, false);
+      qt.cliInit(new File(fpath), false);
 
       int ecode = qt.executeClient(fname);
       if (ecode != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
index c171278..e828dc7 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hive.cli.control;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.File;
+
 import org.apache.hadoop.hive.hbase.HBaseQTestUtil;
 import org.apache.hadoop.hive.hbase.HBaseTestSetup;
 import org.apache.hadoop.hive.ql.QTestProcessExecResult;
@@ -95,7 +97,7 @@ public class CoreHBaseNegativeCliDriver extends CliAdapter {
         return;
       }
 
-      qt.cliInit(fname);
+      qt.cliInit(new File(fpath));
       qt.clearTestSideEffects();
       int ecode = qt.executeClient(fname);
       if (ecode == 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
index 8476877..0788b2d 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hive.cli.control;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.File;
+
 import com.google.common.base.Strings;
 import org.apache.hadoop.hive.ql.QTestProcessExecResult;
 import org.apache.hadoop.hive.ql.QTestUtil;
@@ -119,7 +121,7 @@ public class CoreNegativeCliDriver extends CliAdapter{
         return;
       }
 
-      qt.cliInit(fname, false);
+      qt.cliInit(new File(fpath), false);
       int ecode = qt.executeClient(fname);
       if (ecode == 0) {
         qt.failed(fname, debugHint);

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
index 1a4c7df..8ef9e0a 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
@@ -23,6 +23,8 @@ package org.apache.hadoop.hive.cli.control;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.File;
+
 import com.google.common.base.Strings;
 import org.apache.hadoop.hive.ql.QTestProcessExecResult;
 import org.apache.hadoop.hive.ql.QTestUtil;
@@ -61,7 +63,7 @@ public class CorePerfCliDriver extends CliAdapter{
       String hadoopVer = cliConfig.getHadoopVersion();
       qt = new QTestUtil(cliConfig.getResultsDir(), cliConfig.getLogDir(), miniMR, hiveConfDir,
           hadoopVer, initScript,
-          cleanupScript, false);
+          cleanupScript, false, null);
 
       // do a one time initialization
       qt.cleanUp();
@@ -126,7 +128,7 @@ public class CorePerfCliDriver extends CliAdapter{
         return;
       }
 
-      qt.cliInit(fname, false);
+      qt.cliInit(new File(fpath), false);
 
       int ecode = qt.executeClient(fname);
       if (ecode != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 967b105..d6acce7 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -82,7 +82,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hive.cli.CliDriver;
 import org.apache.hadoop.hive.cli.CliSessionState;
@@ -123,6 +122,9 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.processors.HiveCommand;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.dataset.DatasetCollection;
+import org.apache.hadoop.hive.ql.dataset.DatasetParser;
+import org.apache.hadoop.hive.ql.dataset.Dataset;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.common.util.StreamPrinter;
@@ -165,8 +167,11 @@ public class QTestUtil {
   public static final String PATH_HDFS_REGEX = "(hdfs://)([a-zA-Z0-9:/_\\-\\.=])+";
   public static final String PATH_HDFS_WITH_DATE_USER_GROUP_REGEX = "([a-z]+) ([a-z]+)([ ]+)([0-9]+) ([0-9]{4}-[0-9]{2}-[0-9]{2} [0-9]{2}:[0-9]{2}) " + PATH_HDFS_REGEX;
 
+  public static final String TEST_SRC_TABLES_PROPERTY = "test.src.tables";
+
   private String testWarehouse;
   private final String testFiles;
+  private final File datasetDir;
   protected final String outDir;
   protected String overrideResultsDir;
   protected final String logDir;
@@ -179,7 +184,7 @@ public class QTestUtil {
   private final Set<String> qNoSessionReuseQuerySet;
   private final Set<String> qJavaVersionSpecificOutput;
   private static final String SORT_SUFFIX = ".sorted";
-  private final Set<String> srcTables;
+  private static Set<String> srcTables;
   private final Set<String> srcUDFs;
   private final MiniClusterType clusterType;
   private final FsType fsType;
@@ -210,16 +215,40 @@ public class QTestUtil {
   public interface SuiteAddTestFunctor {
     public void addTestToSuite(TestSuite suite, Object setup, String tName);
   }
-  private HBaseTestingUtility utility;
 
   public static Set<String> getSrcTables() {
-    HashSet<String> srcTables = new HashSet<String>();
+    if (srcTables == null){
+      initSrcTables();
+    }
+    return srcTables;
+  }
+
+  public static void addSrcTable(String table){
+    getSrcTables().add(table);
+    storeSrcTables();
+  }
+
+  public static Set<String> initSrcTables() {
+    if (srcTables == null){
+      initSrcTablesFromSystemProperty();
+      storeSrcTables();
+    }
+
+    return srcTables;
+  }
+
+  private static void storeSrcTables() {
+    System.setProperty(TEST_SRC_TABLES_PROPERTY, String.join(",", srcTables));
+  }
+
+  private static void initSrcTablesFromSystemProperty(){
+    srcTables = new HashSet<String>();
     // FIXME: moved default value to here...for now
     // i think this features is never really used from the command line
     String defaultTestSrcTables = "src,src1,srcbucket,srcbucket2,src_json,src_thrift," +
         "src_sequencefile,srcpart,alltypesorc,src_hbase,cbo_t1,cbo_t2,cbo_t3,src_cbo,part," +
         "lineitem,alltypesparquet";
-    for (String srcTable : System.getProperty("test.src.tables", defaultTestSrcTables).trim().split(",")) {
+    for (String srcTable : System.getProperty(TEST_SRC_TABLES_PROPERTY, defaultTestSrcTables).trim().split(",")) {
       srcTable = srcTable.trim();
       if (!srcTable.isEmpty()) {
         srcTables.add(srcTable);
@@ -228,9 +257,16 @@ public class QTestUtil {
     if (srcTables.isEmpty()) {
       throw new RuntimeException("Source tables cannot be empty");
     }
-    return srcTables;
   }
 
+  private CliDriver getCliDriver() {
+    if(cliDriver == null){
+      cliDriver = new CliDriver();
+    }
+    return cliDriver;
+  }
+
+
   /**
    * Returns the default UDF names which should not be removed when resetting the test database
    * @return The list of the UDF names not to remove
@@ -253,8 +289,6 @@ public class QTestUtil {
     return srcUDFs;
   }
 
-
-
   public HiveConf getConf() {
     return conf;
   }
@@ -285,7 +319,7 @@ public class QTestUtil {
         Path local_path = new Path(dest, name);
 
         // If this is a source table we do not copy it out
-        if (srcTables.contains(name)) {
+        if (getSrcTables().contains(name)) {
           continue;
         }
 
@@ -552,7 +586,6 @@ public class QTestUtil {
     }
     this.outDir = outDir;
     this.logDir = logDir;
-    this.srcTables=getSrcTables();
     this.srcUDFs = getSrcUDFs();
 
     // HIVE-14443 move this fall-back logic to CliConfigs
@@ -599,6 +632,11 @@ public class QTestUtil {
     }
     testFiles = dataDir;
 
+    // Use path relative to dataDir directory if it is not specified
+    datasetDir = conf.get("test.data.set.files") == null
+      ? new File(new File(dataDir).getAbsolutePath() + "/datasets")
+      : new File(conf.get("test.data.set.files"));
+
     // Use the current directory if it is not specified
     String scriptsDir = conf.get("test.data.scripts");
     if (scriptsDir == null) {
@@ -958,7 +996,7 @@ public class QTestUtil {
     for (String dbName : db.getAllDatabases()) {
       SessionState.get().setCurrentDatabase(dbName);
       for (String tblName : db.getAllTables()) {
-        if (!DEFAULT_DATABASE_NAME.equals(dbName) || !srcTables.contains(tblName)) {
+        if (!DEFAULT_DATABASE_NAME.equals(dbName) || !getSrcTables().contains(tblName)) {
           Table tblObj = null;
           try {
             tblObj = db.getTable(tblName);
@@ -986,7 +1024,7 @@ public class QTestUtil {
       FileSystem fileSystem = p.getFileSystem(conf);
       if (fileSystem.exists(p)) {
         for (FileStatus status : fileSystem.listStatus(p)) {
-          if (status.isDirectory() && !srcTables.contains(status.getPath().getName())) {
+          if (status.isDirectory() && !getSrcTables().contains(status.getPath().getName())) {
             fileSystem.delete(status.getPath(), true);
           }
         }
@@ -1037,8 +1075,8 @@ public class QTestUtil {
     cleanUp(null);
   }
 
-  public void cleanUp(String tname) throws Exception {
-    boolean canReuseSession = (tname == null) || !qNoSessionReuseQuerySet.contains(tname);
+  public void cleanUp(String fileName) throws Exception {
+    boolean canReuseSession = (fileName == null) || !qNoSessionReuseQuerySet.contains(fileName);
     if(!isSessionStateStarted) {
       startSessionState(canReuseSession);
     }
@@ -1050,23 +1088,7 @@ public class QTestUtil {
     clearUDFsCreatedDuringTests();
     clearKeysCreatedInTests();
 
-    File cleanupFile = new File(cleanupScript);
-    if (cleanupFile.isFile()) {
-      String cleanupCommands = readEntireFileIntoString(cleanupFile);
-      LOG.info("Cleanup (" + cleanupScript + "):\n" + cleanupCommands);
-      if(cliDriver == null) {
-        cliDriver = new CliDriver();
-      }
-      SessionState.get().getConf().setBoolean("hive.test.shutdown.phase", true);
-      int result = cliDriver.processLine(cleanupCommands);
-      if (result != 0) {
-        LOG.error("Failed during cleanup processLine with code={}. Ignoring", result);
-        // TODO Convert this to an Assert.fail once HIVE-14682 is fixed
-      }
-      SessionState.get().getConf().setBoolean("hive.test.shutdown.phase", false);
-    } else {
-      LOG.info("No cleanup script detected. Skipping.");
-    }
+    cleanupFromFile();
 
     // delete any contents in the warehouse dir
     Path p = new Path(testWarehouse);
@@ -1087,6 +1109,22 @@ public class QTestUtil {
     FunctionRegistry.unregisterTemporaryUDF("test_error");
   }
 
+  private void cleanupFromFile() throws IOException {
+    File cleanupFile = new File(cleanupScript);
+    if (cleanupFile.isFile()) {
+      String cleanupCommands = readEntireFileIntoString(cleanupFile);
+      LOG.info("Cleanup (" + cleanupScript + "):\n" + cleanupCommands);
+
+      int result = getCliDriver().processLine(cleanupCommands);
+      if (result != 0) {
+        LOG.error("Failed during cleanup processLine with code={}. Ignoring", result);
+        // TODO Convert this to an Assert.fail once HIVE-14682 is fixed
+      }
+    } else {
+      LOG.info("No cleanup script detected. Skipping.");
+    }
+  }
+
   protected void runCreateTableCmd(String createTableCmd) throws Exception {
     int ecode = 0;
     ecode = drv.run(createTableCmd).getResponseCode();
@@ -1113,22 +1151,27 @@ public class QTestUtil {
     createSources(null);
   }
 
-  public void createSources(String tname) throws Exception {
-    boolean canReuseSession = (tname == null) || !qNoSessionReuseQuerySet.contains(tname);
+  public void createSources(String fileName) throws Exception {
+    boolean canReuseSession = (fileName == null) || !qNoSessionReuseQuerySet.contains(fileName);
     if(!isSessionStateStarted) {
       startSessionState(canReuseSession);
     }
 
-    if(cliDriver == null) {
-      cliDriver = new CliDriver();
-    }
-    cliDriver.processLine("set test.data.dir=" + testFiles + ";");
+    getCliDriver().processLine("set test.data.dir=" + testFiles + ";");
+
+    conf.setBoolean("hive.test.init.phase", true);
+
+    initFromScript();
+
+    conf.setBoolean("hive.test.init.phase", false);
+  }
+
+  private void initFromScript() throws IOException {
     File scriptFile = new File(this.initScript);
     if (!scriptFile.isFile()) {
       LOG.info("No init script detected. Skipping");
       return;
     }
-    conf.setBoolean("hive.test.init.phase", true);
 
     String initCommands = readEntireFileIntoString(scriptFile);
     LOG.info("Initial setup (" + initScript + "):\n" + initCommands);
@@ -1138,8 +1181,40 @@ public class QTestUtil {
     if (result != 0) {
       Assert.fail("Failed during createSources processLine with code=" + result);
     }
+  }
+
+  private void initDataSetForTest(File file){
+    getCliDriver().processLine("set test.data.dir=" + testFiles + ";");
+
+    DatasetParser parser = new DatasetParser();
+    parser.parse(file);
+
+    DatasetCollection datasets = parser.getDatasets();
+    for (String table : datasets.getTables()){
+      initDataset(table);
+    }
+  }
+
+  private void initDataset(String table) {
+    if (getSrcTables().contains(table)){
+      return;
+    }
 
-    conf.setBoolean("hive.test.init.phase", false);
+    File tableFile = new File(new File(datasetDir, table), Dataset.INIT_FILE_NAME);
+    String commands = null;
+    try {
+      commands = readEntireFileIntoString(tableFile);
+    } catch (IOException e) {
+      throw new RuntimeException(String.format("dataset file not found %s", tableFile), e);
+    }
+
+    int result = getCliDriver().processLine(commands);
+    LOG.info("Result from cliDrriver.processLine in initFromDatasets=" + result);
+    if (result != 0) {
+      Assert.fail("Failed during initFromDatasets processLine with code=" + result);
+    }
+
+    addSrcTable(table);
   }
 
   public void init() throws Exception {
@@ -1165,22 +1240,26 @@ public class QTestUtil {
     sem = new SemanticAnalyzer(queryState);
   }
 
-  public void init(String tname) throws Exception {
-    cleanUp(tname);
-    createSources(tname);
+  public void init(String fileName) throws Exception {
+    cleanUp(fileName);
+    createSources(fileName);
     cliDriver.processCmd("set hive.cli.print.header=true;");
   }
 
-  public void cliInit(String tname) throws Exception {
-    cliInit(tname, true);
+  public void cliInit(File file) throws Exception {
+    cliInit(file, true);
   }
 
-  public String cliInit(String tname, boolean recreate) throws Exception {
+  public String cliInit(File file, boolean recreate) throws Exception {
+    String fileName = file.getName();
+
     if (recreate) {
-      cleanUp(tname);
-      createSources(tname);
+      cleanUp(fileName);
+      createSources(fileName);
     }
 
+    initDataSetForTest(file);
+
     HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
     "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
     Utilities.clearWorkMap(conf);
@@ -1188,23 +1267,23 @@ public class QTestUtil {
     assert ss != null;
     ss.in = System.in;
 
-    String outFileExtension = getOutFileExtension(tname);
+    String outFileExtension = getOutFileExtension(fileName);
     String stdoutName = null;
     if (outDir != null) {
       // TODO: why is this needed?
-      File qf = new File(outDir, tname);
+      File qf = new File(outDir, fileName);
       stdoutName = qf.getName().concat(outFileExtension);
     } else {
-      stdoutName = tname + outFileExtension;
+      stdoutName = fileName + outFileExtension;
     }
 
     File outf = new File(logDir, stdoutName);
     OutputStream fo = new BufferedOutputStream(new FileOutputStream(outf));
-    if (qSortQuerySet.contains(tname)) {
+    if (qSortQuerySet.contains(fileName)) {
       ss.out = new SortPrintStream(fo, "UTF-8");
-    } else if (qHashQuerySet.contains(tname)) {
+    } else if (qHashQuerySet.contains(fileName)) {
       ss.out = new DigestPrintStream(fo, "UTF-8");
-    } else if (qSortNHashQuerySet.contains(tname)) {
+    } else if (qSortNHashQuerySet.contains(fileName)) {
       ss.out = new SortAndDigestPrintStream(fo, "UTF-8");
     } else {
       ss.out = new PrintStream(fo, true, "UTF-8");
@@ -1213,7 +1292,7 @@ public class QTestUtil {
     ss.setIsSilent(true);
     SessionState oldSs = SessionState.get();
 
-    boolean canReuseSession = !qNoSessionReuseQuerySet.contains(tname);
+    boolean canReuseSession = !qNoSessionReuseQuerySet.contains(fileName);
     if (oldSs != null && canReuseSession && clusterType.getCoreClusterType() == CoreClusterType.TEZ) {
       // Copy the tezSessionState from the old CliSessionState.
       TezSessionState tezSessionState = oldSs.getTezSession();
@@ -1239,7 +1318,7 @@ public class QTestUtil {
 
     cliDriver = new CliDriver();
 
-    if (tname.equals("init_file.q")) {
+    if (fileName.equals("init_file.q")) {
       ss.initFiles.add(AbstractCliConfig.HIVE_ROOT + "/data/scripts/test_init_file.sql");
     }
     cliDriver.processInitFiles(ss);
@@ -1325,8 +1404,8 @@ public class QTestUtil {
     return executeClientInternal(commands);
   }
 
-  public int executeClient(String tname) {
-    return executeClientInternal(getCommand(tname));
+  public int executeClient(String fileName) {
+    return executeClientInternal(getCommand(fileName));
   }
 
   private int executeClientInternal(String commands) {
@@ -1453,8 +1532,8 @@ public class QTestUtil {
     return testCommand != null;
   }
 
-  private String getCommand(String tname) {
-    String commands = qMap.get(tname);
+  private String getCommand(String fileName) {
+    String commands = qMap.get(fileName);
     StringBuilder newCommands = new StringBuilder(commands.length());
     int lastMatchEnd = 0;
     Matcher commentMatcher = Pattern.compile("^--.*$", Pattern.MULTILINE).matcher(commands);
@@ -1512,6 +1591,7 @@ public class QTestUtil {
     } else if (e instanceof SemanticException) {
       outfd.write("Semantic Exception: \n");
     } else {
+      outfd.close();
       throw e;
     }
 
@@ -1753,6 +1833,7 @@ public class QTestUtil {
     partialPlanMask = ppm.toArray(new PatternReplacementPair[ppm.size()]);
   }
   /* This list may be modified by specific cli drivers to mask strings that change on every test */
+  @SuppressWarnings("serial")
   private final List<Pair<Pattern, String>> patternsWithMaskComments =
       new ArrayList<Pair<Pattern, String>>() {
         {
@@ -2035,16 +2116,16 @@ public class QTestUtil {
   }
 
   /**
-   * QTRunner: Runnable class for running a a single query file.
+   * QTRunner: Runnable class for running a single query file.
    *
    **/
   public static class QTRunner implements Runnable {
     private final QTestUtil qt;
-    private final String fname;
+    private final File file;
 
-    public QTRunner(QTestUtil qt, String fname) {
+    public QTRunner(QTestUtil qt, File file) {
       this.qt = qt;
-      this.fname = fname;
+      this.file = file;
     }
 
     @Override
@@ -2052,10 +2133,10 @@ public class QTestUtil {
       try {
         // assumption is that environment has already been cleaned once globally
         // hence each thread does not call cleanUp() and createSources() again
-        qt.cliInit(fname, false);
-        qt.executeClient(fname);
+        qt.cliInit(file, false);
+        qt.executeClient(file.getName());
       } catch (Throwable e) {
-        System.err.println("Query file " + fname + " failed with exception "
+        System.err.println("Query file " + file.getName() + " failed with exception "
             + e.getMessage());
         e.printStackTrace();
         outputTestFailureHelpMessage();
@@ -2108,7 +2189,7 @@ public class QTestUtil {
     qt[0].createSources();
     for (int i = 0; i < qfiles.length && !failed; i++) {
       qt[i].clearTestSideEffects();
-      qt[i].cliInit(qfiles[i].getName(), false);
+      qt[i].cliInit(qfiles[i], false);
       qt[i].executeClient(qfiles[i].getName());
       QTestProcessExecResult result = qt[i].checkCliDriverResults(qfiles[i].getName());
       if (result.getReturnCode() != 0) {
@@ -2158,7 +2239,7 @@ public class QTestUtil {
     Thread[] qtThread = new Thread[qfiles.length];
 
     for (int i = 0; i < qfiles.length; i++) {
-      qtRunners[i] = new QTRunner(qt[i], qfiles[i].getName());
+      qtRunners[i] = new QTRunner(qt[i], qfiles[i]);
       qtThread[i] = new Thread(qtRunners[i]);
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/Dataset.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/Dataset.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/Dataset.java
new file mode 100644
index 0000000..0729f76
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/Dataset.java
@@ -0,0 +1,36 @@
+/*
+ * 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.hadoop.hive.ql.dataset;
+
+/**
+ * Dataset: simple class representation of a dataset
+ */
+public class Dataset {
+  public static final String INIT_FILE_NAME = "load.hive.sql";
+  public static final String CLEANUP_FILE_NAME = "cleanup.hive.sql";
+
+  private String table;
+
+  public Dataset(String table) {
+    this.table = table;
+  }
+
+  public String getTable(){
+    return table;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/DatasetCollection.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/DatasetCollection.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/DatasetCollection.java
new file mode 100644
index 0000000..cf3ebd1
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/DatasetCollection.java
@@ -0,0 +1,46 @@
+/*
+ * 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.hadoop.hive.ql.dataset;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * DatasetCollection: utility wrapper class for a set of datasets
+ */
+public class DatasetCollection {
+  private Set<Dataset> coll = new HashSet<Dataset>();
+
+  public void add(Dataset dataset) {
+    coll.add(dataset);
+  }
+
+  public void add(String table) {
+    add(new Dataset(table));
+  }
+
+  public Set<Dataset> getDatasets() {
+    return coll;
+  }
+
+  public Set<String> getTables() {
+    return coll.stream().map(d -> d.getTable()).collect(Collectors.toSet());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/DatasetParser.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/DatasetParser.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/DatasetParser.java
new file mode 100644
index 0000000..ea0b0c5
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/dataset/DatasetParser.java
@@ -0,0 +1,73 @@
+/*
+ * 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.hadoop.hive.ql.dataset;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DatasetParser: a parser which could parse dataset "hooks" from q files, --!qt:dataset:mydataset
+ */
+public class DatasetParser {
+
+  private DatasetCollection datasets = new DatasetCollection();
+  private static final Logger LOG = LoggerFactory.getLogger("DatasetParser");
+
+  public static final String DATASET_PREFIX = "--! qt:dataset:";
+
+  public void parse(File file) {
+    try (BufferedReader br = new BufferedReader(new FileReader(file))) {
+      for (String line = br.readLine(); line != null; line = br.readLine()) {
+        if (line.trim().startsWith(DATASET_PREFIX)) {
+          Set<String> strDatasets = parseDatasetsFromLine(line);
+
+          for (String strDataset : strDatasets) {
+            datasets.add(strDataset);
+          }
+        }
+      }
+    } catch (IOException e) {
+      LOG.debug(
+          String.format("io exception while searching for datasets in qfile: %s", e.getMessage()));
+    }
+  }
+
+  public DatasetCollection getDatasets() {
+    return datasets;
+  }
+
+  public static Set<String> parseDatasetsFromLine(String input) {
+    Set<String> datasets = new HashSet<String>();
+
+    input = input.substring(DATASET_PREFIX.length());
+    if (!input.trim().isEmpty()) {
+      datasets.addAll(Arrays.asList(input.split(",")));
+    }
+
+    return datasets;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
index 8cb3fad..f087393 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.fail;
 import java.io.Serializable;
 import java.util.List;
 
-import com.google.common.base.Strings;
 import org.apache.hadoop.hive.cli.control.AbstractCliConfig;
 import org.apache.hadoop.hive.cli.control.CliAdapter;
 import org.apache.hadoop.hive.cli.control.CliConfigs;
@@ -34,6 +33,8 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
+import com.google.common.base.Strings;
+
 public class CoreParseNegative extends CliAdapter{
 
   private static QTestUtil qt;

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/itests/util/src/test/java/org/apache/hadoop/hive/ql/dataset/TestDatasetParser.java
----------------------------------------------------------------------
diff --git a/itests/util/src/test/java/org/apache/hadoop/hive/ql/dataset/TestDatasetParser.java b/itests/util/src/test/java/org/apache/hadoop/hive/ql/dataset/TestDatasetParser.java
new file mode 100644
index 0000000..3668a67
--- /dev/null
+++ b/itests/util/src/test/java/org/apache/hadoop/hive/ql/dataset/TestDatasetParser.java
@@ -0,0 +1,58 @@
+/*
+ * 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.hadoop.hive.ql.dataset;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * TestDatasetParser: test class for DataSetParser
+ */
+public class TestDatasetParser {
+
+  @Test
+  public void testParseOne() {
+    Set<String> expected = new HashSet<String>(Arrays.asList("mydataset"));
+    Assert.assertEquals(expected, DatasetParser
+        .parseDatasetsFromLine(String.format("%smydataset", DatasetParser.DATASET_PREFIX)));
+  }
+
+  @Test
+  public void testParseMultiple() {
+    Set<String> expected = new HashSet<String>(Arrays.asList("one", "two", "three"));
+    Assert.assertEquals(expected, DatasetParser
+        .parseDatasetsFromLine(String.format("%sone,two,three", DatasetParser.DATASET_PREFIX)));
+  }
+
+  @Test
+  public void testParseUnique() {
+    Set<String> expected = new HashSet<String>(Arrays.asList("one", "two"));
+    Assert.assertEquals(expected, DatasetParser
+        .parseDatasetsFromLine(String.format("%sone,one,two", DatasetParser.DATASET_PREFIX)));
+  }
+
+  @Test
+  public void testParseNone() {
+    Assert.assertTrue(DatasetParser
+        .parseDatasetsFromLine(String.format("%s", DatasetParser.DATASET_PREFIX)).isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/ql/src/java/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java
index cd4c61a..47fff8b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java
@@ -20,7 +20,8 @@ package org.apache.hadoop.hive.ql.hooks;
 
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
 
-import java.util.HashSet;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -33,20 +34,6 @@ import org.apache.hadoop.security.UserGroupInformation;
  */
 public class EnforceReadOnlyTables implements ExecuteWithHookContext {
 
-  private static final Set<String> READ_ONLY_TABLES = new HashSet<String>();
-
-  static {
-    for (String srcTable : System.getProperty("test.src.tables", "").trim().split(",")) {
-      srcTable = srcTable.trim();
-      if (!srcTable.isEmpty()) {
-        READ_ONLY_TABLES.add(srcTable);
-      }
-    }
-    if (READ_ONLY_TABLES.isEmpty()) {
-      throw new AssertionError("Source tables cannot be empty");
-    }
-  }
-
   @Override
   public void run(HookContext hookContext) throws Exception {
     SessionState ss = SessionState.get();
@@ -65,12 +52,14 @@ public class EnforceReadOnlyTables implements ExecuteWithHookContext {
         sess.getConf().getBoolean("hive.test.shutdown.phase", false)) {
       return;
     }
+    List<String> readOnlyTables = Arrays.asList(System.getProperty("test.src.tables").split(","));
+
     for (WriteEntity w: outputs) {
       if ((w.getTyp() == WriteEntity.Type.TABLE) ||
           (w.getTyp() == WriteEntity.Type.PARTITION)) {
         Table t = w.getTable();
         if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(t.getDbName())
-            && READ_ONLY_TABLES.contains(t.getTableName())) {
+            && readOnlyTables.contains(t.getTableName())) {
           throw new RuntimeException ("Cannot overwrite read-only table: " + t.getTableName());
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/ql/src/test/queries/clientpositive/testdataset.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/testdataset.q b/ql/src/test/queries/clientpositive/testdataset.q
new file mode 100644
index 0000000..96b78ad
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/testdataset.q
@@ -0,0 +1,2 @@
+--! qt:dataset:testdataset
+select count(*) from testdataset;

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/ql/src/test/queries/clientpositive/testdataset_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/testdataset_2.q b/ql/src/test/queries/clientpositive/testdataset_2.q
new file mode 100644
index 0000000..96b78ad
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/testdataset_2.q
@@ -0,0 +1,2 @@
+--! qt:dataset:testdataset
+select count(*) from testdataset;

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/ql/src/test/results/clientpositive/testdataset.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/testdataset.q.out b/ql/src/test/results/clientpositive/testdataset.q.out
new file mode 100644
index 0000000..2482bef
--- /dev/null
+++ b/ql/src/test/results/clientpositive/testdataset.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: select count(*) from testdataset
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testdataset
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from testdataset
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testdataset
+#### A masked pattern was here ####
+500

http://git-wip-us.apache.org/repos/asf/hive/blob/7cb31c03/ql/src/test/results/clientpositive/testdataset_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/testdataset_2.q.out b/ql/src/test/results/clientpositive/testdataset_2.q.out
new file mode 100644
index 0000000..2482bef
--- /dev/null
+++ b/ql/src/test/results/clientpositive/testdataset_2.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: select count(*) from testdataset
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testdataset
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from testdataset
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testdataset
+#### A masked pattern was here ####
+500