HIVE-14459: TestBeeLineDriver - migration and re-enable (Peter Vary via Zoltan 
Haindrich reviewed by Vihang Karajgaonkar)

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


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

Branch: refs/heads/master
Commit: ba8de3077673e3a9486c8a9d1625b9640936eedc
Parents: 92aaed2
Author: Peter Vary <pv...@cloudera.com>
Authored: Wed Mar 1 23:21:34 2017 +0100
Committer: Zoltan Haindrich <k...@rxd.hu>
Committed: Wed Mar 1 23:47:35 2017 +0100

----------------------------------------------------------------------
 .../apache/hive/beeline/util/QFileClient.java   |  72 ++-
 .../hive/jdbc/miniHS2/AbstractHiveService.java  | 159 ------
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   | 556 -------------------
 .../hive/cli/DisabledTestBeeLineDriver.java     |  62 ---
 .../hadoop/hive/cli/TestBeeLineDriver.java      |  62 +++
 .../test/resources/testconfiguration.properties | 158 +-----
 itests/util/pom.xml                             |   5 +
 .../hadoop/hive/cli/control/CliConfigs.java     |  14 +-
 .../hive/cli/control/CoreBeeLineDriver.java     |  95 ++--
 .../hive/jdbc/miniHS2/AbstractHiveService.java  | 159 ++++++
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   | 556 +++++++++++++++++++
 .../beeline/escape_comments.q.out               | 416 ++++++++++++++
 12 files changed, 1307 insertions(+), 1007 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/beeline/src/java/org/apache/hive/beeline/util/QFileClient.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/util/QFileClient.java 
