You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2013/10/05 01:35:53 UTC

svn commit: r1529347 [6/6] - in /hive/branches/tez: ./ ant/src/org/apache/hadoop/hive/ant/ beeline/src/java/org/apache/hive/beeline/ beeline/src/test/org/apache/hive/beeline/src/test/ bin/ common/ common/src/test/org/apache/hadoop/hive/common/type/ dat...

Modified: hive/branches/tez/ql/src/test/templates/TestParse.vm
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/templates/TestParse.vm?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/templates/TestParse.vm (original)
+++ hive/branches/tez/ql/src/test/templates/TestParse.vm Fri Oct  4 23:35:50 2013
@@ -29,20 +29,19 @@ import org.apache.hadoop.hive.ql.exec.Ta
 
 public class $className extends TestCase {
 
+  private static final String HIVE_ROOT = QTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
   private static QTestUtil qt;
 
   static {
     try {
-      boolean miniMR = false;
-      if ("$clusterMode".equals("miniMR"))
-        miniMR = true;
+      boolean miniMR = "$clusterMode".equals("miniMR");
       String hadoopVer = "$hadoopVersion";
-      qt = new QTestUtil("$resultsDir", "$logDir", miniMR, hadoopVer);
+      qt = new QTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR, hadoopVer);
       qt.init(null);
     } catch (Exception e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.flush();
+      System.err.flush();
       fail("Unexpected exception in static initialization");
     }
   }
@@ -60,21 +59,33 @@ public class $className extends TestCase
         qt.shutdown();
     }
     catch (Exception e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.flush();
+      System.err.flush();
       fail("Unexpected exception in tearDown");
     }
   }
 
   public static Test suite() {
+    Set<String> qFilesToExecute = new HashSet<String>();
+    String qFiles = System.getProperty("qfile", "").trim();
+    if(!qFiles.isEmpty()) {
+      for(String qFile : qFiles.split(",")) {
+        qFile = qFile.trim();
+        if(!qFile.isEmpty()) {
+          qFilesToExecute.add(qFile);
+        }
+      }
+    }
     TestSuite suite = new TestSuite();
 
 #foreach ($qf in $qfiles)
   #set ($fname = $qf.getName())
   #set ($eidx = $fname.indexOf('.'))
   #set ($tname = $fname.substring(0, $eidx))
-    suite.addTest(new $className("testParse_$tname"));
+    if(qFilesToExecute.isEmpty() || qFilesToExecute.contains("$fname")) {
+      suite.addTest(new $className("testParse_$tname"));
+    }
 #end
     suite.addTest(new $className("testParse_shutdown"));
     return suite;
@@ -84,7 +95,7 @@ public class $className extends TestCase
    * Dummy last test. This is only meant to shutdown qt
    */
   public void testParse_shutdown() {
-    System.out.println ("Cleaning up " + "$className");
+    System.err.println ("Cleaning up " + "$className");
   }
 
   static String debugHint = "\nSee build/ql/tmp/hive.log, "
@@ -96,7 +107,7 @@ public class $className extends TestCase
   #set ($tname = $fname.substring(0, $eidx))
   #set ($fpath = $qfilesMap.get($fname))
   public void testParse_$tname() throws Exception {
-    runTest("$tname", "$fname", "$fpath");
+    runTest("$tname", "$fname", HIVE_ROOT + "$fpath");
   }
 
 #end
@@ -104,7 +115,7 @@ public class $className extends TestCase
   private void runTest(String tname, String fname, String fpath) throws Exception {
     long startTime = System.currentTimeMillis();
     try {
-      System.out.println("Begin query: " + fname);
+      System.err.println("Begin query: " + fname);
 
       qt.addFile(fpath);
       org.apache.hadoop.hive.ql.exec.Operator.resetId();
@@ -121,19 +132,19 @@ public class $className extends TestCase
         fail("Semantic Analysis has unexpected output with error code = " + ecode
             + debugHint);
       }
-      System.out.println("Done query: " + fname);
+      System.err.println("Done query: " + fname);
       qt.getQMap().clear();
     }
     catch (Throwable e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.println("Failed query: " + fname);
-      System.out.flush();
+      System.err.println("Failed query: " + fname);
+      System.err.flush();
       fail("Unexpected exception" + debugHint);
     }
 
     long elapsedTime = System.currentTimeMillis() - startTime;
-    System.out.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
     assertTrue("Test passed", true);
   }
 }

