You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2014/10/29 06:51:37 UTC

svn commit: r1635054 - in /hive/branches/branch-0.14: beeline/src/java/org/apache/hive/beeline/ beeline/src/main/resources/ itests/hive-unit/src/test/java/org/apache/hive/beeline/

Author: thejas
Date: Wed Oct 29 05:51:37 2014
New Revision: 1635054

URL: http://svn.apache.org/r1635054
Log:
HIVE-8615 : beeline csv,tsv outputformat needs backward compatibility mode (Thejas Nair, reviewed by Szehon Ho, Vaibhav Gumashta)

Added:
    hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/DeprecatedSeparatedValuesOutputFormat.java
Modified:
    hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLine.java
    hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
    hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
    hive/branches/branch-0.14/beeline/src/main/resources/BeeLine.properties
    hive/branches/branch-0.14/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java

Modified: hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLine.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLine.java?rev=1635054&r1=1635053&r2=1635054&view=diff
==============================================================================
--- hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLine.java (original)
+++ hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLine.java Wed Oct 29 05:51:37 2014
@@ -145,9 +145,11 @@ public class BeeLine implements Closeabl
   private final Map<Object, Object> formats = map(new Object[] {
       "vertical", new VerticalOutputFormat(this),
       "table", new TableOutputFormat(this),
-      "csv", new SeparatedValuesOutputFormat(this, ','),
-      "tsv", new SeparatedValuesOutputFormat(this, '\t'),
+      "csv2", new SeparatedValuesOutputFormat(this, ','),
+      "tsv2", new SeparatedValuesOutputFormat(this, '\t'),
       "dsv", new SeparatedValuesOutputFormat(this, BeeLineOpts.DEFAULT_DELIMITER_FOR_DSV),
+      "csv", new DeprecatedSeparatedValuesOutputFormat(this, ','),
+      "tsv", new DeprecatedSeparatedValuesOutputFormat(this, '\t'),
       "xmlattr", new XMLAttributeOutputFormat(this),
       "xmlelements", new XMLElementOutputFormat(this),
   });

Modified: hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java?rev=1635054&r1=1635053&r2=1635054&view=diff
==============================================================================
--- hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java (original)
+++ hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java Wed Oct 29 05:51:37 2014
@@ -427,6 +427,9 @@ class BeeLineOpts implements Completor {
   }
 
   public void setOutputFormat(String outputFormat) {
+    if(outputFormat.equalsIgnoreCase("csv") || outputFormat.equalsIgnoreCase("tsv")) {
+      beeLine.info("Format " + outputFormat + " is deprecated, please use " + outputFormat + "2");
+    }
     this.outputFormat = outputFormat;
   }
 

Added: hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/DeprecatedSeparatedValuesOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/DeprecatedSeparatedValuesOutputFormat.java?rev=1635054&view=auto
==============================================================================
--- hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/DeprecatedSeparatedValuesOutputFormat.java (added)
+++ hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/DeprecatedSeparatedValuesOutputFormat.java Wed Oct 29 05:51:37 2014
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * See SQLLine notice in LICENSE
+ */
+package org.apache.hive.beeline;
+
+
+/**
+ * OutputFormat for values separated by a delimiter.
+ *
+ * Note this does not handle escaping of the quote char.
+ * The new SeparatedValuesOutputFormat supports that. The formats supported by
+ * this class are deprecated.
+ *
+ */
+class DeprecatedSeparatedValuesOutputFormat implements OutputFormat {
+
+  private final BeeLine beeLine;
+  private char separator;
+
+  public DeprecatedSeparatedValuesOutputFormat(BeeLine beeLine, char separator) {
+    this.beeLine = beeLine;
+    setSeparator(separator);
+  }
+
+  @Override
+  public int print(Rows rows) {
+    int count = 0;
+    while (rows.hasNext()) {
+      printRow(rows, (Rows.Row) rows.next());
+      count++;
+    }
+    return count - 1; // sans header row
+  }
+
+  public void printRow(Rows rows, Rows.Row row) {
+    String[] vals = row.values;
+    StringBuilder buf = new StringBuilder();
+    for (int i = 0; i < vals.length; i++) {
+      buf.append(buf.length() == 0 ? "" : "" + getSeparator())
+          .append('\'')
+          .append(vals[i] == null ? "" : vals[i])
+          .append('\'');
+    }
+    beeLine.output(buf.toString());
+  }
+
+  public void setSeparator(char separator) {
+    this.separator = separator;
+  }
+
+  public char getSeparator() {
+    return this.separator;
+  }
+}