b/beeline/src/java/org/apache/hive/beeline/util/QFileClient.java
index 81f1b0e..d306b7f 100644
--- a/beeline/src/java/org/apache/hive/beeline/util/QFileClient.java
+++ b/beeline/src/java/org/apache/hive/beeline/util/QFileClient.java
@@ -21,12 +21,15 @@ package org.apache.hive.beeline.util;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.regex.Pattern;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.util.Shell;
+import org.apache.hive.common.util.StreamPrinter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -49,6 +52,8 @@ public class QFileClient {
   private File expectedDirectory;
   private final File scratchDirectory;
   private final File warehouseDirectory;
+  private final File initScript;
+  private final File cleanupScript;
 
   private File testDataDirectory;
   private File testScriptDirectory;
@@ -73,11 +78,13 @@ public class QFileClient {
 
 
   public QFileClient(HiveConf hiveConf, String hiveRootDirectory, String 
qFileDirectory, String outputDirectory,
-      String expectedDirectory) {
+      String expectedDirectory, String initScript, String cleanupScript) {
     this.hiveRootDirectory = new File(hiveRootDirectory);
     this.qFileDirectory = new File(qFileDirectory);
     this.outputDirectory = new File(outputDirectory);
     this.expectedDirectory = new File(expectedDirectory);
+    this.initScript = new File(initScript);
+    this.cleanupScript = new File(cleanupScript);
     this.scratchDirectory = new File(hiveConf.getVar(ConfVars.SCRATCHDIR));
     this.warehouseDirectory = new 
File(hiveConf.getVar(ConfVars.METASTOREWAREHOUSE));
   }
@@ -110,6 +117,9 @@ public class QFileClient {
     String timePattern = "(Mon|Tue|Wed|Thu|Fri|Sat|Sun) "
         + "(Jan|Feb|Mar|Apr|May|Jun|Jul|Aug|Sep|Oct|Nov|Dec) "
         + "\\d{2} \\d{2}:\\d{2}:\\d{2} \\w+ 20\\d{2}";
+    // Pattern to remove the timestamp and other infrastructural info from the 
out file
+    String logPattern = 
"\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d*\\s+\\S+\\s+\\[" +
+                            ".*\\]\\s+\\S+:\\s+";
     String unixTimePattern = "\\D" + currentTimePrefix + "\\d{6}\\D";
     String unixTimeMillisPattern = "\\D" + currentTimePrefix + "\\d{9}\\D";
 
@@ -119,12 +129,15 @@ public class QFileClient {
       + "|SCR|SEL|STATS|TS|UDTF|UNION)_\\d+\"";
 
     filterSet = new RegexFilterSet()
+    .addFilter(logPattern,"")
+    .addFilter("Getting log thread is interrupted, since query is done!\n","")
     .addFilter(scratchDirectory.toString() + "[\\w\\-/]+", 
"!!{hive.exec.scratchdir}!!")
     .addFilter(warehouseDirectory.toString(), 
"!!{hive.metastore.warehouse.dir}!!")
     .addFilter(expectedDirectory.toString(), "!!{expectedDirectory}!!")
     .addFilter(outputDirectory.toString(), "!!{outputDirectory}!!")
     .addFilter(qFileDirectory.toString(), "!!{qFileDirectory}!!")
     .addFilter(hiveRootDirectory.toString(), "!!{hive.root}!!")
+    .addFilter("\\(queryId=[^\\)]*\\)","queryId=(!!{queryId}!!)")
     .addFilter("file:/\\w\\S+", "file:/!!ELIDED!!")
     .addFilter("pfile:/\\w\\S+", "pfile:/!!ELIDED!!")
     .addFilter("hdfs:/\\w\\S+", "hdfs:/!!ELIDED!!")
@@ -134,6 +147,7 @@ public class QFileClient {
     .addFilter("(\\D)" + currentTimePrefix + "\\d{9}(\\D)", 
"$1!!UNIXTIMEMILLIS!!$2")
     .addFilter(userName, "!!{user.name}!!")
     .addFilter(operatorPattern, "\"$1_!!ELIDED!!\"")
+    .addFilter("Time taken: [0-9\\.]* seconds", "Time taken: !!ELIDED!! 
seconds")
     ;
   };
 
@@ -219,7 +233,7 @@ public class QFileClient {
         "USE `" + testname + "`;",
         "set test.data.dir=" + testDataDirectory + ";",
         "set test.script.dir=" + testScriptDirectory + ";",
-        "!run " + testScriptDirectory + "/q_test_init.sql",
+        "!run " + testScriptDirectory + "/" + initScript,
     });
   }
 
@@ -228,6 +242,7 @@ public class QFileClient {
         "!set outputformat table",
         "USE default;",
         "DROP DATABASE IF EXISTS `" + testname + "` CASCADE;",
+        "!run " + testScriptDirectory + "/" + cleanupScript,
     });
   }
 
@@ -295,12 +310,61 @@ public class QFileClient {
     return expectedFile.exists();
   }
 
-  public boolean compareResults() throws IOException {
+  public boolean compareResults() throws IOException, InterruptedException {
     if (!expectedFile.exists()) {
       LOG.error("Expected results file does not exist: " + expectedFile);
       return false;
     }
-    return FileUtils.contentEquals(expectedFile, outputFile);
+    return executeDiff();
+  }
+
+  private boolean executeDiff() throws IOException, InterruptedException {
+    ArrayList<String> diffCommandArgs = new ArrayList<String>();
+    diffCommandArgs.add("diff");
+
+    // Text file comparison
+    diffCommandArgs.add("-a");
+
+    if (Shell.WINDOWS) {
+      // Ignore changes in the amount of white space
+      diffCommandArgs.add("-b");
+
+      // Files created on Windows machines have different line endings
+      // than files created on Unix/Linux. Windows uses carriage return and 
line feed
+      // ("\r\n") as a line ending, whereas Unix uses just line feed ("\n").
+      // Also StringBuilder.toString(), Stream to String conversions adds extra
+      // spaces at the end of the line.
+      diffCommandArgs.add("--strip-trailing-cr"); // Strip trailing carriage 
return on input
+      diffCommandArgs.add("-B"); // Ignore changes whose lines are all blank
+    }
+
+    // Add files to compare to the arguments list
+    diffCommandArgs.add(getQuotedString(expectedFile));
+    diffCommandArgs.add(getQuotedString(outputFile));
+
+    System.out.println("Running: " + 
org.apache.commons.lang.StringUtils.join(diffCommandArgs,
+        ' '));
+    Process executor = Runtime.getRuntime().exec(diffCommandArgs.toArray(
+        new String[diffCommandArgs.size()]));
+
+    StreamPrinter errPrinter = new StreamPrinter(executor.getErrorStream(), 
null, System.err);
+    StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), 
null, System.out);
+
+    outPrinter.start();
+    errPrinter.start();
+
+    int result = executor.waitFor();
+
+    outPrinter.join();
+    errPrinter.join();
+
+    executor.waitFor();
+
+    return (result == 0);
+  }
+
+  private static String getQuotedString(File file) {
+    return Shell.WINDOWS ? String.format("\"%s\"", file.getAbsolutePath()) : 
file.getAbsolutePath();
   }
 
   public void overwriteResults() {

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
----------------------------------------------------------------------
diff --git 
a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
 
b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
deleted file mode 100644
index 98c5d01..0000000
--- 
a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/**
- * 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.hive.jdbc.miniHS2;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-
-/***
- * Base class for Hive service
- * AbstractHiveService.
- *
- */
-public abstract class AbstractHiveService {
-  private HiveConf hiveConf = null;
-  private String hostname;
-  private int binaryPort;
-  private int httpPort;
-  private boolean startedHiveService = false;
-  private List<String> addedProperties = new ArrayList<String>();
-
-  public AbstractHiveService(HiveConf hiveConf, String hostname, int 
binaryPort, int httpPort) {
-    this.hiveConf = hiveConf;
-    this.hostname = hostname;
-    this.binaryPort = binaryPort;
-    this.httpPort = httpPort;
-  }
-
-  /**
-   * Get Hive conf
-   * @return
-   */
-  public HiveConf getHiveConf() {
-    return hiveConf;
-  }
-
-  /**
-   * Get config property
-   * @param propertyKey
-   * @return
-   */
-  public String getConfProperty(String propertyKey) {
-    return hiveConf.get(propertyKey);
-  }
-
-  /**
-   * Set config property
-   * @param propertyKey
-   * @param propertyValue
-   */
-  public void setConfProperty(String propertyKey, String propertyValue) {
-    System.setProperty(propertyKey, propertyValue);
-    hiveConf.set(propertyKey, propertyValue);
-    addedProperties.add(propertyKey);
-  }
-
-  /**
-   * Create system properties set by this server instance. This ensures that
-   * the changes made by current test are not impacting subsequent tests.
-   */
-  public void clearProperties() {
-    for (String propKey : addedProperties ) {
-      System.clearProperty(propKey);
-    }
-  }
-
-  /**
-   * Retrieve warehouse directory
-   * @return
-   */
-  public Path getWareHouseDir() {
-    return new Path(hiveConf.getVar(ConfVars.METASTOREWAREHOUSE));
-  }
-
-  public void setWareHouseDir(String wareHouseURI) {
-    verifyNotStarted();
-    System.setProperty(ConfVars.METASTOREWAREHOUSE.varname, wareHouseURI);
-    hiveConf.setVar(ConfVars.METASTOREWAREHOUSE, wareHouseURI);
-  }
-
-  /**
-   * Set service host
-   * @param hostName
-   */
-  public void setHost(String hostName) {
-    this.hostname = hostName;
-  }
-
-  // get service host
-  public String getHost() {
-    return hostname;
-  }
-
-  /**
-   * Set binary service port #
-   * @param portNum
-   */
-  public void setBinaryPort(int portNum) {
-    this.binaryPort = portNum;
-  }
-
-  /**
-   * Set http service port #
-   * @param portNum
-   */
-  public void setHttpPort(int portNum) {
-    this.httpPort = portNum;
-  }
-
-  // Get binary service port #
-  public int getBinaryPort() {
-    return binaryPort;
-  }
-
-  // Get http service port #
-  public int getHttpPort() {
-    return httpPort;
-  }
-
-  public boolean isStarted() {
-    return startedHiveService;
-  }
-
-  public void setStarted(boolean hiveServiceStatus) {
-    this.startedHiveService =  hiveServiceStatus;
-  }
-
-  protected void verifyStarted() {
-    if (!isStarted()) {
-      throw new IllegalStateException("HiveServer2 is not running");
-    }
-  }
-
-  protected void verifyNotStarted() {
-    if (isStarted()) {
-      throw new IllegalStateException("HiveServer2 already running");
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git 
a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java 
b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
deleted file mode 100644
index 71f9640..0000000
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ /dev/null
@@ -1,556 +0,0 @@
-/**
- * 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.hive.jdbc.miniHS2;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.llap.LlapItUtils;
-import org.apache.hadoop.hive.llap.daemon.MiniLlapCluster;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper;
-import org.apache.hadoop.hive.shims.HadoopShims.MiniDFSShim;
-import org.apache.hadoop.hive.shims.HadoopShims.MiniMrShim;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hive.service.Service;
-import org.apache.hive.service.cli.CLIServiceClient;
-import org.apache.hive.service.cli.SessionHandle;
-import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService;
-import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
-import org.apache.hive.service.cli.thrift.ThriftHttpCLIService;
-import org.apache.hive.service.server.HiveServer2;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MiniHS2 extends AbstractHiveService {
-
-  private static final Logger LOG = LoggerFactory.getLogger(MiniHS2.class);
-
-  public static final String HS2_BINARY_MODE = "binary";
-  public static final String HS2_HTTP_MODE = "http";
-  private static final String driverName = "org.apache.hive.jdbc.HiveDriver";
-  private static final FsPermission FULL_PERM = new FsPermission((short)00777);
-  private static final FsPermission WRITE_ALL_PERM = new 
FsPermission((short)00733);
-  private static final String tmpDir = System.getProperty("test.tmp.dir");
-  private HiveServer2 hiveServer2 = null;
-  private final File baseDir;
-  private final Path baseFsDir;
-  private MiniMrShim mr;
-  private MiniDFSShim dfs;
-  private MiniLlapCluster llapCluster = null;
-  private final FileSystem localFS;
-  private boolean useMiniKdc = false;
-  private final String serverPrincipal;
-  private final boolean isMetastoreRemote;
-  private final boolean cleanupLocalDirOnStartup;
-  private MiniClusterType miniClusterType = MiniClusterType.LOCALFS_ONLY;
-
-  public enum MiniClusterType {
-    MR,
-    TEZ,
-    LLAP,
-    LOCALFS_ONLY;
-  }
-
-  public static class Builder {
-    private HiveConf hiveConf = new HiveConf();
-    private MiniClusterType miniClusterType = MiniClusterType.LOCALFS_ONLY;
-    private boolean useMiniKdc = false;
-    private String serverPrincipal;
-    private String serverKeytab;
-    private boolean isHTTPTransMode = false;
-    private boolean isMetastoreRemote;
-    private boolean usePortsFromConf = false;
-    private String authType = "KERBEROS";
-    private boolean isHA = false;
-    private boolean cleanupLocalDirOnStartup = true;
-
-    public Builder() {
-    }
-
-    public Builder withMiniMR() {
-      this.miniClusterType = MiniClusterType.MR;
-      return this;
-    }
-
-    public Builder withMiniKdc(String serverPrincipal, String serverKeytab) {
-      this.useMiniKdc = true;
-      this.serverPrincipal = serverPrincipal;
-      this.serverKeytab = serverKeytab;
-      return this;
-    }
-
-    public Builder withAuthenticationType(String authType) {
-      this.authType = authType;
-      return this;
-    }
-
-    public Builder withRemoteMetastore() {
-      this.isMetastoreRemote = true;
-      return this;
-    }
-
-    public Builder withConf(HiveConf hiveConf) {
-      this.hiveConf = hiveConf;
-      return this;
-    }
-
-    public Builder withHA() {
-      this.isHA = true;
-      return this;
-    }
-
-    /**
-     * Start HS2 with HTTP transport mode, default is binary mode
-     * @return this Builder
-     */
-    public Builder withHTTPTransport(){
-      this.isHTTPTransMode = true;
-      return this;
-    }
-
-    public Builder cleanupLocalDirOnStartup(boolean val) {
-      this.cleanupLocalDirOnStartup = val;
-      return this;
-    }
-
-    public MiniHS2 build() throws Exception {
-      if (miniClusterType == MiniClusterType.MR && useMiniKdc) {
-        throw new IOException("Can't create secure miniMr ... yet");
-      }
-      if (isHTTPTransMode) {
-        hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_HTTP_MODE);
-      } else {
-        hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_BINARY_MODE);
-      }
-      return new MiniHS2(hiveConf, miniClusterType, useMiniKdc, 
serverPrincipal, serverKeytab,
-          isMetastoreRemote, usePortsFromConf, authType, isHA, 
cleanupLocalDirOnStartup);
-    }
-  }
-
-  public MiniMrShim getMr() {
-    return mr;
-  }
-
-  public void setMr(MiniMrShim mr) {
-    this.mr = mr;
-  }
-
-  public MiniDFSShim getDfs() {
-    return dfs;
-  }
-
-  public void setDfs(MiniDFSShim dfs) {
-    this.dfs = dfs;
-  }
-
-  public FileSystem getLocalFS() {
-    return localFS;
-  }
-
-  public MiniClusterType getMiniClusterType() {
-    return miniClusterType;
-  }
-
-  public void setMiniClusterType(MiniClusterType miniClusterType) {
-    this.miniClusterType = miniClusterType;
-  }
-
-  public boolean isUseMiniKdc() {
-    return useMiniKdc;
-  }
-
-  private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean 
useMiniKdc,
-      String serverPrincipal, String serverKeytab, boolean isMetastoreRemote,
-      boolean usePortsFromConf, String authType, boolean isHA, boolean 
cleanupLocalDirOnStartup) throws Exception {
-    // Always use localhost for hostname as some tests like SSL CN validation 
ones
-    // are tied to localhost being present in the certificate name
-    super(
-        hiveConf,
-        "localhost",
-        (usePortsFromConf ? 
hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT) : MetaStoreUtils
-            .findFreePort()),
-        (usePortsFromConf ? 
hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT) : 
MetaStoreUtils
-            .findFreePort()));
-    hiveConf.setLongVar(ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS, 3l);
-    
hiveConf.setTimeVar(ConfVars.HIVE_SERVER2_SLEEP_INTERVAL_BETWEEN_START_ATTEMPTS,
 10,
-        TimeUnit.SECONDS);
-    this.miniClusterType = miniClusterType;
-    this.useMiniKdc = useMiniKdc;
-    this.serverPrincipal = serverPrincipal;
-    this.isMetastoreRemote = isMetastoreRemote;
-    this.cleanupLocalDirOnStartup = cleanupLocalDirOnStartup;
-    baseDir = getBaseDir();
-    localFS = FileSystem.getLocal(hiveConf);
-    FileSystem fs;
-
-    if (miniClusterType != MiniClusterType.LOCALFS_ONLY) {
-      // Initialize dfs
-      dfs = ShimLoader.getHadoopShims().getMiniDfs(hiveConf, 4, true, null, 
isHA);
-      fs = dfs.getFileSystem();
-      String uriString = fs.getUri().toString();
-
-      // Initialize the execution engine based on cluster type
-      switch (miniClusterType) {
-      case TEZ:
-        // Change the engine to tez
-        hiveConf.setVar(ConfVars.HIVE_EXECUTION_ENGINE, "tez");
-        // TODO: This should be making use of confDir to load configs setup 
for Tez, etc.
-        mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 2, 
uriString, false);
-        break;
-      case LLAP:
-        if (usePortsFromConf) {
-          hiveConf.setBoolean("minillap.usePortsFromConf", true);
-        }
-        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(hiveConf, null, 
null);
-
-        mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 2, 
uriString, true);
-        break;
-      case MR:
-        mr = ShimLoader.getHadoopShims().getMiniMrCluster(hiveConf, 2, 
uriString, 1);
-        break;
-      default:
-        throw new IllegalArgumentException("Unsupported cluster type " + mr);
-      }
-      // store the config in system properties
-      mr.setupConfiguration(getHiveConf());
-      baseFsDir = new Path(new Path(fs.getUri()), "/base");
-    } else {
-      // This is FS only mode, just initialize the dfs root directory.
-      fs = FileSystem.getLocal(hiveConf);
-      baseFsDir = new Path("file://" + baseDir.toURI().getPath());
-
-      if (cleanupLocalDirOnStartup) {
-        // Cleanup baseFsDir since it can be shared across tests.
-        LOG.info("Attempting to cleanup baseFsDir: {} while setting up 
MiniHS2", baseDir);
-        Preconditions.checkState(baseFsDir.depth() >= 3); // Avoid "/tmp", 
directories closer to "/"
-        fs.delete(baseFsDir, true);
-      }
-    }
-    if (useMiniKdc) {
-      hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL, 
serverPrincipal);
-      hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB, serverKeytab);
-      hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, authType);
-    }
-    String metaStoreURL =
-        "jdbc:derby:;databaseName=" + baseDir.getAbsolutePath() + 
File.separator
-            + "test_metastore;create=true";
-
-    fs.mkdirs(baseFsDir);
-    Path wareHouseDir = new Path(baseFsDir, "warehouse");
-    // Create warehouse with 777, so that user impersonation has no issues.
-    FileSystem.mkdirs(fs, wareHouseDir, FULL_PERM);
-
-    fs.mkdirs(wareHouseDir);
-    setWareHouseDir(wareHouseDir.toString());
-    System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, 
metaStoreURL);
-    hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, metaStoreURL);
-    if (!usePortsFromConf) {
-      // reassign a new port, just in case if one of the MR services grabbed 
the last one
-      setBinaryPort(MetaStoreUtils.findFreePort());
-    }
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, getHost());
-    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, getBinaryPort());
-    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, getHttpPort());
-
-    Path scratchDir = new Path(baseFsDir, "scratch");
-    // Create root scratchdir with write all, so that user impersonation has 
no issues.
-    Utilities.createDirsWithPermission(hiveConf, scratchDir, WRITE_ALL_PERM, 
true);
-    System.setProperty(HiveConf.ConfVars.SCRATCHDIR.varname, 
scratchDir.toString());
-    hiveConf.setVar(ConfVars.SCRATCHDIR, scratchDir.toString());
-
-    String localScratchDir = baseDir.getPath() + File.separator + "scratch";
-    System.setProperty(HiveConf.ConfVars.LOCALSCRATCHDIR.varname, 
localScratchDir);
-    hiveConf.setVar(ConfVars.LOCALSCRATCHDIR, localScratchDir);
-  }
-
-  public MiniHS2(HiveConf hiveConf) throws Exception {
-    this(hiveConf, MiniClusterType.LOCALFS_ONLY);
-  }
-
-  public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType) throws 
Exception {
-    this(hiveConf, clusterType, false);
-  }
-
-  public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType,
-      boolean usePortsFromConf) throws Exception {
-    this(hiveConf, clusterType, false, null, null, false, usePortsFromConf,
-      "KERBEROS", false, true);
-  }
-
-  public void start(Map<String, String> confOverlay) throws Exception {
-    if (isMetastoreRemote) {
-      int metaStorePort = MetaStoreUtils.findFreePort();
-      getHiveConf().setVar(ConfVars.METASTOREURIS, "thrift://localhost:" + 
metaStorePort);
-      MetaStoreUtils.startMetaStore(metaStorePort,
-      ShimLoader.getHadoopThriftAuthBridge(), getHiveConf());
-    }
-
-    hiveServer2 = new HiveServer2();
-    // Set confOverlay parameters
-    for (Map.Entry<String, String> entry : confOverlay.entrySet()) {
-      setConfProperty(entry.getKey(), entry.getValue());
-    }
-    hiveServer2.init(getHiveConf());
-    hiveServer2.start();
-    waitForStartup();
-    setStarted(true);
-  }
-
-  public void stop() {
-    verifyStarted();
-    // Currently there is no way to stop the MetaStore service. It will be 
stopped when the
-    // test JVM exits. This is how other tests are also using MetaStore server.
-
-    hiveServer2.stop();
-    setStarted(false);
-    try {
-      if (llapCluster != null) {
-        llapCluster.stop();
-      }
-      if (mr != null) {
-        mr.shutdown();
-        mr = null;
-      }
-      if (dfs != null) {
-        dfs.shutdown();
-        dfs = null;
-      }
-    } catch (IOException e) {
-      // Ignore errors cleaning up miniMR
-    }
-  }
-
-  public void cleanup() {
-    FileUtils.deleteQuietly(baseDir);
-  }
-
-
-  public CLIServiceClient getServiceClient() {
-    verifyStarted();
-    return getServiceClientInternal();
-  }
-
-  public HiveConf getServerConf() {
-    if (hiveServer2 != null) {
-      return hiveServer2.getHiveConf();
-    }
-    return null;
-  }
-
-  public CLIServiceClient getServiceClientInternal() {
-    for (Service service : hiveServer2.getServices()) {
-      if (service instanceof ThriftBinaryCLIService) {
-        return new ThriftCLIServiceClient((ThriftBinaryCLIService) service);
-      }
-      if (service instanceof ThriftHttpCLIService) {
-        return new ThriftCLIServiceClient((ThriftHttpCLIService) service);
-      }
-    }
-    throw new IllegalStateException("HiveServer2 not running Thrift service");
-  }
-
-  /**
-   * return connection URL for this server instance
-   * @return
-   * @throws Exception
-   */
-  public String getJdbcURL() throws Exception {
-    return getJdbcURL("default");
-  }
-
-  /**
-   * return connection URL for this server instance
-   * @param dbName - DB name to be included in the URL
-   * @return
-   * @throws Exception
-   */
-  public String getJdbcURL(String dbName) throws Exception {
-    return getJdbcURL(dbName, "");
-  }
-
-  /**
-   * return connection URL for this server instance
-   * @param dbName - DB name to be included in the URL
-   * @param sessionConfExt - Addional string to be appended to sessionConf 
part of url
-   * @return
-   * @throws Exception
-   */
-  public String getJdbcURL(String dbName, String sessionConfExt) throws 
Exception {
-    return getJdbcURL(dbName, sessionConfExt, "");
-  }
-
-  /**
-   * return connection URL for this server instance
-   * @param dbName - DB name to be included in the URL
-   * @param sessionConfExt - Addional string to be appended to sessionConf 
part of url
-   * @param hiveConfExt - Additional string to be appended to HiveConf part of 
url (excluding the ?)
-   * @return
-   * @throws Exception
-   */
-  public String getJdbcURL(String dbName, String sessionConfExt, String 
hiveConfExt)
-      throws Exception {
-    sessionConfExt = (sessionConfExt == null ? "" : sessionConfExt);
-    hiveConfExt = (hiveConfExt == null ? "" : hiveConfExt);
-    // Strip the leading ";" if provided
-    // (this is the assumption with which we're going to start configuring 
sessionConfExt)
-    if (sessionConfExt.startsWith(";")) {
-      sessionConfExt = sessionConfExt.substring(1);
-    }
-    if (isUseMiniKdc()) {
-      sessionConfExt = "principal=" + serverPrincipal + ";" + sessionConfExt;
-    }
-    if (isHttpTransportMode()) {
-      sessionConfExt = "transportMode=http;httpPath=cliservice" + ";" + 
sessionConfExt;
-    }
-    String baseJdbcURL;
-    if (isDynamicServiceDiscovery()) {
-      sessionConfExt =
-          "serviceDiscoveryMode=zooKeeper;zooKeeperNamespace="
-              + 
getServerConf().getVar(HiveConf.ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE) + ";"
-              + sessionConfExt;
-      baseJdbcURL = getZKBaseJdbcURL();
-    } else {
-      baseJdbcURL = getBaseJdbcURL();
-    }
-
-    baseJdbcURL = baseJdbcURL + dbName;
-    if (!sessionConfExt.isEmpty()) {
-      baseJdbcURL = baseJdbcURL + ";" + sessionConfExt;
-    }
-    if ((hiveConfExt != null) && (!hiveConfExt.trim().isEmpty())) {
-      baseJdbcURL = baseJdbcURL + "?" + hiveConfExt;
-    }
-    return baseJdbcURL;
-  }
-
-  /**
-   * Build base JDBC URL
-   * @return
-   */
-  public String getBaseJdbcURL() {
-    if(isHttpTransportMode()) {
-      return "jdbc:hive2://" + getHost() + ":" + getHttpPort() + "/";
-    }
-    else {
-      return "jdbc:hive2://" + getHost() + ":" + getBinaryPort() + "/";
-    }
-  }
-
-  /**
-   * Build zk base JDBC URL
-   * @return
-   */
-  private String getZKBaseJdbcURL() throws Exception {
-    HiveConf hiveConf = getServerConf();
-    if (hiveConf != null) {
-      String zkEnsemble =  ZooKeeperHiveHelper.getQuorumServers(hiveConf);
-      return "jdbc:hive2://" + zkEnsemble + "/";
-    }
-    throw new Exception("Server's HiveConf is null. Unable to read ZooKeeper 
configs.");
-  }
-
-  private boolean isHttpTransportMode() {
-    String transportMode = 
getConfProperty(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname);
-    return transportMode != null && 
(transportMode.equalsIgnoreCase(HS2_HTTP_MODE));
-  }
-
-  private boolean isDynamicServiceDiscovery() throws Exception {
-    HiveConf hiveConf = getServerConf();
-    if (hiveConf == null) {
-      throw new Exception("Server's HiveConf is null. Unable to read ZooKeeper 
configs.");
-    }
-    if 
(hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY)) {
-      return true;
-    }
-    return false;
-  }
-
-  public static String getJdbcDriverName() {
-    return driverName;
-  }
-
-  public MiniMrShim getMR() {
-    return mr;
-  }
-
-  public MiniDFSShim getDFS() {
-    return dfs;
-  }
-
-  private void waitForStartup() throws Exception {
-    int waitTime = 0;
-    long startupTimeout = 1000L * 1000L;
-    CLIServiceClient hs2Client = getServiceClientInternal();
-    SessionHandle sessionHandle = null;
-    do {
-      Thread.sleep(500L);
-      waitTime += 500L;
-      if (waitTime > startupTimeout) {
-        throw new TimeoutException("Couldn't access new HiveServer2: " + 
getJdbcURL());
-      }
-      try {
-        Map <String, String> sessionConf = new HashMap<String, String>();
-        /**
-        if (isUseMiniKdc()) {
-          getMiniKdc().loginUser(getMiniKdc().getDefaultUserPrincipal());
-          sessionConf.put("principal", serverPrincipal);
-        }
-         */
-        sessionHandle = hs2Client.openSession("foo", "bar", sessionConf);
-      } catch (Exception e) {
-        // service not started yet
-        continue;
-      }
-      hs2Client.closeSession(sessionHandle);
-      break;
-    } while (true);
-  }
-
-  public Service.STATE getState() {
-    return hiveServer2.getServiceState();
-  }
-
-  static File getBaseDir() {
-    File baseDir = new File(tmpDir + "/local_base");
-    return baseDir;
-  }
-
-  public static void cleanupLocalDir() throws IOException {
-    File baseDir = getBaseDir();
-    try {
-      org.apache.hadoop.hive.common.FileUtils.deleteDirectory(baseDir);
-    } catch (FileNotFoundException e) {
-      // Ignore. Safe if it does not exist.
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DisabledTestBeeLineDriver.java
----------------------------------------------------------------------
diff --git 
a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DisabledTestBeeLineDriver.java
 
b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DisabledTestBeeLineDriver.java
deleted file mode 100644
index cb276e6..0000000
--- 
a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DisabledTestBeeLineDriver.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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.cli;
-
-import java.io.File;
-import java.util.List;
-
-import org.apache.hadoop.hive.cli.control.CliAdapter;
-import org.apache.hadoop.hive.cli.control.CliConfigs;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
-public class DisabledTestBeeLineDriver {
-
-  static CliAdapter adapter = new CliConfigs.BeeLineConfig().getCliAdapter();
-
-  @Parameters(name = "{0}")
-  public static List<Object[]> getParameters() throws Exception {
-    return adapter.getParameters();
-  }
-
-  @ClassRule
-  public static TestRule cliClassRule = adapter.buildClassRule();
-
-  @Rule
-  public TestRule cliTestRule = adapter.buildTestRule();
-
-  private String name;
-  private File qfile;
-
-  public DisabledTestBeeLineDriver(String name, File qfile) {
-    this.name = name;
-    this.qfile = qfile;
-  }
-
-  @Test
-  public void testCliDriver() throws Exception {
-    adapter.runTest(name, qfile);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java
----------------------------------------------------------------------
diff --git 
a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java 
b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java
new file mode 100644
index 0000000..24eeb9d
--- /dev/null
+++ 
b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java
@@ -0,0 +1,62 @@
+/**
+ * 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.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestBeeLineDriver {
+
+  static CliAdapter adapter = new CliConfigs.BeeLineConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestBeeLineDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties 
b/itests/src/test/resources/testconfiguration.properties
index 9c6a069..b01ebd8 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -722,163 +722,7 @@ encrypted.query.files=encryption_join_unencrypted_tbl.q,\
   encryption_with_trash.q \
   encryption_ctas.q
 
-beeline.positive.exclude=add_part_exist.q,\
-  alter1.q,\
-  alter2.q,\
-  alter4.q,\
-  alter5.q,\
-  alter_rename_partition.q,\
-  alter_rename_partition_authorization.q,\
-  archive.q,\
-  archive_corrupt.q,\
-  archive_mr_1806.q,\
-  archive_multi.q,\
-  archive_multi_mr_1806.q,\
-  authorization_1.q,\
-  authorization_2.q,\
-  authorization_4.q,\
-  authorization_5.q,\
-  authorization_6.q,\
-  authorization_7.q,\
-  ba_table1.q,\
-  ba_table2.q,\
-  ba_table3.q,\
-  ba_table_udfs.q,\
-  binary_table_bincolserde.q,\
-  binary_table_colserde.q,\
-  cluster.q,\
-  columnarserde_create_shortcut.q,\
-  combine2.q,\
-  constant_prop.q,\
-  create_nested_type.q,\
-  create_or_replace_view.q,\
-  create_struct_table.q,\
-  create_union_table.q,\
-  database.q,\
-  database_location.q,\
-  database_properties.q,\
-  describe_database_json.q,\
-  drop_database_removes_partition_dirs.q,\
-  escape1.q,\
-  escape2.q,\
-  exim_00_nonpart_empty.q,\
-  exim_01_nonpart.q,\
-  exim_02_00_part_empty.q,\
-  exim_02_part.q,\
-  exim_03_nonpart_over_compat.q,\
-  exim_04_all_part.q,\
-  exim_04_evolved_parts.q,\
-  exim_05_some_part.q,\
-  exim_06_one_part.q,\
-  exim_07_all_part_over_nonoverlap.q,\
-  exim_08_nonpart_rename.q,\
-  exim_09_part_spec_nonoverlap.q,\
-  exim_10_external_managed.q,\
-  exim_11_managed_external.q,\
-  exim_12_external_location.q,\
-  exim_13_managed_location.q,\
-  exim_14_managed_location_over_existing.q,\
-  exim_15_external_part.q,\
-  exim_16_part_external.q,\
-  exim_17_part_managed.q,\
-  exim_18_part_external.q,\
-  exim_19_00_part_external_location.q,\
-  exim_19_part_external_location.q,\
-  exim_20_part_managed_location.q,\
-  exim_21_export_authsuccess.q,\
-  exim_22_import_exist_authsuccess.q,\
-  exim_23_import_part_authsuccess.q,\
-  exim_24_import_nonexist_authsuccess.q,\
-  global_limit.q,\
-  groupby_complex_types.q,\
-  groupby_complex_types_multi_single_reducer.q,\
-  index_auth.q,\
-  index_auto.q,\
-  index_auto_empty.q,\
-  index_bitmap.q,\
-  index_bitmap1.q,\
-  index_bitmap2.q,\
-  index_bitmap3.q,\
-  index_bitmap_auto.q,\
-  index_bitmap_rc.q,\
-  index_compact.q,\
-  index_compact_1.q,\
-  index_compact_2.q,\
-  index_compact_3.q,\
-  index_stale_partitioned.q,\
-  init_file.q,\
-  input16.q,\
-  input16_cc.q,\
-  input46.q,\
-  input_columnarserde.q,\
-  input_dynamicserde.q,\
-  input_lazyserde.q,\
-  input_testxpath3.q,\
-  input_testxpath4.q,\
-  insert2_overwrite_partitions.q,\
-  insertexternal1.q,\
-  join_thrift.q,\
-  lateral_view.q,\
-  load_binary_data.q,\
-  load_exist_part_authsuccess.q,\
-  load_nonpart_authsuccess.q,\
-  load_part_authsuccess.q,\
-  loadpart_err.q,\
-  lock1.q,\
-  lock2.q,\
-  lock3.q,\
-  lock4.q,\
-  merge_dynamic_partition.q,\
-  multi_insert.q,\
-  multi_insert_move_tasks_share_dependencies.q,\
-  null_column.q,\
-  ppd_clusterby.q,\
-  query_with_semi.q,\
-  rename_column.q,\
-  sample6.q,\
-  sample_islocalmode_hook.q,\
-  set_processor_namespaces.q,\
-  show_tables.q,\
-  source.q,\
-  split_sample.q,\
-  str_to_map.q,\
-  transform1.q,\
-  udaf_collect_set.q,\
-  udaf_context_ngrams.q,\
-  udaf_histogram_numeric.q,\
-  udaf_ngrams.q,\
-  udaf_percentile_approx.q,\
-  udf_array.q,\
-  udf_bitmap_and.q,\
-  udf_bitmap_or.q,\
-  udf_explode.q,\
-  udf_format_number.q,\
-  udf_map.q,\
-  udf_map_keys.q,\
-  udf_map_values.q,\
-  udf_mask.q,\
-  udf_mask_first_n.q,\
-  udf_mask_hash.q,\
-  udf_mask_last_n.q,\
-  udf_mask_show_first_n.q,\
-  udf_mask_show_last_n.q,\
-  udf_max.q,\
-  udf_min.q,\
-  udf_named_struct.q,\
-  udf_percentile.q,\
-  udf_printf.q,\
-  udf_sentences.q,\
-  udf_sort_array.q,\
-  udf_sort_array_by.q,\
-  udf_split.q,\
-  udf_struct.q,\
-  udf_substr.q,\
-  udf_translate.q,\
-  udf_union.q,\
-  udf_xpath.q,\
-  udtf_stack.q,\
-  view.q,\
-  virtual_column.q
+beeline.positive.include=escape_comments.q
 
 minimr.query.negative.files=cluster_tasklog_retrieval.q,\
   file_with_header_footer_negative.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/util/pom.xml
----------------------------------------------------------------------
diff --git a/itests/util/pom.xml b/itests/util/pom.xml
index 6d93dc1..3740af8 100644
--- a/itests/util/pom.xml
+++ b/itests/util/pom.xml
@@ -71,6 +71,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-beeline</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-hbase-handler</artifactId>
       <version>${project.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
----------------------------------------------------------------------
diff --git 
a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java 
b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
index af8ec67..67064b8 100644
--- 
a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
+++ 
b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -406,19 +406,17 @@ public class CliConfigs {
 
   public static class BeeLineConfig extends AbstractCliConfig {
     public BeeLineConfig() {
-      // FIXME: beeline is disabled...
-      super(null);
-      // super(CoreBeeLineDriver.class);
+      super(CoreBeeLineDriver.class);
       try {
         setQueryDir("ql/src/test/queries/clientpositive");
 
-        excludesFrom(testConfigProps, "beeline.positive.exclude");
+        includesFrom(testConfigProps, "beeline.positive.include");
 
-        setResultsDir("ql/src/test/results/clientpositive");
+        setResultsDir("ql/src/test/results/clientpositive/beeline");
         setLogDir("itests/qtest/target/qfile-results/beelinepositive");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_src.sql");
+        setCleanupScript("q_test_cleanup_src.sql");
 
         setHiveConfDir("");
         setClusterType(MiniClusterType.none);
@@ -434,8 +432,6 @@ public class CliConfigs {
       try {
         setQueryDir("accumulo-handler/src/test/queries/positive");
 
-        excludesFrom(testConfigProps, "beeline.positive.exclude");
-
         setResultsDir("accumulo-handler/src/test/results/positive");
         
setLogDir("itests/qtest/target/qfile-results/accumulo-handler/positive");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java
----------------------------------------------------------------------
diff --git 
a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java
 
b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java
index e5144e3..aba1fde 100644
--- 
a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java
+++ 
b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java
@@ -16,104 +16,87 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hive.cli.control;
-//beeline is excluded by default
-//AFAIK contains broken tests
-//and produces compile errors...i'll comment out this whole class for now...
-/*
 
 import static org.junit.Assert.fail;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.*;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.QTestUtil;
 import org.apache.hive.beeline.util.QFileClient;
-import org.apache.hive.service.server.HiveServer2;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-// HIVE-14444: i've dropped this: @RunWith(ConcurrentTestRunner.class)
+
+import java.util.HashMap;
+
 public class CoreBeeLineDriver extends CliAdapter {
   private final String hiveRootDirectory = AbstractCliConfig.HIVE_ROOT;
   private final String queryDirectory;
   private final String logDirectory;
   private final String resultsDirectory;
+  private final String initScript;
+  private final String cleanupScript;
   private boolean overwrite = false;
-  private static String scratchDirectory;
-  private static QTestUtil.QTestSetup miniZKCluster = null;
-
-  private static HiveServer2 hiveServer2;
+  private MiniHS2 miniHS2;
+//  private static QTestUtil.QTestSetup miniZKCluster = null;
 
   public CoreBeeLineDriver(AbstractCliConfig testCliConfig) {
     super(testCliConfig);
     queryDirectory = testCliConfig.getQueryDirectory();
     logDirectory = testCliConfig.getLogDir();
     resultsDirectory = testCliConfig.getResultsDir();
+    initScript = testCliConfig.getInitScript();
+    cleanupScript = testCliConfig.getCleanupScript();
   }
 
   @Override
   @BeforeClass
   public void beforeClass() throws Exception {
-    HiveConf hiveConf = new HiveConf();
-    hiveConf.logVars(System.err);
-    System.err.flush();
-
-    scratchDirectory = hiveConf.getVar(SCRATCHDIR);
-
     String testOutputOverwrite = System.getProperty("test.output.overwrite");
     if (testOutputOverwrite != null && 
"true".equalsIgnoreCase(testOutputOverwrite)) {
       overwrite = true;
     }
 
-    miniZKCluster = new QTestUtil.QTestSetup();
-    miniZKCluster.preTest(hiveConf);
-
-    System.setProperty("hive.zookeeper.quorum",
-        hiveConf.get("hive.zookeeper.quorum"));
-    System.setProperty("hive.zookeeper.client.port",
-        hiveConf.get("hive.zookeeper.client.port"));
-
     String disableserver = System.getProperty("test.service.disable.server");
     if (null != disableserver && disableserver.equalsIgnoreCase("true")) {
-      System.err.println("test.service.disable.server=true "
-        + "Skipping HiveServer2 initialization!");
+      System.err.println("test.service.disable.server=true Skipping 
HiveServer2 initialization!");
       return;
     }
 
-    hiveServer2 = new HiveServer2();
-    hiveServer2.init(hiveConf);
-    System.err.println("Starting HiveServer2...");
-    hiveServer2.start();
-    Thread.sleep(5000);
+    HiveConf hiveConf = new HiveConf();
+    // We do not need Zookeeper at the moment
+    hiveConf.set(HiveConf.ConfVars.HIVE_LOCK_MANAGER.varname,
+        "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager");
+
+    // But if we need later we can enable it with this, or create one ourself
+//    miniZKCluster = new QTestUtil.QTestSetup();
+//    miniZKCluster.preTest(hiveConf);
+
+    hiveConf.logVars(System.err);
+    System.err.flush();
+
+    miniHS2 = new 
MiniHS2.Builder().withConf(hiveConf).cleanupLocalDirOnStartup(true).build();
+
+    miniHS2.start(new HashMap<String, String>());
   }
 
 
   @Override
   @AfterClass
-  public void shutdown() {
-    try {
-      if (hiveServer2 != null) {
-        System.err.println("Stopping HiveServer2...");
-        hiveServer2.stop();
-      }
-    } catch (Throwable t) {
-      t.printStackTrace();
-    }
-
-    if (miniZKCluster != null) {
-      try {
-        miniZKCluster.tearDown();
-      } catch (Exception e) {
-        e.printStackTrace();
-      }
+  public void shutdown() throws Exception {
+    if (miniHS2 != null) {
+      miniHS2.stop();
     }
+//    if (miniZKCluster != null) {
+//      miniZKCluster.tearDown();
+//    }
   }
 
   public void runTest(String qFileName) throws Exception {
-    QFileClient qClient = new QFileClient(new HiveConf(), hiveRootDirectory,
-        queryDirectory, logDirectory, resultsDirectory)
+    QFileClient qClient = new QFileClient(miniHS2.getHiveConf(), 
hiveRootDirectory,
+        queryDirectory, logDirectory, resultsDirectory, initScript, 
cleanupScript)
     .setQFileName(qFileName)
     .setUsername("user")
     .setPassword("password")
-    .setJdbcUrl("jdbc:hive2://localhost:10000")
+    .setJdbcUrl(miniHS2.getJdbcURL())
     .setJdbcDriver("org.apache.hive.jdbc.HiveDriver")
     .setTestDataDirectory(hiveRootDirectory + "/data/files")
     .setTestScriptDirectory(hiveRootDirectory + "/data/scripts");
@@ -150,22 +133,14 @@ public class CoreBeeLineDriver extends CliAdapter {
 
   @Override
   public void setUp() {
-    // TODO Auto-generated method stub
-
   }
 
   @Override
   public void tearDown() {
-    // TODO Auto-generated method stub
-
   }
 
   @Override
   public void runTest(String name, String name2, String absolutePath) throws 
Exception {
     runTest(name2);
   }
-
 }
-
-
-*/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
----------------------------------------------------------------------
diff --git 
a/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
 
b/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
new file mode 100644
index 0000000..98c5d01
--- /dev/null
+++ 
b/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
@@ -0,0 +1,159 @@
+/**
+ * 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.hive.jdbc.miniHS2;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+
+/***
+ * Base class for Hive service
+ * AbstractHiveService.
+ *
+ */
+public abstract class AbstractHiveService {
+  private HiveConf hiveConf = null;
+  private String hostname;
+  private int binaryPort;
+  private int httpPort;
+  private boolean startedHiveService = false;
+  private List<String> addedProperties = new ArrayList<String>();
+
+  public AbstractHiveService(HiveConf hiveConf, String hostname, int 
binaryPort, int httpPort) {
+    this.hiveConf = hiveConf;
+    this.hostname = hostname;
+    this.binaryPort = binaryPort;
+    this.httpPort = httpPort;
+  }
+
+  /**
+   * Get Hive conf
+   * @return
+   */
+  public HiveConf getHiveConf() {
+    return hiveConf;
+  }
+
+  /**
+   * Get config property
+   * @param propertyKey
+   * @return
+   */
+  public String getConfProperty(String propertyKey) {
+    return hiveConf.get(propertyKey);
+  }
+
+  /**
+   * Set config property
+   * @param propertyKey
+   * @param propertyValue
+   */
+  public void setConfProperty(String propertyKey, String propertyValue) {
+    System.setProperty(propertyKey, propertyValue);
+    hiveConf.set(propertyKey, propertyValue);
+    addedProperties.add(propertyKey);
+  }
+
+  /**
+   * Create system properties set by this server instance. This ensures that
+   * the changes made by current test are not impacting subsequent tests.
+   */
+  public void clearProperties() {
+    for (String propKey : addedProperties ) {
+      System.clearProperty(propKey);
+    }
+  }
+
+  /**
+   * Retrieve warehouse directory
+   * @return
+   */
+  public Path getWareHouseDir() {
+    return new Path(hiveConf.getVar(ConfVars.METASTOREWAREHOUSE));
+  }
+
+  public void setWareHouseDir(String wareHouseURI) {
+    verifyNotStarted();
+    System.setProperty(ConfVars.METASTOREWAREHOUSE.varname, wareHouseURI);
+    hiveConf.setVar(ConfVars.METASTOREWAREHOUSE, wareHouseURI);
+  }
+
+  /**
+   * Set service host
+   * @param hostName
+   */
+  public void setHost(String hostName) {
+    this.hostname = hostName;
+  }
+
+  // get service host
+  public String getHost() {
+    return hostname;
+  }
+
+  /**
+   * Set binary service port #
+   * @param portNum
+   */
+  public void setBinaryPort(int portNum) {
+    this.binaryPort = portNum;
+  }
+
+  /**
+   * Set http service port #
+   * @param portNum
+   */
+  public void setHttpPort(int portNum) {
+    this.httpPort = portNum;
+  }
+
+  // Get binary service port #
+  public int getBinaryPort() {
+    return binaryPort;
+  }
+
+  // Get http service port #
+  public int getHttpPort() {
+    return httpPort;
+  }
+
+  public boolean isStarted() {
+    return startedHiveService;
+  }
+
+  public void setStarted(boolean hiveServiceStatus) {
+    this.startedHiveService =  hiveServiceStatus;
+  }
+
+  protected void verifyStarted() {
+    if (!isStarted()) {
+      throw new IllegalStateException("HiveServer2 is not running");
+    }
+  }
+
+  protected void verifyNotStarted() {
+    if (isStarted()) {
+      throw new IllegalStateException("HiveServer2 already running");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8de307/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git 
a/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java 
b/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
new file mode 100644
index 0000000..71f9640
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -0,0 +1,556 @@
+/**
+ * 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.hive.jdbc.miniHS2;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.LlapItUtils;
+import org.apache.hadoop.hive.llap.daemon.MiniLlapCluster;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper;
+import org.apache.hadoop.hive.shims.HadoopShims.MiniDFSShim;
+import org.apache.hadoop.hive.shims.HadoopShims.MiniMrShim;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hive.service.Service;
+import org.apache.hive.service.cli.CLIServiceClient;
+import org.apache.hive.service.cli.SessionHandle;
+import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService;
+import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
+import org.apache.hive.service.cli.thrift.ThriftHttpCLIService;
+import org.apache.hive.service.server.HiveServer2;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MiniHS2 extends AbstractHiveService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MiniHS2.class);
+
+  public static final String HS2_BINARY_MODE = "binary";
+  public static final String HS2_HTTP_MODE = "http";
+  private static final String driverName = "org.apache.hive.jdbc.HiveDriver";
+  private static final FsPermission FULL_PERM = new FsPermission((short)00777);
+  private static final FsPermission WRITE_ALL_PERM = new 
FsPermission((short)00733);
+  private static final String tmpDir = System.getProperty("test.tmp.dir");
+  private HiveServer2 hiveServer2 = null;
+  private final File baseDir;
+  private final Path baseFsDir;
+  private MiniMrShim mr;
+  private MiniDFSShim dfs;
+  private MiniLlapCluster llapCluster = null;
+  private final FileSystem localFS;
+  private boolean useMiniKdc = false;
+  private final String serverPrincipal;
+  private final boolean isMetastoreRemote;
+  private final boolean cleanupLocalDirOnStartup;
+  private MiniClusterType miniClusterType = MiniClusterType.LOCALFS_ONLY;
+
+  public enum MiniClusterType {
+    MR,
+    TEZ,
+    LLAP,
+    LOCALFS_ONLY;
+  }
+
+  public static class Builder {
+    private HiveConf hiveConf = new HiveConf();
+    private MiniClusterType miniClusterType = MiniClusterType.LOCALFS_ONLY;
+    private boolean useMiniKdc = false;
+    private String serverPrincipal;
+    private String serverKeytab;
+    private boolean isHTTPTransMode = false;
+    private boolean isMetastoreRemote;
+    private boolean usePortsFromConf = false;
+    private String authType = "KERBEROS";
+    private boolean isHA = false;
+    private boolean cleanupLocalDirOnStartup = true;
+
+    public Builder() {
+    }
+
+    public Builder withMiniMR() {
+      this.miniClusterType = MiniClusterType.MR;
+      return this;
+    }
+
+    public Builder withMiniKdc(String serverPrincipal, String serverKeytab) {
+      this.useMiniKdc = true;
+      this.serverPrincipal = serverPrincipal;
+      this.serverKeytab = serverKeytab;
+      return this;
+    }
+
+    public Builder withAuthenticationType(String authType) {
+      this.authType = authType;
+      return this;
+    }
+
+    public Builder withRemoteMetastore() {
+      this.isMetastoreRemote = true;
+      return this;
+    }
+
+    public Builder withConf(HiveConf hiveConf) {
+      this.hiveConf = hiveConf;
+      return this;
+    }
+
+    public Builder withHA() {
+      this.isHA = true;
+      return this;
+    }
+
+    /**
+     * Start HS2 with HTTP transport mode, default is binary mode
+     * @return this Builder
+     */
+    public Builder withHTTPTransport(){
+      this.isHTTPTransMode = true;
+      return this;
+    }
+
+    public Builder cleanupLocalDirOnStartup(boolean val) {
+      this.cleanupLocalDirOnStartup = val;
+      return this;
+    }
+
+    public MiniHS2 build() throws Exception {
+      if (miniClusterType == MiniClusterType.MR && useMiniKdc) {
+        throw new IOException("Can't create secure miniMr ... yet");
+      }
+      if (isHTTPTransMode) {
+        hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_HTTP_MODE);
+      } else {
+        hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_BINARY_MODE);
+      }
+      return new MiniHS2(hiveConf, miniClusterType, useMiniKdc, 
serverPrincipal, serverKeytab,
+          isMetastoreRemote, usePortsFromConf, authType, isHA, 
cleanupLocalDirOnStartup);
+    }
+  }
+
+  public MiniMrShim getMr() {
+    return mr;
+  }
+
+  public void setMr(MiniMrShim mr) {
+    this.mr = mr;
+  }
+
+  public MiniDFSShim getDfs() {
+    return dfs;
+  }
+
+  public void setDfs(MiniDFSShim dfs) {
+    this.dfs = dfs;
+  }
+
+  public FileSystem getLocalFS() {
+    return localFS;
+  }
+
+  public MiniClusterType getMiniClusterType() {
+    return miniClusterType;
+  }
+
+  public void setMiniClusterType(MiniClusterType miniClusterType) {
+    this.miniClusterType = miniClusterType;
+  }
+
+  public boolean isUseMiniKdc() {
+    return useMiniKdc;
+  }
+
+  private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean 
useMiniKdc,
+      String serverPrincipal, String serverKeytab, boolean isMetastoreRemote,
+      boolean usePortsFromConf, String authType, boolean isHA, boolean 
cleanupLocalDirOnStartup) throws Exception {
+    // Always use localhost for hostname as some tests like SSL CN validation 
ones
+    // are tied to localhost being present in the certificate name
+    super(
+        hiveConf,
+        "localhost",
+        (usePortsFromConf ? 
hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT) : MetaStoreUtils
+            .findFreePort()),
+        (usePortsFromConf ? 
hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT) : 
MetaStoreUtils
+            .findFreePort()));
+    hiveConf.setLongVar(ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS, 3l);
+    
hiveConf.setTimeVar(ConfVars.HIVE_SERVER2_SLEEP_INTERVAL_BETWEEN_START_ATTEMPTS,
 10,
+        TimeUnit.SECONDS);
+    this.miniClusterType = miniClusterType;
+    this.useMiniKdc = useMiniKdc;
+    this.serverPrincipal = serverPrincipal;
+    this.isMetastoreRemote = isMetastoreRemote;
+    this.cleanupLocalDirOnStartup = cleanupLocalDirOnStartup;
+    baseDir = getBaseDir();
+    localFS = FileSystem.getLocal(hiveConf);
+    FileSystem fs;
+
+    if (miniClusterType != MiniClusterType.LOCALFS_ONLY) {
+      // Initialize dfs
+      dfs = ShimLoader.getHadoopShims().getMiniDfs(hiveConf, 4, true, null, 
isHA);
+      fs = dfs.getFileSystem();
+      String uriString = fs.getUri().toString();
+
+      // Initialize the execution engine based on cluster type
+      switch (miniClusterType) {
+      case TEZ:
+        // Change the engine to tez
+        hiveConf.setVar(ConfVars.HIVE_EXECUTION_ENGINE, "tez");
+        // TODO: This should be making use of confDir to load configs setup 
for Tez, etc.
+        mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 2, 
uriString, false);
+        break;
+      case LLAP:
+        if (usePortsFromConf) {
+          hiveConf.setBoolean("minillap.usePortsFromConf", true);
+        }
+        llapCluster = LlapItUtils.startAndGetMiniLlapCluster(hiveConf, null, 
null);
+
+        mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 2, 
uriString, true);
+        break;
+      case MR:
+        mr = ShimLoader.getHadoopShims().getMiniMrCluster(hiveConf, 2, 
uriString, 1);
+        break;
+      default:
+        throw new IllegalArgumentException("Unsupported cluster type " + mr);
+      }
+      // store the config in system properties
+      mr.setupConfiguration(getHiveConf());
+      baseFsDir = new Path(new Path(fs.getUri()), "/base");
+    } else {
+      // This is FS only mode, just initialize the dfs root directory.
+      fs = FileSystem.getLocal(hiveConf);
+      baseFsDir = new Path("file://" + baseDir.toURI().getPath());
+
+      if (cleanupLocalDirOnStartup) {
+        // Cleanup baseFsDir since it can be shared across tests.
+        LOG.info("Attempting to cleanup baseFsDir: {} while setting up 
MiniHS2", baseDir);
+        Preconditions.checkState(baseFsDir.depth() >= 3); // Avoid "/tmp", 
directories closer to "/"
+        fs.delete(baseFsDir, true);
+      }
+    }
+    if (useMiniKdc) {
+      hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL, 
serverPrincipal);
+      hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB, serverKeytab);
+      hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, authType);
+    }
+    String metaStoreURL =
+        "jdbc:derby:;databaseName=" + baseDir.getAbsolutePath() + 
File.separator
+            + "test_metastore;create=true";
+
+    fs.mkdirs(baseFsDir);
+    Path wareHouseDir = new Path(baseFsDir, "warehouse");
+    // Create warehouse with 777, so that user impersonation has no issues.
+    FileSystem.mkdirs(fs, wareHouseDir, FULL_PERM);
+
+    fs.mkdirs(wareHouseDir);
+    setWareHouseDir(wareHouseDir.toString());
+    System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, 
metaStoreURL);
+    hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, metaStoreURL);
+    if (!usePortsFromConf) {
+      // reassign a new port, just in case if one of the MR services grabbed 
the last one
+      setBinaryPort(MetaStoreUtils.findFreePort());
+    }
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, getHost());
+    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, getBinaryPort());
+    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, getHttpPort());
+
+    Path scratchDir = new Path(baseFsDir, "scratch");
+    // Create root scratchdir with write all, so that user impersonation has 
no issues.
+    Utilities.createDirsWithPermission(hiveConf, scratchDir, WRITE_ALL_PERM, 
true);
+    System.setProperty(HiveConf.ConfVars.SCRATCHDIR.varname, 
scratchDir.toString());
+    hiveConf.setVar(ConfVars.SCRATCHDIR, scratchDir.toString());
+
+    String localScratchDir = baseDir.getPath() + File.separator + "scratch";
+    System.setProperty(HiveConf.ConfVars.LOCALSCRATCHDIR.varname, 
localScratchDir);
+    hiveConf.setVar(ConfVars.LOCALSCRATCHDIR, localScratchDir);
+  }
+
+  public MiniHS2(HiveConf hiveConf) throws Exception {
+    this(hiveConf, MiniClusterType.LOCALFS_ONLY);
+  }
+
+  public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType) throws 
Exception {
+    this(hiveConf, clusterType, false);
+  }
+
+  public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType,
+      boolean usePortsFromConf) throws Exception {
+    this(hiveConf, clusterType, false, null, null, false, usePortsFromConf,
+      "KERBEROS", false, true);
+  }
+
+  public void start(Map<String, String> confOverlay) throws Exception {
+    if (isMetastoreRemote) {
+      int metaStorePort = MetaStoreUtils.findFreePort();
+      getHiveConf().setVar(ConfVars.METASTOREURIS, "thrift://localhost:" + 
metaStorePort);
+      MetaStoreUtils.startMetaStore(metaStorePort,
+      ShimLoader.getHadoopThriftAuthBridge(), getHiveConf());
+    }
+
+    hiveServer2 = new HiveServer2();
+    // Set confOverlay parameters
+    for (Map.Entry<String, String> entry : confOverlay.entrySet()) {
+      setConfProperty(entry.getKey(), entry.getValue());
+    }
+    hiveServer2.init(getHiveConf());
+    hiveServer2.start();
+    waitForStartup();
+    setStarted(true);
+  }
+
+  public void stop() {
+    verifyStarted();
+    // Currently there is no way to stop the MetaStore service. It will be 
stopped when the
+    // test JVM exits. This is how other tests are also using MetaStore server.
+
+    hiveServer2.stop();
+    setStarted(false);
+    try {
+      if (llapCluster != null) {
+        llapCluster.stop();
+      }
+      if (mr != null) {
+        mr.shutdown();
+        mr = null;
+      }
+      if (dfs != null) {
+        dfs.shutdown();
+        dfs = null;
+      }
+    } catch (IOException e) {
+      // Ignore errors cleaning up miniMR
+    }
+  }
+
+  public void cleanup() {
+    FileUtils.deleteQuietly(baseDir);
+  }
+
+
+  public CLIServiceClient getServiceClient() {
+    verifyStarted();
+    return getServiceClientInternal();
+  }
+
+  public HiveConf getServerConf() {
+    if (hiveServer2 != null) {
+      return hiveServer2.getHiveConf();
+    }
+    return null;
+  }
+
+  public CLIServiceClient getServiceClientInternal() {
+    for (Service service : hiveServer2.getServices()) {
+      if (service instanceof ThriftBinaryCLIService) {
+        return new ThriftCLIServiceClient((ThriftBinaryCLIService) service);
+      }
+      if (service instanceof ThriftHttpCLIService) {
+        return new ThriftCLIServiceClient((ThriftHttpCLIService) service);
+      }
+    }
+    throw new IllegalStateException("HiveServer2 not running Thrift service");
+  }
+
+  /**
+   * return connection URL for this server instance
+   * @return
+   * @throws Exception
+   */
+  public String getJdbcURL() throws Exception {
+    return getJdbcURL("default");
+  }
+
+  /**
+   * return connection URL for this server instance
+   * @param dbName - DB name to be included in the URL
+   * @return
+   * @throws Exception
+   */
+  public String getJdbcURL(String dbName) throws Exception {
+    return getJdbcURL(dbName, "");
+  }
+
+  /**
+   * return connection URL for this server instance
+   * @param dbName - DB name to be included in the URL
+   * @param sessionConfExt - Addional string to be appended to sessionConf 
part of url
+   * @return
+   * @throws Exception
+   */
+  public String getJdbcURL(String dbName, String sessionConfExt) throws 
Exception {
+    return getJdbcURL(dbName, sessionConfExt, "");
+  }
+
+  /**
+   * return connection URL for this server instance
+   * @param dbName - DB name to be included in the URL
+   * @param sessionConfExt - Addional string to be appended to sessionConf 
part of url
+   * @param hiveConfExt - Additional string to be appended to HiveConf part of 
url (excluding the ?)
+   * @return
+   * @throws Exception
+   */
+  public String getJdbcURL(String dbName, String sessionConfExt, String 
hiveConfExt)
+      throws Exception {
+    sessionConfExt = (sessionConfExt == null ? "" : sessionConfExt);
+    hiveConfExt = (hiveConfExt == null ? "" : hiveConfExt);
+    // Strip the leading ";" if provided
+    // (this is the assumption with which we're going to start configuring 
sessionConfExt)
+    if (sessionConfExt.startsWith(";")) {
+      sessionConfExt = sessionConfExt.substring(1);
+    }
+    if (isUseMiniKdc()) {
+      sessionConfExt = "principal=" + serverPrincipal + ";" + sessionConfExt;
+    }
+    if (isHttpTransportMode()) {
+      sessionConfExt = "transportMode=http;httpPath=cliservice" + ";" + 
sessionConfExt;
+    }
+    String baseJdbcURL;
+    if (isDynamicServiceDiscovery()) {
+      sessionConfExt =
+          "serviceDiscoveryMode=zooKeeper;zooKeeperNamespace="
+              + 
getServerConf().getVar(HiveConf.ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE) + ";"
+              + sessionConfExt;
+      baseJdbcURL = getZKBaseJdbcURL();
+    } else {
+      baseJdbcURL = getBaseJdbcURL();
+    }
+
+    baseJdbcURL = baseJdbcURL + dbName;
+    if (!sessionConfExt.isEmpty()) {
+      baseJdbcURL = baseJdbcURL + ";" + sessionConfExt;
+    }
+    if ((hiveConfExt != null) && (!hiveConfExt.trim().isEmpty())) {
+      baseJdbcURL = baseJdbcURL + "?" + hiveConfExt;
+    }
+    return baseJdbcURL;
+  }
+
+  /**
+   * Build base JDBC URL
+   * @return
+   */
+  public String getBaseJdbcURL() {
+    if(isHttpTransportMode()) {
+      return "jdbc:hive2://" + getHost() + ":" + getHttpPort() + "/";
+    }
+    else {
+      return "jdbc:hive2://" + getHost() + ":" + getBinaryPort() + "/";
+    }
+  }
+
+  /**
+   * Build zk base JDBC URL
+   * @return
+   */
+  private String getZKBaseJdbcURL() throws Exception {
+    HiveConf hiveConf = getServerConf();
+    if (hiveConf != null) {
+      String zkEnsemble =  ZooKeeperHiveHelper.getQuorumServers(hiveConf);
+      return "jdbc:hive2://" + zkEnsemble + "/";
+    }
+    throw new Exception("Server's HiveConf is null. Unable to read ZooKeeper 
configs.");
+  }
+
+  private boolean isHttpTransportMode() {
+    String transportMode = 
getConfProperty(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname);
+    return transportMode != null && 
(transportMode.equalsIgnoreCase(HS2_HTTP_MODE));
+  }
+
+  private boolean isDynamicServiceDiscovery() throws Exception {
+    HiveConf hiveConf = getServerConf();
+    if (hiveConf == null) {
+      throw new Exception("Server's HiveConf is null. Unable to read ZooKeeper 
configs.");
+    }
+    if 
(hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY)) {
+      return true;
+    }
+    return false;
+  }
+
+  public static String getJdbcDriverName() {
+    return driverName;
+  }
+
+  public MiniMrShim getMR() {
+    return mr;
+  }
+
+  public MiniDFSShim getDFS() {
+    return dfs;
+  }
+
+  private void waitForStartup() throws Exception {
+    int waitTime = 0;
+    long startupTimeout = 1000L * 1000L;
+    CLIServiceClient hs2Client = getServiceClientInternal();
+    SessionHandle sessionHandle = null;
+    do {
+      Thread.sleep(500L);
+      waitTime += 500L;
+      if (waitTime > startupTimeout) {
+        throw new TimeoutException("Couldn't access new HiveServer2: " + 
getJdbcURL());
+      }
+      try {
+        Map <String, String> sessionConf = new HashMap<String, String>();
+        /**
+        if (isUseMiniKdc()) {
+          getMiniKdc().loginUser(getMiniKdc().getDefaultUserPrincipal());
+          sessionConf.put("principal", serverPrincipal);
+        }
+         */
+        sessionHandle = hs2Client.openSession("foo", "bar", sessionConf);
+      } catch (Exception e) {
+        // service not started yet
+        continue;
+      }
+      hs2Client.closeSession(sessionHandle);
+      break;
+    } while (true);
+  }
+
+  public Service.STATE getState() {
+    return hiveServer2.getServiceState();
+  }
+
+  static File getBaseDir() {
+    File baseDir = new File(tmpDir + "/local_base");
+    return baseDir;
+  }
+
+  public static void cleanupLocalDir() throws IOException {
+    File baseDir = getBaseDir();
+    try {
+      org.apache.hadoop.hive.common.FileUtils.deleteDirectory(baseDir);
+    } catch (FileNotFoundException e) {
+      // Ignore. Safe if it does not exist.
+    }
+  }
+}

Reply via email to