Modified: hive/branches/tez/ql/src/test/templates/TestParseNegative.vm
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/test/templates/TestParseNegative.vm?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/ql/src/test/templates/TestParseNegative.vm (original)
+++ hive/branches/tez/ql/src/test/templates/TestParseNegative.vm Fri Oct  4 23:35:50 2013
@@ -29,19 +29,18 @@ import org.apache.hadoop.hive.ql.exec.Ta
 
 public class $className extends TestCase {
 
+  private static final String HIVE_ROOT = QTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
   private static QTestUtil qt;
-  
+ 
   static {
     try {
-      boolean miniMR = false;
-      if ("$clusterMode".equals("miniMR"))
-        miniMR = true;
+      boolean miniMR = "$clusterMode".equals("miniMR");
       String hadoopVer = "$hadoopVersion";
-      qt = new QTestUtil("$resultsDir", "$logDir", miniMR, hadoopVer);
+      qt = new QTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR, hadoopVer);
     } catch (Exception e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.flush();
+      System.err.flush();
       fail("Unexpected exception in static initialization");
     }
   }
@@ -58,9 +57,9 @@ public class $className extends TestCase
         qt.shutdown();
     }
     catch (Exception e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.flush();
+      System.err.flush();
       fail("Unexpected exception in tearDown");
     }
   }
@@ -69,17 +68,29 @@ public class $className extends TestCase
    * Dummy last test. This is only meant to shutdown qt
    */
   public void testParseNegative_shutdown() {
-    System.out.println ("Cleaning up " + "$className");
+    System.err.println ("Cleaning up " + "$className");
   }
 
   public static Test suite() {
+    Set<String> qFilesToExecute = new HashSet<String>();
+    String qFiles = System.getProperty("qfile", "").trim();
+    if(!qFiles.isEmpty()) {
+      for(String qFile : qFiles.split(",")) {
+        qFile = qFile.trim();
+        if(!qFile.isEmpty()) {
+          qFilesToExecute.add(qFile);
+        }
+      }
+    }
     TestSuite suite = new TestSuite();
 
 #foreach ($qf in $qfiles)
   #set ($fname = $qf.getName())
   #set ($eidx = $fname.indexOf('.'))
   #set ($tname = $fname.substring(0, $eidx))
-    suite.addTest(new $className("testParseNegative_$tname"));
+    if(qFilesToExecute.isEmpty() || qFilesToExecute.contains("$fname")) {
+      suite.addTest(new $className("testParseNegative_$tname"));
+    }
 #end
     suite.addTest(new $className("testParseNegative_shutdown"));
     return suite;
@@ -94,7 +105,7 @@ public class $className extends TestCase
   #set ($tname = $fname.substring(0, $eidx))
   #set ($fpath = $qfilesMap.get($fname))
   public void testParseNegative_$tname() throws Exception {
-    runTest("$tname", "$fname", "$fpath");
+    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
   }
 
 #end
@@ -102,7 +113,7 @@ public class $className extends TestCase
   private void runTest(String tname, String fname, String fpath) throws Exception {
     long startTime = System.currentTimeMillis();
     try {
-      System.out.println("Begin query: " + fname);
+      System.err.println("Begin query: " + fname);
 
       qt.addFile(fpath);
 
@@ -124,15 +135,15 @@ public class $className extends TestCase
       }
     }
     catch (Throwable e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.println("Failed query: " + fname);
-      System.out.flush();
+      System.err.println("Failed query: " + fname);
+      System.err.flush();
       fail("Unexpected exception" + debugHint);
     }
 
     long elapsedTime = System.currentTimeMillis() - startTime;
-    System.out.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
     assertTrue("Test passed", true);
   }
 }

Modified: hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveVarcharWritable.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveVarcharWritable.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveVarcharWritable.java (original)
+++ hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveVarcharWritable.java Fri Oct  4 23:35:50 2013
@@ -68,6 +68,7 @@ public class HiveVarcharWritable impleme
   }
 
   public void set(String val, int maxLength) {
+    characterLength = -1;
     value.set(HiveBaseChar.enforceMaxLength(val, maxLength));
   }
 
@@ -88,6 +89,7 @@ public class HiveVarcharWritable impleme
   }
 
   public void readFields(DataInput in) throws IOException {
+    characterLength = -1;
     value.readFields(in);
   }
 

Modified: hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java (original)
+++ hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java Fri Oct  4 23:35:50 2013
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaStringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
@@ -188,20 +187,32 @@ public final class ObjectInspectorConver
     return getConvertedOI(inputOI, outputOI, null, true);
   }
 