Modified: hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java?rev=1635054&r1=1635053&r2=1635054&view=diff
==============================================================================
--- hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java (original)
+++ hive/branches/branch-0.14/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java Wed Oct 29 05:51:37 2014
@@ -51,11 +51,14 @@ class SeparatedValuesOutputFormat implem
       char newDel = beeLine.getOpts().getDelimiterForDSV();
       // if delimiter changed, rebuild the csv preference
       if (newDel != curDel) {
+        // "" is passed as the end of line symbol in following function, as
+        // beeline itself adds newline
         csvPreference = new CsvPreference.Builder('"', newDel, "").build();
       }
     }
   }
 
+  @Override
   public int print(Rows rows) {
     updateCsvPreference();
 

Modified: hive/branches/branch-0.14/beeline/src/main/resources/BeeLine.properties
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/beeline/src/main/resources/BeeLine.properties?rev=1635054&r1=1635053&r2=1635054&view=diff
==============================================================================
--- hive/branches/branch-0.14/beeline/src/main/resources/BeeLine.properties (original)
+++ hive/branches/branch-0.14/beeline/src/main/resources/BeeLine.properties Wed Oct 29 05:51:37 2014
@@ -68,7 +68,7 @@ help-procedures: List all the procedures
 help-tables: List all the tables in the database
 help-columns: List all the columns for the specified table
 help-properties: Connect to the database specified in the properties file(s)
-help-outputformat: Set the output format for displaying results (table,vertical,csv,dsv,tsv,xmlattrs,xmlelements)
+help-outputformat: Set the output format for displaying results (table,vertical,csv2,dsv,tsv2,xmlattrs,xmlelements, and deprecated formats(csv, tsv))
 help-delimiterForDSV: Set the delimiter for dsv output format
 help-nullemptystring: Set to true to get historic behavior of printing null as empty string. Default is false.
 
@@ -167,7 +167,8 @@ cmd-usage: Usage: java org.apache.hive.c
 \  --maxColumnWidth=MAXCOLWIDTH    the maximum width to use when displaying columns\n \
 \  --silent=[true/false]           be more silent\n \
 \  --autosave=[true/false]         automatically save preferences\n \
-\  --outputformat=[table/vertical/csv/tsv/dsv]   format mode for result display\n \
+\  --outputformat=[table/vertical/csv2/tsv2/dsv/csv/tsv]  format mode for result display\n \
+\                                  Note that csv, and tsv are deprecated - use csv2, tsv2 instead\n\
 \  --truncateTable=[true/false]    truncate table column when it exceeds length\n \
 \  --delimiterForDSV=DELIMITER     specify the delimiter for delimiter-separated values output format (default: |)\n \
 \  --isolation=LEVEL               set the transaction isolation level\n \

Modified: hive/branches/branch-0.14/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java?rev=1635054&r1=1635053&r2=1635054&view=diff
==============================================================================
--- hive/branches/branch-0.14/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java (original)
+++ hive/branches/branch-0.14/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java Wed Oct 29 05:51:37 2014
@@ -34,11 +34,12 @@ import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hive.service.server.HiveServer2;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -48,15 +49,12 @@ import org.junit.Test;
  * TestBeeLineWithArgs - executes tests of the command-line arguments to BeeLine
  *
  */
-//public class TestBeeLineWithArgs extends TestCase {
 public class TestBeeLineWithArgs {
   // Default location of HiveServer2
-  final private static String JDBC_URL = BeeLine.BEELINE_DEFAULT_JDBC_URL + "localhost:10000";
   private static final String tableName = "TestBeelineTable1";
   private static final String tableComment = "Test table comment";
 
-
-  private static HiveServer2 hiveServer2;
+  private static MiniHS2 miniHS2;
 
   private List<String> getBaseArgs(String jdbcUrl) {
     List<String> argList = new ArrayList<String>(8);
@@ -74,17 +72,9 @@ public class TestBeeLineWithArgs {
     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.logVars(System.err);
-    // System.err.flush();
-
-    hiveServer2 = new HiveServer2();
-    hiveServer2.init(hiveConf);
-    System.err.println("Starting HiveServer2...");
-    hiveServer2.start();
-    Thread.sleep(1000);
+    miniHS2 = new MiniHS2(hiveConf);
+    miniHS2.start(new HashMap<String,  String>());
     createTable();
-
   }
 
   /**
@@ -94,7 +84,7 @@ public class TestBeeLineWithArgs {
    */
   private static void createTable() throws ClassNotFoundException, SQLException {
     Class.forName(BeeLine.BEELINE_DEFAULT_JDBC_DRIVER);
-    Connection con = DriverManager.getConnection(JDBC_URL,"", "");
+    Connection con = DriverManager.getConnection(miniHS2.getBaseJdbcURL(),"", "");
 
     assertNotNull("Connection is null", con);
     assertFalse("Connection should not be closed", con.isClosed());
@@ -129,13 +119,8 @@ public class TestBeeLineWithArgs {
    */
   @AfterClass
   public static void postTests() {
-    try {
-      if (hiveServer2 != null) {
-        System.err.println("Stopping HiveServer2...");
-        hiveServer2.stop();
-      }
-    } catch (Throwable t) {
-      t.printStackTrace();
+    if (miniHS2.isStarted()) {
+      miniHS2.stop();
     }
   }
 
@@ -164,22 +149,20 @@ public class TestBeeLineWithArgs {
    * Test for presence of an expected pattern
    * in the output (stdout or stderr), fail if not found
    * Print PASSED or FAILED
-   * @paramm testName Name of test to print
    * @param expectedPattern Text to look for in command output/error
    * @param shouldMatch true if the pattern should be found, false if it should not
    * @throws Exception on command execution error
    */
-  private void testScriptFile(String testName, String scriptText, String expectedPattern,
+  private void testScriptFile(String scriptText, String expectedPattern,
       boolean shouldMatch, List<String> argList) throws Throwable {
 
     // Put the script content in a temp file
-    File scriptFile = File.createTempFile(testName, "temp");
+    File scriptFile = File.createTempFile(this.getClass().getSimpleName(), "temp");
     scriptFile.deleteOnExit();
     PrintStream os = new PrintStream(new FileOutputStream(scriptFile));
     os.print(scriptText);
     os.close();
 
-    System.out.println(">>> STARTED -f " + testName);
     {
       List<String> copy = new ArrayList<String>(argList);
       copy.add("-f");
@@ -189,12 +172,11 @@ public class TestBeeLineWithArgs {
       boolean matches = output.contains(expectedPattern);
       if (shouldMatch != matches) {
         //failed
-        fail(testName + ": Output" + output + " should" +  (shouldMatch ? "" : " not") +
+        fail("Output" + output + " should" +  (shouldMatch ? "" : " not") +
             " contain " + expectedPattern);
       }
     }
 
-    System.out.println(">>> STARTED -i " + testName);
     {
       List<String> copy = new ArrayList<String>(argList);
       copy.add("-i");
@@ -204,7 +186,7 @@ public class TestBeeLineWithArgs {
       boolean matches = output.contains(expectedPattern);
       if (shouldMatch != matches) {
         //failed
-        fail(testName + ": Output" + output + " should" +  (shouldMatch ? "" : " not") +
+        fail("Output" + output + " should" +  (shouldMatch ? "" : " not") +
             " contain " + expectedPattern);
       }
     }
@@ -217,11 +199,10 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testWhitespaceBeforeCommentScriptFile() throws Throwable {
-	  final String TEST_NAME = "testWhitespaceBeforeCommentScriptFile";
-	  final String SCRIPT_TEXT = " 	 	-- comment has spaces and tabs before it\n 	 	# comment has spaces and tabs before it\n";
-	  final String EXPECTED_PATTERN = "cannot recognize input near '<EOF>'";
-	  List<String> argList = getBaseArgs(JDBC_URL);
-	  testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, false, argList);
+    final String SCRIPT_TEXT = " 	 	-- comment has spaces and tabs before it\n 	 	# comment has spaces and tabs before it\n";
+    final String EXPECTED_PATTERN = "cannot recognize input near '<EOF>'";
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, false, argList);
   }
 
   /**
@@ -232,11 +213,10 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testPositiveScriptFile() throws Throwable {
-    final String TEST_NAME = "testPositiveScriptFile";
     final String SCRIPT_TEXT = "show databases;\n";
     final String EXPECTED_PATTERN = " default ";
-    List<String> argList = getBaseArgs(JDBC_URL);
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+    testScriptFile( SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
   /**
@@ -247,24 +227,22 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testBeelineHiveVariable() throws Throwable {
-    List<String> argList = getBaseArgs(JDBC_URL);
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--hivevar");
     argList.add("DUMMY_TBL=dummy");
-    final String TEST_NAME = "testHiveCommandLineHiveVariable";
     final String SCRIPT_TEXT = "create table ${DUMMY_TBL} (d int);\nshow tables;\n";
     final String EXPECTED_PATTERN = "dummy";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
   @Test
   public void testBeelineHiveConfVariable() throws Throwable {
-    List<String> argList = getBaseArgs(JDBC_URL);
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--hiveconf");
     argList.add("test.hive.table.name=dummy");
-    final String TEST_NAME = "testBeelineHiveConfVariable";
     final String SCRIPT_TEXT = "create table ${hiveconf:test.hive.table.name} (d int);\nshow tables;\n";
     final String EXPECTED_PATTERN = "dummy";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
   /**
@@ -274,7 +252,7 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testBeelineMultiHiveVariable() throws Throwable {
-    List<String> argList = getBaseArgs(JDBC_URL);
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--hivevar");
     argList.add("TABLE_NAME=dummy2");
 
@@ -289,10 +267,9 @@ public class TestBeeLineWithArgs {
     argList.add("--hiveconf");
     argList.add("COLUMN_TYPE=int");
 
-    final String TEST_NAME = "testHiveCommandLineHiveVariable";
     final String SCRIPT_TEXT = "${COMMAND} ${OBJECT} ${TABLE_NAME} (${hiveconf:COLUMN_NAME} ${hiveconf:COLUMN_TYPE});\nshow tables;\n";
     final String EXPECTED_PATTERN = "dummy2";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
   /**
@@ -302,20 +279,18 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testBreakOnErrorScriptFile() throws Throwable {
-    List<String> argList = getBaseArgs(JDBC_URL);
-    final String TEST_NAME = "testBreakOnErrorScriptFile";
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     final String SCRIPT_TEXT = "select * from abcdefg01;\nshow databases;\n";
     final String EXPECTED_PATTERN = " default ";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, false, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, false, argList);
   }
 
   @Test
   public void testBeelineShellCommand() throws Throwable {
-    List<String> argList = getBaseArgs(JDBC_URL);
-    final String TEST_NAME = "testBeelineShellCommand";
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     final String SCRIPT_TEXT = "!sh echo \"hello world.\" > hw.txt\n!sh cat hw.txt\n!rm hw.txt";
     final String EXPECTED_PATTERN = "hello world";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
   /**
@@ -324,11 +299,10 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testNullDefault() throws Throwable {
-    final String TEST_NAME = "testNullDefault";
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "select null from " + tableName + " limit 1 ;\n";
     final String EXPECTED_PATTERN = "NULL";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(JDBC_URL));
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
   }
 
   /**
@@ -337,39 +311,123 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testNullNonEmpty() throws Throwable {
-    final String TEST_NAME = "testNullNonDefault";
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "!set nullemptystring false\n select null from " + tableName + " limit 1 ;\n";
     final String EXPECTED_PATTERN = "NULL";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(JDBC_URL));
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
   }
 
   @Test
   public void testGetVariableValue() throws Throwable {
-    final String TEST_NAME = "testGetVariableValue";
     final String SCRIPT_TEXT = "set env:TERM;";
     final String EXPECTED_PATTERN = "env:TERM";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(JDBC_URL));
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
   }
 
   /**
    * Select null from table , check if setting null to empty string works.
-   * Original beeline/sqlline used to print nulls as empty strings
+   * Original beeline/sqlline used to print nulls as empty strings.
+   * Also test csv2 output format
    * Print PASSED or FAILED
    */
   @Test
   public void testNullEmpty() throws Throwable {
-    final String TEST_NAME = "testNullNonDefault";
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
                 "!set nullemptystring true\n select 'abc',null,'def' from " + tableName + " limit 1 ;\n";
     final String EXPECTED_PATTERN = "abc,,def";
 
-    List<String> argList = getBaseArgs(JDBC_URL);
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+    argList.add("--outputformat=csv2");
+
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+  }
+
+  /**
+   * Test writing output using DSV format, with custom delimiter ";"
+   */
+  @Test
+  public void testDSVOutput() throws Throwable {
+    String SCRIPT_TEXT = getFormatTestQuery();
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+    argList.add("--outputformat=dsv");
+    argList.add("--delimiterForDSV=;");
+
+    final String EXPECTED_PATTERN = "1;NULL;defg;\"ab\"\"c\";1.0";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+  }
+
+  /**
+   * Test writing output using TSV (new) format
+   */
+  @Test
+  public void testTSV2Output() throws Throwable {
+    String SCRIPT_TEXT = getFormatTestQuery();
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+    argList.add("--outputformat=tsv2");
+
+    final String EXPECTED_PATTERN = "1\tNULL\tdefg\t\"ab\"\"c\"\t1.0";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+  }
+
+  /**
+   * Test writing output using TSV deprecated format
+   */
+  @Test
+  public void testTSVOutput() throws Throwable {
+    String SCRIPT_TEXT = getFormatTestQuery();
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+    argList.add("--outputformat=tsv");
+
+    final String EXPECTED_PATTERN = "'1'\t'NULL'\t'defg'\t'ab\"c\'\t'1.0'";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+  }
+
+
+  /**
+   * Test writing output using TSV deprecated format
+   * Check for deprecation message
+   */
+  @Test
+  public void testTSVOutputDeprecation() throws Throwable {
+    String SCRIPT_TEXT = getFormatTestQuery();
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+    argList.add("--outputformat=tsv");
+
+    final String EXPECTED_PATTERN = "Format tsv is deprecated, please use tsv2";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+  }
+
+  /**
+   * Test writing output using CSV deprecated format
+   * Check for deprecation message
+   */
+  @Test
+  public void testCSVOutputDeprecation() throws Throwable {
+    String SCRIPT_TEXT = getFormatTestQuery();
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--outputformat=csv");
 
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    final String EXPECTED_PATTERN = "Format csv is deprecated, please use csv2";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+  }
+
+  /**
+   * Test writing output using CSV deprecated format
+   */
+  @Test
+  public void testCSVOutput() throws Throwable {
+    String SCRIPT_TEXT = getFormatTestQuery();
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+    argList.add("--outputformat=csv");
+    final String EXPECTED_PATTERN = "'1','NULL','defg','ab\"c\','1.0'";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
+
+  private String getFormatTestQuery() {
+    return "set hive.support.concurrency = false;\n" +
+        "select 1, null, 'defg', 'ab\"c', 1.0D from " + tableName + " limit 1 ;\n";
+  }
   /**
    * Select null from table , check if setting null to empty string works - Using beeling cmd line
    *  argument.
@@ -378,49 +436,36 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testNullEmptyCmdArg() throws Throwable {
-    final String TEST_NAME = "testNullNonDefault";
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
                 "select 'abc',null,'def' from " + tableName + " limit 1 ;\n";
-    //final String EXPECTED_PATTERN = "| abc  |      | def  |";
-    final String EXPECTED_PATTERN = "abc,,def";
+    final String EXPECTED_PATTERN = "'abc','','def'";
 
-    List<String> argList = getBaseArgs(JDBC_URL);
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--nullemptystring=true");
     argList.add("--outputformat=csv");
 
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
   /**
    * Attempt to execute a missing script file with the -f option to BeeLine
-   * Print PASSED or FAILED
    */
   @Test
   public void testNegativeScriptFile() throws Throwable {
-    final String TEST_NAME = "testNegativeScriptFile";
     final String EXPECTED_PATTERN = " default ";
 
-    long startTime = System.currentTimeMillis();
-    System.out.println(">>> STARTED " + TEST_NAME);
-
     // Create and delete a temp file
     File scriptFile = File.createTempFile("beelinenegative", "temp");
     scriptFile.delete();
 
-    List<String> argList = getBaseArgs(JDBC_URL);
+    List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("-f");
     argList.add(scriptFile.getAbsolutePath());
 
     try {
-        String output = testCommandLineScript(argList, null);
-      long elapsedTime = (System.currentTimeMillis() - startTime)/1000;
-      String time = "(" + elapsedTime + "s)";
+      String output = testCommandLineScript(argList, null);
       if (output.contains(EXPECTED_PATTERN)) {
-        System.err.println("Output: " + output);
-        System.err.println(">>> FAILED " + TEST_NAME + " (ERROR) " + time);
-        fail(TEST_NAME);
-      } else {
-        System.out.println(">>> PASSED " + TEST_NAME + " " + time);
+        fail("Output: " + output +  " Negative pattern: " + EXPECTED_PATTERN);
       }
     } catch (Throwable e) {
       e.printStackTrace();
@@ -456,11 +501,10 @@ public class TestBeeLineWithArgs {
 
   @Test
   public void testHiveVarSubstitution() throws Throwable {
-    List<String> argList = getBaseArgs(JDBC_URL + "#D_TBL=dummy_t");
-    final String TEST_NAME = "testHiveVarSubstitution";
+    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 EXPECTED_PATTERN = "dummy_t";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
   @Test
@@ -469,13 +513,12 @@ public class TestBeeLineWithArgs {
     List<String> argList = getBaseArgs(embeddedJdbcURL);
 	  argList.add("--hivevar");
     argList.add("DUMMY_TBL=embedded_table");
-    final String TEST_NAME = "testEmbeddedBeelineConnection";
     // 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";
     final String EXPECTED_PATTERN = "embedded_table";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
 
   /**
@@ -484,11 +527,10 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testQueryProgress() throws Throwable {
-    final String TEST_NAME = "testQueryProgress";
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "select count(*) from " + tableName + ";\n";
     final String EXPECTED_PATTERN = "Parsing command";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(JDBC_URL));
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
   }
 
   /**
@@ -497,11 +539,10 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testQueryProgressHidden() throws Throwable {
-    final String TEST_NAME = "testQueryProgress";
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "!set silent true\n" +
         "select count(*) from " + tableName + ";\n";
     final String EXPECTED_PATTERN = "Parsing command";
-    testScriptFile(TEST_NAME, SCRIPT_TEXT, EXPECTED_PATTERN, false, getBaseArgs(JDBC_URL));
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, false, getBaseArgs(miniHS2.getBaseJdbcURL()));
   }
 }