You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2017/02/24 01:08:34 UTC

[02/50] [abbrv] hive git commit: HIVE-15900 : Beeline prints tez job progress in stdout instead of stderr (Thejas Nair, reviewed by Daniel Dai, Anishek Agarwal)

HIVE-15900 : Beeline prints tez job progress in stdout instead of stderr (Thejas Nair, reviewed by Daniel Dai, Anishek Agarwal)


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

Branch: refs/heads/hive-14535
Commit: 60a36d124d437bea123c94d50ccb171e24ff2f3d
Parents: 6e652a3
Author: Thejas M Nair <th...@hortonworks.com>
Authored: Thu Feb 16 11:43:45 2017 -0800
Committer: Thejas M Nair <th...@hortonworks.com>
Committed: Thu Feb 16 11:43:52 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/Commands.java  |   2 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   2 +
 .../hive/beeline/TestBeeLineWithArgs.java       | 169 +++++++++++++------
 .../TestOperationLoggingAPIWithTez.java         |   2 -
 4 files changed, 118 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/60a36d12/beeline/src/java/org/apache/hive/beeline/Commands.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java
index 99db643..6a3ad42 100644
--- a/beeline/src/java/org/apache/hive/beeline/Commands.java
+++ b/beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -985,7 +985,7 @@ public class Commands {
             logThread.start();
             if (stmnt instanceof HiveStatement) {
               ((HiveStatement) stmnt).setInPlaceUpdateStream(
-                  new BeelineInPlaceUpdateStream(beeLine.getOutputStream())
+                  new BeelineInPlaceUpdateStream(beeLine.getErrorStream())
               );
             }
             hasResults = stmnt.execute(sql);

http://git-wip-us.apache.org/repos/asf/hive/blob/60a36d12/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
index e641253..71f9640 100644
--- 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
@@ -222,6 +222,8 @@ public class MiniHS2 extends AbstractHiveService {
       // 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;

http://git-wip-us.apache.org/repos/asf/hive/blob/60a36d12/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
index 9e99a91..8fe3789 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
@@ -36,11 +36,16 @@ import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.jdbc.Utils;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -51,11 +56,15 @@ import org.junit.Test;
  *
  */
 public class TestBeeLineWithArgs {
+  private enum OutStream {
+    ERR, OUT
+  };
+
   // Default location of HiveServer2
   private static final String tableName = "TestBeelineTable1";
   private static final String tableComment = "Test table comment";
-
   private static MiniHS2 miniHS2;
+  private static final String userName = System.getProperty("user.name");
 
   private List<String> getBaseArgs(String jdbcUrl) {
     List<String> argList = new ArrayList<String>(8);
@@ -63,6 +72,8 @@ public class TestBeeLineWithArgs {
     argList.add(BeeLine.BEELINE_DEFAULT_JDBC_DRIVER);
     argList.add("-u");
     argList.add(jdbcUrl);
+    argList.add("-n");
+    argList.add(userName);
     return argList;
   }
   /**
@@ -71,11 +82,15 @@ public class TestBeeLineWithArgs {
   @BeforeClass
   public static void preTests() throws Exception {
     HiveConf hiveConf = new HiveConf();
-    // Set to non-zk lock manager to prevent HS2 from trying to connect
-    hiveConf.setVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER, "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager");
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER,
+        "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager");
     hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false);
-    miniHS2 = new MiniHS2(hiveConf);
-    miniHS2.start(new HashMap<String,  String>());
+    hiveConf.set(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname, "verbose");
+    miniHS2 = new MiniHS2(hiveConf, MiniClusterType.TEZ);
+
+    Map<String, String> confOverlay = new HashMap<String, String>();
+    miniHS2.start(confOverlay);
+
     createTable();
   }
 
@@ -86,7 +101,8 @@ public class TestBeeLineWithArgs {
    */
   private static void createTable() throws ClassNotFoundException, SQLException {
     Class.forName(BeeLine.BEELINE_DEFAULT_JDBC_DRIVER);
-    Connection con = DriverManager.getConnection(miniHS2.getBaseJdbcURL(),"", "");
+    Connection con = DriverManager.getConnection(miniHS2.getBaseJdbcURL(),
+        userName , "");
 
     assertNotNull("Connection is null", con);
     assertFalse("Connection should not be closed", con.isClosed());
@@ -128,16 +144,27 @@ public class TestBeeLineWithArgs {
 
   /**
    * Execute a script with "beeline -f or -i"
-   *
+   * @param argList List of arguments for beeline
+   * @param inputStream input stream if any
+   * @param streamType if output from STDERR or STDOUT needs to be returned
    * @return The stderr and stdout from running the script
+   * @throws Throwable
    */
-  private String testCommandLineScript(List<String> argList, InputStream inputStream)
+  private String testCommandLineScript(List<String> argList, InputStream inputStream, OutStream streamType)
       throws Throwable {
     BeeLine beeLine = new BeeLine();
     ByteArrayOutputStream os = new ByteArrayOutputStream();
     PrintStream beelineOutputStream = new PrintStream(os);
-    beeLine.setOutputStream(beelineOutputStream);
-    beeLine.setErrorStream(beelineOutputStream);
+    switch (streamType) {
+    case OUT:
+      beeLine.setOutputStream(beelineOutputStream);
+      break;
+    case ERR:
+      beeLine.setErrorStream(beelineOutputStream);
+      break;
+    default:
+      throw new RuntimeException("Unexpected outstream type " + streamType);
+    }
     String[] args = argList.toArray(new String[argList.size()]);
     beeLine.begin(args, inputStream);
     String output = os.toString("UTF8");
@@ -147,33 +174,53 @@ public class TestBeeLineWithArgs {
   }
 
   /**
+   * Attempt to execute a simple script file with the -f and -i option to
+   * BeeLine to test for presence of an expected pattern in the output (stdout
+   * or stderr), fail if not found. Print PASSED or FAILED
+   * 
+   * @param expectedRegex
+   *          Text to look for in command output (stdout)
+   * @param shouldMatch
+   *          true if the pattern should be found, false if it should not
+   * @throws Exception
+   *           on command execution error
+   */
+  private void testScriptFile(String scriptText, String expectedRegex,
+      boolean shouldMatch, List<String> argList) throws Throwable {
+    testScriptFile(scriptText, expectedRegex, shouldMatch, argList, true, true, OutStream.OUT);
+  }
+
+  /**
    * Attempt to execute a simple script file with the -f and -i option
    * to BeeLine to test for presence of an expected pattern
    * in the output (stdout or stderr), fail if not found.
    * Print PASSED or FAILED
-   * @param expectedPattern Text to look for in command output/error
+   * @param expectedRegex Text to look for in command output (stdout)
    * @param shouldMatch true if the pattern should be found, false if it should not
-   * @throws Exception on command execution error
+   * @param argList arguments
+   * @param outType output stream type
+   * @throws Throwable
    */
-  private void testScriptFile(String scriptText, String expectedPattern,
-      boolean shouldMatch, List<String> argList) throws Throwable {
-    testScriptFile(scriptText, expectedPattern, shouldMatch, argList, true, true);
+  private void testScriptFile(String scriptText, String expectedRegex,
+      boolean shouldMatch, List<String> argList, OutStream outType) throws Throwable {
+    testScriptFile(scriptText, expectedRegex, shouldMatch, argList, true, true, outType);
   }
-
+  
   /**
    * Attempt to execute a simple script file with the -f or -i option
    * to BeeLine (or both) to  test for presence of an expected pattern
    * in the output (stdout or stderr), fail if not found.
    * Print PASSED or FAILED
-   * @param expectedPattern Text to look for in command output/error
+   * @param expectedRegex Text to look for in command output/error
    * @param shouldMatch true if the pattern should be found, false if it should not
    * @param testScript Whether we should test -f
    * @param testInit Whether we should test -i
+   * @param streamType Whether match should be done against STDERR or STDOUT
    * @throws Exception on command execution error
    */
-  private void testScriptFile(String scriptText, String expectedPattern,
+  private void testScriptFile(String scriptText, String expectedRegex,
       boolean shouldMatch, List<String> argList,
-      boolean testScript, boolean testInit) throws Throwable {
+      boolean testScript, boolean testInit, OutStream streamType) throws Throwable {
 
     // Put the script content in a temp file
     File scriptFile = File.createTempFile(this.getClass().getSimpleName(), "temp");
@@ -183,17 +230,20 @@ public class TestBeeLineWithArgs {
     os.print(scriptText);
     os.close();
 
+    Pattern expectedPattern = Pattern.compile(".*" + expectedRegex + ".*", Pattern.DOTALL);
     if (testScript) {
       List<String> copy = new ArrayList<String>(argList);
       copy.add("-f");
       copy.add(scriptFile.getAbsolutePath());
 
-      String output = testCommandLineScript(copy, null);
-      boolean matches = output.contains(expectedPattern);
+      String output = testCommandLineScript(copy, null, streamType);
+
+      Matcher m = expectedPattern.matcher(output);
+      boolean matches = m.matches();
       if (shouldMatch != matches) {
         //failed
         fail("Output" + output + " should" +  (shouldMatch ? "" : " not") +
-            " contain " + expectedPattern);
+            " contain " + expectedRegex);
       }
     }
 
@@ -205,12 +255,13 @@ public class TestBeeLineWithArgs {
       copy.add("-i");
       copy.add(scriptFile.getAbsolutePath());
 
-      String output = testCommandLineScript(copy, new StringBufferInputStream("!quit\n"));
-      boolean matches = output.contains(expectedPattern);
+      String output = testCommandLineScript(copy, new StringBufferInputStream("!quit\n"), streamType);
+      Matcher m = expectedPattern.matcher(output);
+      boolean matches = m.matches();
       if (shouldMatch != matches) {
         //failed
         fail("Output" + output + " should" +  (shouldMatch ? "" : " not") +
-            " contain " + expectedPattern);
+            " contain " + expectedRegex);
       }
     }
     scriptFile.delete();
@@ -225,14 +276,15 @@ public class TestBeeLineWithArgs {
    * @param shouldMatch true if the pattern should be found, false if it should not
    * @throws Exception on command execution error
    */
+
   private void testCommandEnclosedQuery(String enclosedQuery, String expectedPattern,
-      boolean shouldMatch, List<String> argList) throws Throwable {
+      boolean shouldMatch, List<String> argList, OutStream out) throws Throwable {
 
     List<String> copy = new ArrayList<String>(argList);
     copy.add("-e");
     copy.add(enclosedQuery);
 
-    String output = testCommandLineScript(copy, null);
+    String output = testCommandLineScript(copy, null, out);
     boolean matches = output.contains(expectedPattern);
     if (shouldMatch != matches) {
       //failed
@@ -290,7 +342,7 @@ public class TestBeeLineWithArgs {
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--hivevar");
     argList.add("DUMMY_TBL=dummy");
-    final String SCRIPT_TEXT = "create table ${DUMMY_TBL} (d int);\nshow tables;\n";
+    final String SCRIPT_TEXT = "create table ${DUMMY_TBL} (d int);\nshow tables;\n drop table  ${DUMMY_TBL};";
     final String EXPECTED_PATTERN = "dummy";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -300,7 +352,8 @@ public class TestBeeLineWithArgs {
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--hiveconf");
     argList.add("test.hive.table.name=dummy");
-    final String SCRIPT_TEXT = "create table ${hiveconf:test.hive.table.name} (d int);\nshow tables;\n";
+    final String SCRIPT_TEXT = "create table ${hiveconf:test.hive.table.name} (d int);\nshow tables;\n"
+        + " drop table ${hiveconf:test.hive.table.name};\n";
     final String EXPECTED_PATTERN = "dummy";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -327,7 +380,9 @@ public class TestBeeLineWithArgs {
     argList.add("--hiveconf");
     argList.add("COLUMN_TYPE=int");
 
-    final String SCRIPT_TEXT = "${COMMAND} ${OBJECT} ${TABLE_NAME} (${hiveconf:COLUMN_NAME} ${hiveconf:COLUMN_TYPE});\nshow tables;\n";
+    final String SCRIPT_TEXT = "${COMMAND} ${OBJECT} ${TABLE_NAME} "
+        + "(${hiveconf:COLUMN_NAME} ${hiveconf:COLUMN_TYPE});"
+        + "\nshow tables;\n drop ${OBJECT} ${TABLE_NAME};\n";
     final String EXPECTED_PATTERN = "dummy2";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -348,7 +403,8 @@ public class TestBeeLineWithArgs {
   @Test
   public void testTabInScriptFile() throws Throwable {
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
-    final String SCRIPT_TEXT = "CREATE\tTABLE IF NOT EXISTS testTabInScriptFile\n(id\tint);\nSHOW TABLES;";
+    final String SCRIPT_TEXT = "CREATE\tTABLE IF NOT EXISTS testTabInScriptFile\n(id\tint);\nSHOW TABLES;"
+        + "\ndrop table testTabInScriptFile";
     final String EXPECTED_PATTERN = "testTabInScriptFile";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -537,7 +593,7 @@ public class TestBeeLineWithArgs {
     argList.add("--outputformat=tsv");
 
     final String EXPECTED_PATTERN = "Format tsv is deprecated, please use tsv2";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, OutStream.ERR);
   }
 
   /**
@@ -551,7 +607,7 @@ public class TestBeeLineWithArgs {
     argList.add("--outputformat=csv");
 
     final String EXPECTED_PATTERN = "Format csv is deprecated, please use csv2";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, true, OutStream.ERR);
   }
 
   /**
@@ -612,7 +668,7 @@ public class TestBeeLineWithArgs {
     argList.add(scriptFile.getAbsolutePath());
 
     try {
-      String output = testCommandLineScript(argList, null);
+      String output = testCommandLineScript(argList, null, OutStream.OUT);
       if (output.contains(EXPECTED_PATTERN)) {
         fail("Output: " + output +  " Negative pattern: " + EXPECTED_PATTERN);
       }
@@ -651,7 +707,7 @@ public class TestBeeLineWithArgs {
   @Test
   public void testHiveVarSubstitution() throws Throwable {
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL() + "#D_TBL=dummy_t");
-    final String SCRIPT_TEXT = "create table ${D_TBL} (d int);\nshow tables;\n";
+    final String SCRIPT_TEXT = "create table ${D_TBL} (d int);\nshow tables;\ndrop  table ${D_TBL};\n";
     final String EXPECTED_PATTERN = "dummy_t";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -665,7 +721,7 @@ public class TestBeeLineWithArgs {
     // Set to non-zk lock manager to avoid trying to connect to zookeeper
     final String SCRIPT_TEXT =
         "set hive.lock.manager=org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager;\n" +
-        "create table ${DUMMY_TBL} (d int);\nshow tables;\n";
+        "create table ${DUMMY_TBL} (d int);\nshow tables;\n drop table ${DUMMY_TBL};\n";
     final String EXPECTED_PATTERN = "embedded_table";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -678,8 +734,10 @@ public class TestBeeLineWithArgs {
   public void testQueryProgress() throws Throwable {
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "select count(*) from " + tableName + ";\n";
-    final String EXPECTED_PATTERN = "number of splits";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
+    // Check for part of log message as well as part of progress information
+    final String EXPECTED_PATTERN = "Number of reducers determined to be.*ELAPSED TIME";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()),
+        OutStream.ERR);
   }
 
   /**
@@ -692,8 +750,10 @@ public class TestBeeLineWithArgs {
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "set hive.exec.parallel = true;\n" +
         "select count(*) from " + tableName + ";\n";
-    final String EXPECTED_PATTERN = "number of splits";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
+    // Check for part of log message as well as part of progress information
+    final String EXPECTED_PATTERN = "Number of reducers determined to be.*ELAPSED TIME";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()),
+        OutStream.ERR);
   }
 
   /**
@@ -706,7 +766,7 @@ public class TestBeeLineWithArgs {
         "!set silent true\n" +
         "select count(*) from " + tableName + ";\n";
     final String EXPECTED_PATTERN = "Executing command";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, false, getBaseArgs(miniHS2.getBaseJdbcURL()));
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, false, getBaseArgs(miniHS2.getBaseJdbcURL()), OutStream.ERR);
   }
 
   @Test
@@ -727,10 +787,10 @@ public class TestBeeLineWithArgs {
         +"(key int);show tables; --multicommands in one line";
     final String EXPECTED_PATTERN = " multicmdtbl ";
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
-    testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList);
+    testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList, OutStream.OUT);
 
     final String QUERY_TEXT_DROP = "drop table multiCmdTbl;show tables;";
-    testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+    testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList, OutStream.OUT);
   }
 
   @Test
@@ -765,10 +825,10 @@ public class TestBeeLineWithArgs {
         + " TERMINATED BY '\\n';show tables;";
     final String EXPECTED_PATTERN = " multicmdtbl ";
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
-    testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList);
+    testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList, OutStream.OUT);
 
     final String QUERY_TEXT_DROP = "drop table multiCmdTbl;show tables;";
-    testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+    testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList, OutStream.OUT);
   }
 
   @Test
@@ -779,9 +839,10 @@ public class TestBeeLineWithArgs {
     final String SCRIPT_TEXT = "set hive.lock.manager=org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager;\n"
         + "set hive.compute.query.using.stats=false;\n"
         + "create table if not exists embeddedBeelineOutputs(d int);\n"
-        + "set a=1;\nselect count(*) from embeddedBeelineOutputs;\n";
+        + "set a=1;\nselect count(*) from embeddedBeelineOutputs;\n"
+        + "drop table embeddedBeelineOutputs;\n";
     final String EXPECTED_PATTERN = "Stage-1 map =";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, OutStream.ERR);
   }
 
   @Test
@@ -806,7 +867,7 @@ public class TestBeeLineWithArgs {
     argList.add(BeeLine.BEELINE_DEFAULT_JDBC_DRIVER);
 
     final String SCRIPT_TEXT =
-        "create table blueconnecttest (d int);\nshow tables;\n";
+        "create table blueconnecttest (d int);\nshow tables;\ndrop table blueconnecttest;\n";
     final String EXPECTED_PATTERN = "blueconnecttest";
 
     // We go through these hijinxes because java considers System.getEnv
@@ -826,7 +887,7 @@ public class TestBeeLineWithArgs {
     };
     BeeLineOpts.setEnv(newEnv);
 
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false, OutStream.OUT);
   }
 
   /**
@@ -839,10 +900,10 @@ public class TestBeeLineWithArgs {
     final String SCRIPT_TEXT =
         "!close\n" +
         "!reconnect\n\n\n" +
-        "create table reconnecttest (d int);\nshow tables;\n";
+        "create table reconnecttest (d int);\nshow tables;\ndrop table reconnecttest;\n";
     final String EXPECTED_PATTERN = "reconnecttest";
 
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false, OutStream.OUT);
 
   }
 
@@ -894,7 +955,7 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testShowDbInPrompt() throws Throwable {
-    final String EXPECTED_PATTERN = " (default)>";
+    final String EXPECTED_PATTERN = " \\(default\\)>";
     List<String> argList = new ArrayList<String>();
     argList.add("--showDbInPrompt");
     argList.add("-u");
@@ -909,7 +970,7 @@ public class TestBeeLineWithArgs {
     List<String> argList = new ArrayList<String>();
     final String SCRIPT_TEXT = "!sh echo hello world";
     final String EXPECTED_PATTERN = "hello world";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList,true,false);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false, OutStream.OUT);
   }
 
   /**
@@ -924,6 +985,6 @@ public class TestBeeLineWithArgs {
     final String EXPECTED_PATTERN = "2 rows selected";
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--force");
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, OutStream.ERR);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/60a36d12/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java
index e98406d..388486d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java
@@ -58,8 +58,6 @@ public class TestOperationLoggingAPIWithTez extends OperationLoggingAPITestBase
     };
     hiveConf = new HiveConf();
     hiveConf.set(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname, "verbose");
-    // Change the engine to tez
-    hiveConf.setVar(ConfVars.HIVE_EXECUTION_ENGINE, "tez");
     // Set tez execution summary to false.
     hiveConf.setBoolVar(ConfVars.TEZ_EXEC_SUMMARY, false);
     miniHS2 = new MiniHS2(hiveConf, MiniClusterType.TEZ);