-  /*
+  /**
    * Utility function to convert from one object inspector type to another.
+   * The output object inspector type should have all fields as settableOI type.
+   * The above condition can be violated only if equalsCheck is true and inputOI is
+   * equal to outputOI.
+   * @param inputOI : input object inspector
+   * @param outputOI : output object inspector
+   * @param oiSettableProperties : The object inspector to isSettable mapping used to cache
+   *                               intermediate results.
+   * @param equalsCheck : Do we need to check if the inputOI and outputOI are the same?
+   *                      true : If they are the same, we return the object inspector directly.
+   *                      false : Do not perform an equality check on inputOI and outputOI
+   * @return : The output object inspector containing all settable fields. The return value
+   *           can contain non-settable fields only if inputOI equals outputOI and equalsCheck is
+   *           true.
    */
   private static ObjectInspector getConvertedOI(
       ObjectInspector inputOI,
       ObjectInspector outputOI,
       Map<ObjectInspector, Boolean> oiSettableProperties,
       boolean equalsCheck) {
-    ObjectInspector retOI = outputOI.getCategory() == Category.PRIMITIVE ? inputOI : outputOI;
-    // If the inputOI is the same as the outputOI, just return it
-    // If the retOI has all fields settable, return it
+    // 1. If equalsCheck is true and the inputOI is the same as the outputOI OR
+    // 2. If the outputOI has all fields settable, return it
     if ((equalsCheck && inputOI.equals(outputOI)) ||
-        ObjectInspectorUtils.hasAllFieldsSettable(retOI, oiSettableProperties) == true) {
-      return retOI;
+        ObjectInspectorUtils.hasAllFieldsSettable(outputOI, oiSettableProperties) == true) {
+      return outputOI;
     }
     // Return the settable equivalent object inspector for primitive categories
     // For eg: for table T containing partitions p1 and p2 (possibly different
@@ -209,11 +220,12 @@ public final class ObjectInspectorConver
     // T is settable recursively i.e all the nested fields are also settable.
     switch (outputOI.getCategory()) {
     case PRIMITIVE:
-      PrimitiveObjectInspector primInputOI = (PrimitiveObjectInspector) inputOI;
-      return PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(primInputOI);
+      // Create a writable object inspector for primitive type and return it.
+      PrimitiveObjectInspector primOutputOI = (PrimitiveObjectInspector) outputOI;
+      return PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(primOutputOI);
     case STRUCT:
       StructObjectInspector structOutputOI = (StructObjectInspector) outputOI;
-      // create a standard settable struct object inspector
+      // create a standard settable struct object inspector.
       List<? extends StructField> listFields = structOutputOI.getAllStructFieldRefs();
       List<String> structFieldNames = new ArrayList<String>(listFields.size());
       List<ObjectInspector> structFieldObjectInspectors = new ArrayList<ObjectInspector>(
@@ -221,6 +233,10 @@ public final class ObjectInspectorConver
 
       for (StructField listField : listFields) {
         structFieldNames.add(listField.getFieldName());
+        // We need to make sure that the underlying fields are settable as well.
+        // Hence, the recursive call for each field.
+        // Note that equalsCheck is false while invoking getConvertedOI() because
+        // we need to bypass the initial inputOI.equals(outputOI) check.
         structFieldObjectInspectors.add(getConvertedOI(listField.getFieldObjectInspector(),
             listField.getFieldObjectInspector(), oiSettableProperties, false));
       }
@@ -229,11 +245,13 @@ public final class ObjectInspectorConver
           structFieldObjectInspectors);
     case LIST:
       ListObjectInspector listOutputOI = (ListObjectInspector) outputOI;
+      // We need to make sure that the list element type is settable.
       return ObjectInspectorFactory.getStandardListObjectInspector(
           getConvertedOI(listOutputOI.getListElementObjectInspector(),
               listOutputOI.getListElementObjectInspector(), oiSettableProperties, false));
     case MAP:
       MapObjectInspector mapOutputOI = (MapObjectInspector) outputOI;
+      // We need to make sure that the key type and the value types are settable.
       return ObjectInspectorFactory.getStandardMapObjectInspector(
           getConvertedOI(mapOutputOI.getMapKeyObjectInspector(),
               mapOutputOI.getMapKeyObjectInspector(), oiSettableProperties, false),
@@ -246,11 +264,13 @@ public final class ObjectInspectorConver
       List<ObjectInspector> unionFieldObjectInspectors = new ArrayList<ObjectInspector>(
           unionListFields.size());
       for (ObjectInspector listField : unionListFields) {
+        // We need to make sure that all the field associated with the union are settable.
         unionFieldObjectInspectors.add(getConvertedOI(listField, listField, oiSettableProperties,
             false));
       }
       return ObjectInspectorFactory.getStandardUnionObjectInspector(unionFieldObjectInspectors);
     default:
+      // Unsupported in-memory structure.
       throw new RuntimeException("Hive internal error: conversion of "
           + inputOI.getTypeName() + " to " + outputOI.getTypeName()
           + " not supported yet.");

Modified: hive/branches/tez/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java (original)
+++ hive/branches/tez/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java Fri Oct  4 23:35:50 2013
@@ -25,6 +25,9 @@ import org.apache.hadoop.hive.serde2.io.
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeParams;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -171,4 +174,22 @@ public class TestObjectInspectorConverte
     }
 
   }
+
+  public void testGetConvertedOI() throws Throwable {
+    // Try with types that have type params
+    PrimitiveTypeInfo varchar5TI =
+        (PrimitiveTypeInfo) TypeInfoFactory.getPrimitiveTypeInfo("varchar(5)");
+    PrimitiveTypeInfo varchar10TI =
+        (PrimitiveTypeInfo) TypeInfoFactory.getPrimitiveTypeInfo("varchar(10)");
+    PrimitiveObjectInspector varchar5OI =
+        PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(varchar5TI);
+    PrimitiveObjectInspector varchar10OI =
+        PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(varchar10TI);
+
+    // output OI should have varchar type params
+    PrimitiveObjectInspector poi = (PrimitiveObjectInspector)
+        ObjectInspectorConverters.getConvertedOI(varchar10OI, varchar5OI);
+    VarcharTypeParams vcParams = (VarcharTypeParams) poi.getTypeParams();
+    assertEquals("varchar length doesn't match", 5, vcParams.length);
+  }
 }

Modified: hive/branches/tez/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java (original)
+++ hive/branches/tez/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java Fri Oct  4 23:35:50 2013
@@ -91,7 +91,7 @@ public class SQLOperation extends Execut
     String SQLState = null;
 
     try {
-      driver = new Driver(getParentSession().getHiveConf());
+      driver = new Driver(getParentSession().getHiveConf(), getParentSession().getUserName());
       // In Hive server mode, we are not able to retry in the FetchTask
       // case, when calling fetch queries since execute() has returned.
       // For now, we disable the test attempts.

Modified: hive/branches/tez/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (original)
+++ hive/branches/tez/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java Fri Oct  4 23:35:50 2013
@@ -184,13 +184,18 @@ public class HiveSessionImpl implements 
       boolean runAsync)
       throws HiveSQLException {
     acquire();
-    try {
-      ExecuteStatementOperation operation = getOperationManager()
+
+    OperationManager operationManager = getOperationManager();
+    ExecuteStatementOperation operation = operationManager
           .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync);
+    OperationHandle opHandle = operation.getHandle();
+    try {
       operation.run();
-      OperationHandle opHandle = operation.getHandle();
       opHandleSet.add(opHandle);
       return opHandle;
+    } catch (HiveSQLException e) {
+      operationManager.closeOperation(opHandle);
+      throw e;
     } finally {
       release();
     }
@@ -199,12 +204,17 @@ public class HiveSessionImpl implements 
   public OperationHandle getTypeInfo()
       throws HiveSQLException {
     acquire();
+
+    OperationManager operationManager = getOperationManager();
+    GetTypeInfoOperation operation = operationManager.newGetTypeInfoOperation(getSession());
+    OperationHandle opHandle = operation.getHandle();
     try {
-      GetTypeInfoOperation operation = getOperationManager().newGetTypeInfoOperation(getSession());
       operation.run();
-      OperationHandle opHandle = operation.getHandle();
       opHandleSet.add(opHandle);
       return opHandle;
+    } catch (HiveSQLException e) {
+      operationManager.closeOperation(opHandle);
+      throw e;
     } finally {
       release();
     }
@@ -213,12 +223,17 @@ public class HiveSessionImpl implements 
   public OperationHandle getCatalogs()
       throws HiveSQLException {
     acquire();
+
+    OperationManager operationManager = getOperationManager();
+    GetCatalogsOperation operation = operationManager.newGetCatalogsOperation(getSession());
+    OperationHandle opHandle = operation.getHandle();
     try {
-      GetCatalogsOperation operation = getOperationManager().newGetCatalogsOperation(getSession());
       operation.run();
-      OperationHandle opHandle = operation.getHandle();
       opHandleSet.add(opHandle);
       return opHandle;
+    } catch (HiveSQLException e) {
+      operationManager.closeOperation(opHandle);
+      throw e;
     } finally {
       release();
     }
@@ -226,14 +241,19 @@ public class HiveSessionImpl implements 
 
   public OperationHandle getSchemas(String catalogName, String schemaName)
       throws HiveSQLException {
-      acquire();
+    acquire();
+
+    OperationManager operationManager = getOperationManager();
+    GetSchemasOperation operation =
+        operationManager.newGetSchemasOperation(getSession(), catalogName, schemaName);
+    OperationHandle opHandle = operation.getHandle();
     try {
-      GetSchemasOperation operation =
-          getOperationManager().newGetSchemasOperation(getSession(), catalogName, schemaName);
       operation.run();
-      OperationHandle opHandle = operation.getHandle();
       opHandleSet.add(opHandle);
       return opHandle;
+    } catch (HiveSQLException e) {
+      operationManager.closeOperation(opHandle);
+      throw e;
     } finally {
       release();
     }
@@ -242,14 +262,19 @@ public class HiveSessionImpl implements 
   public OperationHandle getTables(String catalogName, String schemaName, String tableName,
       List<String> tableTypes)
       throws HiveSQLException {
-      acquire();
+    acquire();
+
+    OperationManager operationManager = getOperationManager();
+    MetadataOperation operation =
+        operationManager.newGetTablesOperation(getSession(), catalogName, schemaName, tableName, tableTypes);
+    OperationHandle opHandle = operation.getHandle();
     try {
-      MetadataOperation operation =
-          getOperationManager().newGetTablesOperation(getSession(), catalogName, schemaName, tableName, tableTypes);
       operation.run();
-      OperationHandle opHandle = operation.getHandle();
       opHandleSet.add(opHandle);
       return opHandle;
+    } catch (HiveSQLException e) {
+      operationManager.closeOperation(opHandle);
+      throw e;
     } finally {
       release();
     }
@@ -257,13 +282,18 @@ public class HiveSessionImpl implements 
 
   public OperationHandle getTableTypes()
       throws HiveSQLException {
-      acquire();
+    acquire();
+
+    OperationManager operationManager = getOperationManager();
+    GetTableTypesOperation operation = operationManager.newGetTableTypesOperation(getSession());
+    OperationHandle opHandle = operation.getHandle();
     try {
-      GetTableTypesOperation operation = getOperationManager().newGetTableTypesOperation(getSession());
       operation.run();
-      OperationHandle opHandle = operation.getHandle();
       opHandleSet.add(opHandle);
       return opHandle;
+    } catch (HiveSQLException e) {
+      operationManager.closeOperation(opHandle);
+      throw e;
     } finally {
       release();
     }
@@ -272,13 +302,18 @@ public class HiveSessionImpl implements 
   public OperationHandle getColumns(String catalogName, String schemaName,
       String tableName, String columnName)  throws HiveSQLException {
     acquire();
-    try {
-    GetColumnsOperation operation = getOperationManager().newGetColumnsOperation(getSession(),
+
+    OperationManager operationManager = getOperationManager();
+    GetColumnsOperation operation = operationManager.newGetColumnsOperation(getSession(),
         catalogName, schemaName, tableName, columnName);
-    operation.run();
     OperationHandle opHandle = operation.getHandle();
+    try {
+    operation.run();
     opHandleSet.add(opHandle);
     return opHandle;
+    } catch (HiveSQLException e) {
+      operationManager.closeOperation(opHandle); 
+      throw e;
     } finally {
       release();
     }
@@ -287,13 +322,18 @@ public class HiveSessionImpl implements 
   public OperationHandle getFunctions(String catalogName, String schemaName, String functionName)
       throws HiveSQLException {
     acquire();
+
+    OperationManager operationManager = getOperationManager();
+    GetFunctionsOperation operation = operationManager
+        .newGetFunctionsOperation(getSession(), catalogName, schemaName, functionName);
+    OperationHandle opHandle = operation.getHandle();
     try {
-      GetFunctionsOperation operation = getOperationManager()
-          .newGetFunctionsOperation(getSession(), catalogName, schemaName, functionName);
       operation.run();
-      OperationHandle opHandle = operation.getHandle();
       opHandleSet.add(opHandle);
       return opHandle;
+    } catch (HiveSQLException e) {
+      operationManager.closeOperation(opHandle);
+      throw e;
     } finally {
       release();
     }

Modified: hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (original)
+++ hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java Fri Oct  4 23:35:50 2013
@@ -122,7 +122,7 @@ public abstract class ThriftCLIService e
       resp.setConfiguration(new HashMap<String, String>());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error opening session: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -174,7 +174,7 @@ public abstract class ThriftCLIService e
       cliService.closeSession(sessionHandle);
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error closing session: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -190,7 +190,7 @@ public abstract class ThriftCLIService e
       resp.setInfoValue(getInfoValue.toTGetInfoValue());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting info: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -210,8 +210,8 @@ public abstract class ThriftCLIService e
           resp.setOperationHandle(operationHandle.toTOperationHandle());
           resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
-      resp.setStatus(HiveSQLException.toTStatus(e));
+       LOG.warn("Error fetching results: ", e);
+       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
   }
@@ -224,7 +224,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(operationHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error executing statement: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -238,7 +238,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting type info: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -253,7 +253,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting catalogs: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -269,7 +269,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting schemas: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -283,7 +283,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting tables: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -302,7 +302,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting table types: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -318,7 +318,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting columns: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -332,7 +332,7 @@ public abstract class ThriftCLIService e
       resp.setOperationState(operationState.toTOperationState());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting functions: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -345,7 +345,7 @@ public abstract class ThriftCLIService e
       cliService.cancelOperation(new OperationHandle(req.getOperationHandle()));
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting operation status: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -358,7 +358,7 @@ public abstract class ThriftCLIService e
       cliService.closeOperation(new OperationHandle(req.getOperationHandle()));
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error canceling operation: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -373,7 +373,7 @@ public abstract class ThriftCLIService e
       resp.setSchema(schema.toTTableSchema());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error closing operation: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -391,7 +391,7 @@ public abstract class ThriftCLIService e
       resp.setHasMoreRows(false);
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.warn("Error getting result set metadata: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;

Modified: hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java (original)
+++ hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java Fri Oct  4 23:35:50 2013
@@ -122,18 +122,23 @@ public abstract class CLIServiceTest {
     assertNotNull(sessionHandle);
 
     // Change lock manager, otherwise unit-test doesn't go through
-    String setLockMgr = "SET hive.lock.manager=" +
+    String queryString = "SET hive.lock.manager=" +
         "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager";
-    client.executeStatement(sessionHandle, setLockMgr, confOverlay);
+    client.executeStatement(sessionHandle, queryString, confOverlay);
 
-    String createTable = "CREATE TABLE TEST_EXEC(ID STRING)";
-    client.executeStatement(sessionHandle, createTable, confOverlay);
+    // Drop the table if it exists
+    queryString = "DROP TABLE IF EXISTS TEST_EXEC";
+    client.executeStatement(sessionHandle, queryString, confOverlay);
 
-    // blocking execute
-    String select = "SELECT ID FROM TEST_EXEC";
-    OperationHandle ophandle = client.executeStatement(sessionHandle, select, confOverlay);
+    // Create a test table
+    queryString = "CREATE TABLE TEST_EXEC(ID STRING)";
+    client.executeStatement(sessionHandle, queryString, confOverlay);
 
-    // expect query to be completed now
+    // Blocking execute
+    queryString = "SELECT ID FROM TEST_EXEC";
+    OperationHandle ophandle = client.executeStatement(sessionHandle, queryString, confOverlay);
+
+    // Expect query to be completed now
     assertEquals("Query should be finished",
         OperationState.FINISHED, client.getOperationStatus(ophandle));
   }
@@ -150,23 +155,28 @@ public abstract class CLIServiceTest {
     OperationHandle ophandle;
 
     // Change lock manager, otherwise unit-test doesn't go through
-    String setLockMgr = "SET hive.lock.manager=" +
+    String queryString = "SET hive.lock.manager=" +
         "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager";
-    client.executeStatement(sessionHandle, setLockMgr, confOverlay);
+    client.executeStatement(sessionHandle, queryString, confOverlay);
 
-    String createTable = "CREATE TABLE TEST_EXEC_ASYNC(ID STRING)";
-    client.executeStatementAsync(sessionHandle, createTable, confOverlay);
+    // Drop the table if it exists
+    queryString = "DROP TABLE IF EXISTS TEST_EXEC_ASYNC";
+    client.executeStatement(sessionHandle, queryString, confOverlay);
+
+    // Create a test table
+    queryString = "CREATE TABLE TEST_EXEC_ASYNC(ID STRING)";
+    client.executeStatement(sessionHandle, queryString, confOverlay);
 
     // Test async execution response when query is malformed
-    String wrongQuery = "SELECT NAME FROM TEST_EXEC";
-    ophandle = client.executeStatementAsync(sessionHandle, wrongQuery, confOverlay);
+    String wrongQueryString = "SELECT NAME FROM TEST_EXEC";
+    ophandle = client.executeStatementAsync(sessionHandle, wrongQueryString, confOverlay);
 
     int count = 0;
     while (true) {
       // Break if polling times out
       if (System.currentTimeMillis() > pollTimeout) {
-          System.out.println("Polling timed out");
-          break;
+        System.out.println("Polling timed out");
+        break;
       }
       state = client.getOperationStatus(ophandle);
       System.out.println("Polling: " + ophandle + " count=" + (++count)
@@ -182,16 +192,16 @@ public abstract class CLIServiceTest {
         OperationState.ERROR, client.getOperationStatus(ophandle));
 
     // Test async execution when query is well formed
-    String select = "SELECT ID FROM TEST_EXEC_ASYNC";
+    queryString = "SELECT ID FROM TEST_EXEC_ASYNC";
     ophandle =
-        client.executeStatementAsync(sessionHandle, select, confOverlay);
+        client.executeStatementAsync(sessionHandle, queryString, confOverlay);
 
     count = 0;
     while (true) {
       // Break if polling times out
       if (System.currentTimeMillis() > pollTimeout) {
-          System.out.println("Polling timed out");
-          break;
+        System.out.println("Polling timed out");
+        break;
       }
       state = client.getOperationStatus(ophandle);
       System.out.println("Polling: " + ophandle + " count=" + (++count)
@@ -207,7 +217,7 @@ public abstract class CLIServiceTest {
         OperationState.FINISHED, client.getOperationStatus(ophandle));
 
     // Cancellation test
-    ophandle = client.executeStatementAsync(sessionHandle, select, confOverlay);
+    ophandle = client.executeStatementAsync(sessionHandle, queryString, confOverlay);
     System.out.println("cancelling " + ophandle);
     client.cancelOperation(ophandle);
     state = client.getOperationStatus(ophandle);

Modified: hive/branches/tez/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java (original)
+++ hive/branches/tez/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java Fri Oct  4 23:35:50 2013
@@ -174,10 +174,16 @@ public abstract class ThriftCLIServiceTe
         "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager";
     executeQuerySync(queryString, sessHandle);
 
-    queryString = "CREATE TABLE TEST_EXEC(ID STRING)";
+    // Drop the table if it exists
+    queryString = "DROP TABLE IF EXISTS TEST_EXEC_THRIFT";
     executeQuerySync(queryString, sessHandle);
 
-    queryString = "SELECT ID FROM TEST_EXEC";
+    // Create a test table
+    queryString = "CREATE TABLE TEST_EXEC_THRIFT(ID STRING)";
+    executeQuerySync(queryString, sessHandle);
+
+    // Execute another query to test
+    queryString = "SELECT ID FROM TEST_EXEC_THRIFT";
     TExecuteStatementResp execResp = executeQuerySync(queryString, sessHandle);
     TOperationHandle operationHandle = execResp.getOperationHandle();
     assertNotNull(operationHandle);
@@ -187,7 +193,7 @@ public abstract class ThriftCLIServiceTe
     assertNotNull(opStatusReq);
     TGetOperationStatusResp opStatusResp = client.GetOperationStatus(opStatusReq);
 
-    // expect query to be completed now since it was an async call
+    // Expect query to be completed now
     assertEquals("Query should be finished",
         OperationState.FINISHED, OperationState.getOperationState(opStatusResp.getOperationState()));
 

Modified: hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java (original)
+++ hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java Fri Oct  4 23:35:50 2013
@@ -107,6 +107,7 @@ public class PTest {
     put("clearLibraryCache", String.valueOf(configuration.isClearLibraryCache())).
     put("workingDir", mExecutionContext.getLocalWorkingDirectory()).
     put("antArgs", configuration.getAntArgs()).
+    put("antTestArgs", configuration.getAntTestArgs()).
     put("buildTag", buildTag).
     put("logDir", logDir.getAbsolutePath()).
     put("javaHome", configuration.getJavaHome()).
@@ -228,6 +229,7 @@ public class PTest {
   private static final String PATCH = "patch";
   private static final String JAVA_HOME = TestConfiguration.JAVA_HOME;
   private static final String JAVA_HOME_TEST = TestConfiguration.JAVA_HOME_TEST;
+  private static final String ANT_TEST_ARGS = TestConfiguration.ANT_TEST_ARGS;
   private static final String ANT_ENV_OPTS = TestConfiguration.ANT_ENV_OPTS;
   /**
    * All args override properties file settings except
@@ -246,7 +248,8 @@ public class PTest {
     options.addOption(null, PATCH, true, "URI to patch, either file:/// or http(s)://");
     options.addOption(ANT_ARG, null, true, "Supplemntal ant arguments");
     options.addOption(null, JAVA_HOME, true, "Java Home for compiling and running tests (unless " + JAVA_HOME_TEST + " is specified)");
-    options.addOption(null, JAVA_HOME_TEST, true, "Java Home for running tests (optional)");
+    options.addOption(null, JAVA_HOME_TEST, true, "Java Home for running tests (optional)");    
+    options.addOption(null, ANT_TEST_ARGS, true, "Arguments to ant test on slave nodes only");
     options.addOption(null, ANT_ENV_OPTS, true, "ANT_OPTS environemnt variable setting");
     CommandLine commandLine = parser.parse(options, args);
     if(!commandLine.hasOption(PROPERTIES)) {
@@ -289,6 +292,10 @@ public class PTest {
         if(!javaHomeForTests.isEmpty()) {
           conf.setJavaHomeForTests(javaHomeForTests);
         }
+        String antTestArgs = Strings.nullToEmpty(commandLine.getOptionValue(ANT_TEST_ARGS)).trim();
+        if(!antTestArgs.isEmpty()) {
+          conf.setAntTestArgs(antTestArgs);
+        }
         String antEnvOpts = Strings.nullToEmpty(commandLine.getOptionValue(ANT_ENV_OPTS)).trim();
         if(!antEnvOpts.isEmpty()) {
           conf.setAntEnvOpts(antEnvOpts);

Modified: hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java (original)
+++ hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java Fri Oct  4 23:35:50 2013
@@ -38,6 +38,7 @@ public class TestConfiguration {
   public static final String JAVA_HOME = "javaHome";
   public static final String JAVA_HOME_TEST = "javaHomeForTests";
   public static final String ANT_ENV_OPTS = "antEnvOpts";
+  public static final String ANT_TEST_ARGS = "antTestArgs";
   
   private static final String REPOSITORY_TYPE = "repositoryType";
   private static final String GIT = "git";
@@ -50,6 +51,7 @@ public class TestConfiguration {
 
   private final Context context;
   private String antArgs;
+  private String antTestArgs;
   private String antEnvOpts;
   private String repositoryType;
   private String repository;
@@ -80,6 +82,7 @@ public class TestConfiguration {
       throw new IllegalArgumentException("Unkown repository type '" + repositoryType + "'");
     }
     antArgs =  Preconditions.checkNotNull(context.getString(ANT_ARGS), ANT_ARGS).trim();
+    antTestArgs =  context.getString(ANT_TEST_ARGS, "").trim();
     antEnvOpts =  context.getString(ANT_ENV_OPTS, "").trim();
     javaHome =  context.getString(JAVA_HOME, "").trim();
     javaHomeForTests = context.getString(JAVA_HOME_TEST, "").trim();
@@ -133,6 +136,12 @@ public class TestConfiguration {
   public String getAntArgs() {
     return antArgs;
   }
+  public String getAntTestArgs() {
+    return antTestArgs;
+  }
+  public String getAntEnvOpts() {
+    return antEnvOpts;
+  }
   public String getJavaHome() {
     return javaHome;
   }
@@ -163,8 +172,8 @@ public class TestConfiguration {
   public void setAntArgs(String antArgs) {
     this.antArgs = Strings.nullToEmpty(antArgs);
   }
-  public String getAntEnvOpts() {
-    return antEnvOpts;
+  public void setAntTestArgs(String antTestArgs) {
+    this.antTestArgs = antTestArgs;
   }
   public void setAntEnvOpts(String antEnvOpts) {
     this.antEnvOpts = Strings.nullToEmpty(antEnvOpts);

Modified: hive/branches/tez/testutils/ptest2/src/main/resources/batch-exec.vm
URL: http://svn.apache.org/viewvc/hive/branches/tez/testutils/ptest2/src/main/resources/batch-exec.vm?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/testutils/ptest2/src/main/resources/batch-exec.vm (original)
+++ hive/branches/tez/testutils/ptest2/src/main/resources/batch-exec.vm Fri Oct  4 23:35:50 2013
@@ -41,7 +41,7 @@ cd $localDir/$instanceName/${repositoryN
    -Divy.default.always.check.exact.revision=false \
    -Divy.cache.ttl.default=eternal -Divy.checkmodified=false \
    -Dmvn.local.repo=$localDir/$instanceName/maven \
-   $antArgs $testArguments 1>$logDir/ant-test.txt 2>&1
+   $antArgs $antTestArgs $testArguments 1>$logDir/ant-test.txt 2>&1
 ret=$?
 if [[ $ret -ne 0 ]]
 then

Modified: hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java (original)
+++ hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java Fri Oct  4 23:35:50 2013
@@ -73,6 +73,7 @@ public class TestScripts  {
     templateVariables.put("clearLibraryCache", "true");
     templateVariables.put("javaHome", "/usr/java/jdk1.7");
     templateVariables.put("antEnvOpts", "-Dhttp.proxyHost=somehost -Dhttp.proxyPort=3128");
+    templateVariables.put("antTestArgs", "-DgrammarBuild.notRequired=true -Dskip.javadoc=true");
     String template = readResource("batch-exec.vm");
     String actual = getTemplateResult(template, templateVariables);
     Approvals.verify(actual);
@@ -97,6 +98,7 @@ public class TestScripts  {
     templateVariables.put("javaHome", "/usr/java/jdk1.7");
     templateVariables.put("javaHomeForTests", "/usr/java/jdk1.7-other");
     templateVariables.put("antEnvOpts", "-Dhttp.proxyHost=somehost -Dhttp.proxyPort=3128");
+    templateVariables.put("antTestArgs", "");
     String template = readResource("batch-exec.vm");
     String actual = getTemplateResult(template, templateVariables);
     Approvals.verify(actual);

Modified: hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt (original)
+++ hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt Fri Oct  4 23:35:50 2013
@@ -40,7 +40,7 @@ cd /some/local/dir/instance-1/apache-sou
    -Divy.default.always.check.exact.revision=false \
    -Divy.cache.ttl.default=eternal -Divy.checkmodified=false \
    -Dmvn.local.repo=/some/local/dir/instance-1/maven \
-   -Dant=arg1 -Dtest=arg1 1>/some/log/dir/ant-test.txt 2>&1
+   -Dant=arg1  -Dtest=arg1 1>/some/log/dir/ant-test.txt 2>&1
 ret=$?
 if [[ $ret -ne 0 ]]
 then

Modified: hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt (original)
+++ hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt Fri Oct  4 23:35:50 2013
@@ -40,7 +40,7 @@ cd /some/local/dir/instance-1/apache-sou
    -Divy.default.always.check.exact.revision=false \
    -Divy.cache.ttl.default=eternal -Divy.checkmodified=false \
    -Dmvn.local.repo=/some/local/dir/instance-1/maven \
-   -Dant=arg1 -Dtest=arg1 1>/some/log/dir/ant-test.txt 2>&1
+   -Dant=arg1 -DgrammarBuild.notRequired=true -Dskip.javadoc=true -Dtest=arg1 1>/some/log/dir/ant-test.txt 2>&1
 ret=$?
 if [[ $ret -ne 0 ]]
 then