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/11/05 08:01:58 UTC

svn commit: r1538880 [2/46] - in /hive/branches/tez: ./ ant/ ant/src/org/apache/hadoop/hive/ant/ beeline/ beeline/src/java/org/apache/hive/beeline/ beeline/src/main/ beeline/src/test/org/apache/hive/beeline/src/test/ cli/ common/ common/src/java/conf/ ...

Propchange: hive/branches/tez/
------------------------------------------------------------------------------
  Merged /hive/trunk:r1536916-1538724

Modified: hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java (original)
+++ hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java Tue Nov  5 07:01:32 2013
@@ -215,6 +215,9 @@ public class GenVectorCode extends Task 
       {"FilterStringColumnCompareScalar", "Greater", ">"},
       {"FilterStringColumnCompareScalar", "GreaterEqual", ">="},
 
+      {"FilterStringColumnBetween", ""},
+      {"FilterStringColumnBetween", "!"},
+
       {"StringColumnCompareScalar", "Equal", "=="},
       {"StringColumnCompareScalar", "NotEqual", "!="},
       {"StringColumnCompareScalar", "Less", "<"},
@@ -276,6 +279,11 @@ public class GenVectorCode extends Task 
         {"FilterColumnCompareColumn", "GreaterEqual", "long", "long", ">="},
         {"FilterColumnCompareColumn", "GreaterEqual", "double", "long", ">="},
 
+      {"FilterColumnBetween", "long", ""},
+      {"FilterColumnBetween", "double", ""},
+      {"FilterColumnBetween", "long", "!"},
+      {"FilterColumnBetween", "double", "!"},
+
       {"ColumnCompareColumn", "Equal", "long", "double", "=="},
       {"ColumnCompareColumn", "Equal", "double", "double", "=="},
       {"ColumnCompareColumn", "NotEqual", "long", "double", "!="},
@@ -452,7 +460,7 @@ public class GenVectorCode extends Task 
   public void init(String templateBaseDir, String buildDir) {
     File generationDirectory = new File(templateBaseDir);
 
-    String buildPath = joinPath(buildDir, "ql", "gen", "vector");
+    String buildPath = joinPath(buildDir, "generated-sources", "java");
 
     File exprOutput = new File(joinPath(buildPath, "org", "apache", "hadoop",
         "hive", "ql", "exec", "vector", "expressions", "gen"));
@@ -470,7 +478,7 @@ public class GenVectorCode extends Task 
 
     File testCodeOutput =
         new File(
-            joinPath(buildDir, "ql", "test", "src", "org",
+            joinPath(buildDir, "generated-test-sources", "java", "org",
                 "apache", "hadoop", "hive", "ql", "exec", "vector",
                 "expressions", "gen"));
     testCodeGen = new GenVectorTestCode(testCodeOutput.getAbsolutePath(),
@@ -511,6 +519,8 @@ public class GenVectorCode extends Task 
         generateFilterColumnCompareScalar(tdesc);
       } else if (tdesc[0].equals("FilterScalarCompareColumn")) {
         generateFilterScalarCompareColumn(tdesc);
+      } else if (tdesc[0].equals("FilterColumnBetween")) {
+        generateFilterColumnBetween(tdesc);
       } else if (tdesc[0].equals("ScalarArithmeticColumn")) {
         generateScalarArithmeticColumn(tdesc);
       } else if (tdesc[0].equals("FilterColumnCompareColumn")) {
@@ -535,6 +545,8 @@ public class GenVectorCode extends Task 
         generateVectorUDAFVar(tdesc);
       } else if (tdesc[0].equals("FilterStringColumnCompareScalar")) {
         generateFilterStringColumnCompareScalar(tdesc);
+      } else if (tdesc[0].equals("FilterStringColumnBetween")) {
+        generateFilterStringColumnBetween(tdesc);
       } else if (tdesc[0].equals("StringColumnCompareScalar")) {
         generateStringColumnCompareScalar(tdesc);
       } else if (tdesc[0].equals("FilterStringScalarCompareColumn")) {
@@ -553,6 +565,40 @@ public class GenVectorCode extends Task 
     testCodeGen.generateTestSuites();
   }
 
+  private void generateFilterStringColumnBetween(String[] tdesc) throws IOException {
+    String optionalNot = tdesc[1];
+    String className = "FilterStringColumn" + (optionalNot.equals("!") ? "Not" : "")
+        + "Between";
+    String outputFile = joinPath(this.expressionOutputDirectory, className + ".java");
+
+    // Read the template into a string, expand it, and write it.
+    String templateFile = joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt");
+    String templateString = readFile(templateFile);
+    templateString = templateString.replaceAll("<ClassName>", className);
+    templateString = templateString.replaceAll("<OptionalNot>", optionalNot);
+    writeFile(outputFile, templateString);
+  }
+
+  private void generateFilterColumnBetween(String[] tdesc) throws IOException {
+    String operandType = tdesc[1];
+    String optionalNot = tdesc[2];
+
+    String className = "Filter" + getCamelCaseType(operandType) + "Column" +
+      (optionalNot.equals("!") ? "Not" : "") + "Between";
+    String inputColumnVectorType = getColumnVectorType(operandType);
+    String outputFile = joinPath(this.expressionOutputDirectory, className + ".java");
+
+    // Read the template into a string, expand it, and write it.
+    String templateFile = joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt");
+    String templateString = readFile(templateFile);
+    templateString = templateString.replaceAll("<ClassName>", className);
+    templateString = templateString.replaceAll("<InputColumnVectorType>", inputColumnVectorType);
+    templateString = templateString.replaceAll("<OperandType>", operandType);
+    templateString = templateString.replaceAll("<OptionalNot>", optionalNot);
+
+    writeFile(outputFile, templateString);
+  }
+
   private void generateColumnCompareColumn(String[] tdesc) throws IOException {
     //The variables are all same as ColumnCompareScalar except that
     //this template doesn't need a return type. Pass anything as return type.

Modified: hive/branches/tez/beeline/src/java/org/apache/hive/beeline/BeeLine.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/beeline/src/java/org/apache/hive/beeline/BeeLine.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/beeline/src/java/org/apache/hive/beeline/BeeLine.java (original)
+++ hive/branches/tez/beeline/src/java/org/apache/hive/beeline/BeeLine.java Tue Nov  5 07:01:32 2013
@@ -95,7 +95,7 @@ import jline.SimpleCompletor;
  */
 public class BeeLine {
   private static final ResourceBundle resourceBundle =
-      ResourceBundle.getBundle(BeeLine.class.getName());
+      ResourceBundle.getBundle(BeeLine.class.getSimpleName());
   private final BeeLineSignalHandler signalHandler = null;
   private static final String separator = System.getProperty("line.separator");
   private boolean exit = false;

Modified: hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Nov  5 07:01:32 2013
@@ -27,10 +27,12 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -440,8 +442,10 @@ public class HiveConf extends Configurat
 
     // Default file format for CREATE TABLE statement
     // Options: TextFile, SequenceFile
-    HIVEDEFAULTFILEFORMAT("hive.default.fileformat", "TextFile"),
-    HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "TextFile"),
+    HIVEDEFAULTFILEFORMAT("hive.default.fileformat", "TextFile",
+        new StringsValidator("TextFile", "SequenceFile", "RCfile", "ORC")),
+    HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "TextFile",
+        new StringsValidator("TextFile", "SequenceFile", "RCfile")),
     HIVECHECKFILEFORMAT("hive.fileformat.check", true),
 
     // default serde for rcfile
@@ -658,10 +662,13 @@ public class HiveConf extends Configurat
     HIVEOUTERJOINSUPPORTSFILTERS("hive.outerjoin.supports.filters", true),
 
     // 'minimal', 'more' (and 'all' later)
-    HIVEFETCHTASKCONVERSION("hive.fetch.task.conversion", "minimal"),
+    HIVEFETCHTASKCONVERSION("hive.fetch.task.conversion", "minimal",
+        new StringsValidator("minimal", "more")),
 
     HIVEFETCHTASKAGGR("hive.fetch.task.aggr", false),
 
+    HIVEOPTIMIZEMETADATAQUERIES("hive.compute.query.using.stats", false),
+
     // Serde for FetchTask
     HIVEFETCHOUTPUTSERDE("hive.fetch.output.serde", "org.apache.hadoop.hive.serde2.DelimitedJSONSerDe"),
 
@@ -734,7 +741,8 @@ public class HiveConf extends Configurat
     HIVE_ENTITY_SEPARATOR("hive.entity.separator", "@"),
 
     // binary or http
-    HIVE_SERVER2_TRANSPORT_MODE("hive.server2.transport.mode", "binary"),
+    HIVE_SERVER2_TRANSPORT_MODE("hive.server2.transport.mode", "binary",
+        new StringsValidator("binary", "http")),
 
     // http (over thrift) transport settings
     HIVE_SERVER2_THRIFT_HTTP_PORT("hive.server2.thrift.http.port", 10001),
@@ -745,7 +753,8 @@ public class HiveConf extends Configurat
     // binary transport settings
     HIVE_SERVER2_THRIFT_PORT("hive.server2.thrift.port", 10000),
     HIVE_SERVER2_THRIFT_BIND_HOST("hive.server2.thrift.bind.host", ""),
-    HIVE_SERVER2_THRIFT_SASL_QOP("hive.server2.thrift.sasl.qop", "auth"),
+    HIVE_SERVER2_THRIFT_SASL_QOP("hive.server2.thrift.sasl.qop", "auth",
+        new StringsValidator("auth", "auth-int", "auth-conf")),
     HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS("hive.server2.thrift.min.worker.threads", 5),
     HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS("hive.server2.thrift.max.worker.threads", 500),
 
@@ -757,7 +766,8 @@ public class HiveConf extends Configurat
 
 
     // HiveServer2 auth configuration
-    HIVE_SERVER2_AUTHENTICATION("hive.server2.authentication", "NONE"),
+    HIVE_SERVER2_AUTHENTICATION("hive.server2.authentication", "NONE",
+        new StringsValidator("NOSASL", "NONE", "LDAP", "KERBEROS", "CUSTOM")),
     HIVE_SERVER2_KERBEROS_KEYTAB("hive.server2.authentication.kerberos.keytab", ""),
     HIVE_SERVER2_KERBEROS_PRINCIPAL("hive.server2.authentication.kerberos.principal", ""),
     HIVE_SERVER2_PLAIN_LDAP_URL("hive.server2.authentication.ldap.url", null),
@@ -765,7 +775,8 @@ public class HiveConf extends Configurat
     HIVE_SERVER2_PLAIN_LDAP_DOMAIN("hive.server2.authentication.ldap.Domain", null),
     HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS("hive.server2.custom.authentication.class", null),
     HIVE_SERVER2_ENABLE_DOAS("hive.server2.enable.doAs", true),
-    HIVE_SERVER2_TABLE_TYPE_MAPPING("hive.server2.table.type.mapping", "CLASSIC"),
+    HIVE_SERVER2_TABLE_TYPE_MAPPING("hive.server2.table.type.mapping", "CLASSIC",
+        new StringsValidator("CLASSIC", "HIVE")),
     HIVE_SERVER2_SESSION_HOOK("hive.server2.session.hook", ""),
 
     HIVE_SECURITY_COMMAND_WHITELIST("hive.security.command.whitelist", "set,reset,dfs,add,delete,compile"),
@@ -836,7 +847,13 @@ public class HiveConf extends Configurat
 
     private final VarType type;
 
+    private final Validator validator;
+
     ConfVars(String varname, String defaultVal) {
+      this(varname, defaultVal, null);
+    }
+
+    ConfVars(String varname, String defaultVal, Validator validator) {
       this.varname = varname;
       this.valClass = String.class;
       this.defaultVal = defaultVal;
@@ -845,9 +862,14 @@ public class HiveConf extends Configurat
       this.defaultFloatVal = -1;
       this.defaultBoolVal = false;
       this.type = VarType.STRING;
+      this.validator = validator;
     }
 
-    ConfVars(String varname, int defaultIntVal) {
+    ConfVars(String varname, int defaultVal) {
+      this(varname, defaultVal, null);
+    }
+
+    ConfVars(String varname, int defaultIntVal, Validator validator) {
       this.varname = varname;
       this.valClass = Integer.class;
       this.defaultVal = Integer.toString(defaultIntVal);
@@ -856,9 +878,14 @@ public class HiveConf extends Configurat
       this.defaultFloatVal = -1;
       this.defaultBoolVal = false;
       this.type = VarType.INT;
+      this.validator = validator;
+    }
+
+    ConfVars(String varname, long defaultVal) {
+      this(varname, defaultVal, null);
     }
 
-    ConfVars(String varname, long defaultLongVal) {
+    ConfVars(String varname, long defaultLongVal, Validator validator) {
       this.varname = varname;
       this.valClass = Long.class;
       this.defaultVal = Long.toString(defaultLongVal);
@@ -867,9 +894,14 @@ public class HiveConf extends Configurat
       this.defaultFloatVal = -1;
       this.defaultBoolVal = false;
       this.type = VarType.LONG;
+      this.validator = validator;
     }
 
-    ConfVars(String varname, float defaultFloatVal) {
+    ConfVars(String varname, float defaultVal) {
+      this(varname, defaultVal, null);
+    }
+
+    ConfVars(String varname, float defaultFloatVal, Validator validator) {
       this.varname = varname;
       this.valClass = Float.class;
       this.defaultVal = Float.toString(defaultFloatVal);
@@ -878,6 +910,7 @@ public class HiveConf extends Configurat
       this.defaultFloatVal = defaultFloatVal;
       this.defaultBoolVal = false;
       this.type = VarType.FLOAT;
+      this.validator = validator;
     }
 
     ConfVars(String varname, boolean defaultBoolVal) {
@@ -889,12 +922,17 @@ public class HiveConf extends Configurat
       this.defaultFloatVal = -1;
       this.defaultBoolVal = defaultBoolVal;
       this.type = VarType.BOOLEAN;
+      this.validator = null;
     }
 
     public boolean isType(String value) {
       return type.isType(value);
     }
 
+    public String validate(String value) {
+      return validator == null ? null : validator.validate(value);
+    }
+
     public String typeString() {
       return type.typeString();
     }
@@ -1294,4 +1332,41 @@ public class HiveConf extends Configurat
     }
   }
 
+  /**
+   * validate value for a ConfVar, return non-null string for fail message
+   */
+  public static interface Validator {
+    String validate(String value);
+  }
+
+  public static class StringsValidator implements Validator {
+    private final Set<String> expected = new LinkedHashSet<String>();
+    private StringsValidator(String... values) {
+      for (String value : values) {
+        expected.add(value.toLowerCase());
+      }
+    }
+    @Override
+    public String validate(String value) {
+      if (value == null || !expected.contains(value.toLowerCase())) {
+        return "Invalid value.. expects one of " + expected;
+      }
+      return null;
+    }
+  }
+
+  public static class RatioValidator implements Validator {
+    @Override
+    public String validate(String value) {
+      try {
+        float fvalue = Float.valueOf(value);
+        if (fvalue <= 0 || fvalue >= 1) {
+          return "Invalid ratio " + value + ", which should be in between 0 to 1";
+        }
+      } catch (NumberFormatException e) {
+        return e.toString();
+      }
+      return null;
+    }
+  }
 }

Modified: hive/branches/tez/common/src/scripts/saveVersion.sh
URL: http://svn.apache.org/viewvc/hive/branches/tez/common/src/scripts/saveVersion.sh?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/common/src/scripts/saveVersion.sh (original)
+++ hive/branches/tez/common/src/scripts/saveVersion.sh Tue Nov  5 07:01:32 2013
@@ -33,7 +33,7 @@ dir=`pwd`
 cwd=`dirname $dir`
 if [ "$revision" = "" ]; then
     if git rev-parse HEAD 2>/dev/null > /dev/null ; then
-        revision=`git log -1 --pretty=format:"%H" ../`
+        revision=`git log -1 --pretty=format:"%H"`
         hostname=`hostname`
         branch=`git branch | sed -n -e 's/^* //p'`
         url="git://${hostname}${cwd}"
@@ -57,7 +57,13 @@ if [ "$url" = "" ]; then
     url="file://$cwd"
 fi
 
-srcChecksum=`find ../ -name '*.java' | grep -v generated-sources | LC_ALL=C sort | xargs md5sum | md5sum | cut -d ' ' -f 1`
+if [ -x /sbin/md5 ]; then
+  md5="/sbin/md5"
+else
+  md5="md5sum"
+fi
+
+srcChecksum=`find ../ -name '*.java' | grep -v generated-sources | LC_ALL=C sort | xargs $md5 | $md5 | cut -d ' ' -f 1`
 
 mkdir -p $src_dir/gen/org/apache/hive/common
 

Modified: hive/branches/tez/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java (original)
+++ hive/branches/tez/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java Tue Nov  5 07:01:32 2013
@@ -44,7 +44,7 @@ public class TestHiveLogging extends Tes
     process = null;
   }
 
-  private void configLog(String hiveLog4jTest, String hiveExecLog4jTest) 
+  private void configLog(String hiveLog4jTest, String hiveExecLog4jTest)
   throws Exception {
     String expectedLog4jTestPath = HiveTestUtils.getFileFromClasspath(hiveLog4jTest);
     String expectedLog4jExecPath = HiveTestUtils.getFileFromClasspath(hiveExecLog4jTest);

Modified: hive/branches/tez/common/src/test/resources/hive-exec-log4j-test.properties
URL: http://svn.apache.org/viewvc/hive/branches/tez/common/src/test/resources/hive-exec-log4j-test.properties?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/common/src/test/resources/hive-exec-log4j-test.properties (original)
+++ hive/branches/tez/common/src/test/resources/hive-exec-log4j-test.properties Tue Nov  5 07:01:32 2013
@@ -53,3 +53,7 @@ log4j.category.JPOX.Query=ERROR,FA
 log4j.category.JPOX.General=ERROR,FA
 log4j.category.JPOX.Enhancer=ERROR,FA
 
+
+# Silence useless ZK logs
+log4j.logger.org.apache.zookeeper.server.NIOServerCnxn=WARN,FA
+log4j.logger.org.apache.zookeeper.ClientCnxnSocketNIO=WARN,FA

Modified: hive/branches/tez/common/src/test/resources/hive-log4j-test.properties
URL: http://svn.apache.org/viewvc/hive/branches/tez/common/src/test/resources/hive-log4j-test.properties?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/common/src/test/resources/hive-log4j-test.properties (original)
+++ hive/branches/tez/common/src/test/resources/hive-log4j-test.properties Tue Nov  5 07:01:32 2013
@@ -65,3 +65,7 @@ log4j.category.JPOX.Query=ERROR,DRFA
 log4j.category.JPOX.General=ERROR,DRFA
 log4j.category.JPOX.Enhancer=ERROR,DRFA
 
+
+# Silence useless ZK logs
+log4j.logger.org.apache.zookeeper.server.NIOServerCnxn=WARN,DRFA
+log4j.logger.org.apache.zookeeper.ClientCnxnSocketNIO=WARN,DRFA

Modified: hive/branches/tez/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/branches/tez/conf/hive-default.xml.template?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/conf/hive-default.xml.template (original)
+++ hive/branches/tez/conf/hive-default.xml.template Tue Nov  5 07:01:32 2013
@@ -2041,6 +2041,17 @@
 </property>
 
 <property>
+  <name>hive.compute.query.using.stats</name>
+  <value>false</value>
+  <description>
+  When set to true hive will answer few queries like count(1) purely using stats
+  stored in metastore. For basic stats collection turn on the config hive.stats.autogather to true.
+  For more advanced stats collection need to run analyze table queries.
+  </description>
+</property>
+
+
+<property>
   <name>hive.metastore.schema.verification</name>
   <value>false</value>
    <description>

Modified: hive/branches/tez/contrib/src/java/org/apache/hadoop/hive/contrib/metastore/hooks/TestURLHook.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/java/org/apache/hadoop/hive/contrib/metastore/hooks/TestURLHook.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/java/org/apache/hadoop/hive/contrib/metastore/hooks/TestURLHook.java (original)
+++ hive/branches/tez/contrib/src/java/org/apache/hadoop/hive/contrib/metastore/hooks/TestURLHook.java Tue Nov  5 07:01:32 2013
@@ -33,7 +33,7 @@ public class TestURLHook implements JDOC
   public String getJdoConnectionUrl(Configuration conf) throws Exception {
     if (originalUrl == null) {
       originalUrl = conf.get(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, "");
-      return "jdbc:derby:;databaseName=../build/test/junit_metastore_db_blank;create=true";
+      return "jdbc:derby:;databaseName=target/tmp/junit_metastore_db_blank;create=true";
     } else {
       return originalUrl;
     }

Modified: hive/branches/tez/contrib/src/test/queries/clientnegative/case_with_row_sequence.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientnegative/case_with_row_sequence.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientnegative/case_with_row_sequence.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientnegative/case_with_row_sequence.q Tue Nov  5 07:01:32 2013
@@ -1,6 +1,6 @@
 drop temporary function row_sequence;
 
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 create temporary function row_sequence as 
 'org.apache.hadoop.hive.contrib.udf.UDFRowSequence';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientnegative/invalid_row_sequence.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientnegative/invalid_row_sequence.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientnegative/invalid_row_sequence.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientnegative/invalid_row_sequence.q Tue Nov  5 07:01:32 2013
@@ -2,7 +2,7 @@
 
 drop temporary function row_sequence;
 
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 create temporary function row_sequence as 
 'org.apache.hadoop.hive.contrib.udf.UDFRowSequence';

Modified: hive/branches/tez/contrib/src/test/queries/clientnegative/serde_regex.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientnegative/serde_regex.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientnegative/serde_regex.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientnegative/serde_regex.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 USE default;
 

Modified: hive/branches/tez/contrib/src/test/queries/clientnegative/udtf_explode2.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientnegative/udtf_explode2.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientnegative/udtf_explode2.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientnegative/udtf_explode2.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION explode2 AS 'org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFExplode2';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientnegative/url_hook.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientnegative/url_hook.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientnegative/url_hook.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientnegative/url_hook.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 set hive.metastore.force.reload.conf=true;
 SHOW TABLES 'src';
 set hive.metastore.ds.connection.url.hook=org.apache.hadoop.hive.contrib.metastore.hooks.TestURLHook;

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/dboutput.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/dboutput.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/dboutput.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/dboutput.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-ADD JAR ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION dboutput AS 'org.apache.hadoop.hive.contrib.genericudf.example.GenericUDFDBOutput';
 
@@ -7,7 +7,7 @@ set mapred.reduce.tasks.speculative.exec
 set mapred.map.tasks=1;
 set mapred.reduce.tasks=1;
 
-ADD JAR ${system:build.ivy.lib.dir}/default/derby-${system:derby.version}.jar;
+ADD JAR ${system:maven.local.repository}/org/apache/derby/derby/${system:derby.version}/derby-${system:derby.version}.jar;
 
 DESCRIBE FUNCTION dboutput;
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/fileformat_base64.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/fileformat_base64.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/fileformat_base64.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/fileformat_base64.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 DROP TABLE base64_test;
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/java_mr_example.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/java_mr_example.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/java_mr_example.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/java_mr_example.q Tue Nov  5 07:01:32 2013
@@ -1,10 +1,10 @@
 FROM (
   FROM src
    MAP value, key
- USING 'java -cp ${system:build.dir}/hive-contrib-${system:hive.version}.jar org.apache.hadoop.hive.contrib.mr.example.IdentityMapper'
+ USING 'java -cp ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar org.apache.hadoop.hive.contrib.mr.example.IdentityMapper'
     AS k, v
  CLUSTER BY k) map_output
   REDUCE k, v
-   USING 'java -cp ${system:build.dir}/hive-contrib-${system:hive.version}.jar org.apache.hadoop.hive.contrib.mr.example.WordCountReduce'
+   USING 'java -cp ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar org.apache.hadoop.hive.contrib.mr.example.WordCountReduce'
    AS k, v
 ;
\ No newline at end of file

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/lateral_view_explode2.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/lateral_view_explode2.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/lateral_view_explode2.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/lateral_view_explode2.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION explode2 AS 'org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFExplode2';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/serde_regex.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/serde_regex.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/serde_regex.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/serde_regex.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 EXPLAIN
 CREATE TABLE serde_regex(
@@ -35,7 +35,7 @@ WITH SERDEPROPERTIES (
 )
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH "../data/files/apache.access.log" INTO TABLE serde_regex;
-LOAD DATA LOCAL INPATH "../data/files/apache.access.2.log" INTO TABLE serde_regex;
+LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex;
+LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex;
 
 SELECT * FROM serde_regex ORDER BY time;
\ No newline at end of file

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/serde_s3.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/serde_s3.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/serde_s3.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/serde_s3.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 DROP TABLE s3log;
 CREATE TABLE s3log
@@ -7,7 +7,7 @@ STORED AS TEXTFILE;
 
 DESCRIBE s3log;
 
-LOAD DATA LOCAL INPATH '../contrib/data/files/s3.log' INTO TABLE s3log;
+LOAD DATA LOCAL INPATH '../../contrib/data/files/s3.log' INTO TABLE s3log;
 
 SELECT a.* FROM s3log a;
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 drop table dest1;
 CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes2.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes2.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes2.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes2.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 drop table dest1;
 CREATE TABLE dest1(key SMALLINT, value STRING) STORED AS TEXTFILE;

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes3.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes3.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes3.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes3.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 drop table dest1;
 CREATE TABLE dest1(key STRING, value STRING) STORED AS TEXTFILE;

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes4.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes4.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes4.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes4.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 drop table dest1;
 CREATE TABLE dest1(key STRING, value STRING) STORED AS TEXTFILE;

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes5.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes5.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes5.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes5.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 drop table dest1;
 CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;
@@ -8,7 +8,7 @@ FROM (
   FROM src
   SELECT TRANSFORM(src.key, src.value) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDWRITER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordWriter'
-  USING 'python ../data/scripts/cat.py'
+  USING 'python ../../data/scripts/cat.py'
   AS (tkey, tvalue) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader'
 ) tmap
@@ -18,7 +18,7 @@ FROM (
   FROM src
   SELECT TRANSFORM(src.key, src.value) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDWRITER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordWriter'
-  USING 'python ../data/scripts/cat.py'
+  USING 'python ../../data/scripts/cat.py'
   AS (tkey, tvalue) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader'
 ) tmap

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes_null.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes_null.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes_null.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/serde_typedbytes_null.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 DROP TABLE table1;
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_avg.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_avg.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_avg.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_avg.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION example_avg AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleAvg';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_group_concat.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_group_concat.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_group_concat.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_group_concat.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION example_group_concat AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleGroupConcat';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_max.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_max.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_max.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_max.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION example_max AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_max_n.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_max_n.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_max_n.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_max_n.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION example_max_n AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMaxN';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_min.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_min.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_min.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_min.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION example_min AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMin';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_min_n.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_min_n.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_min_n.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udaf_example_min_n.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 CREATE TEMPORARY FUNCTION example_min_n AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMinN';
 
 EXPLAIN

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_add.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_add.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_add.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_add.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION example_add AS 'org.apache.hadoop.hive.contrib.udf.example.UDFExampleAdd';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_arraymapstruct.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_arraymapstruct.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_arraymapstruct.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_arraymapstruct.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION example_arraysum    AS 'org.apache.hadoop.hive.contrib.udf.example.UDFExampleArraySum';
 CREATE TEMPORARY FUNCTION example_mapconcat   AS 'org.apache.hadoop.hive.contrib.udf.example.UDFExampleMapConcat';

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_format.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_format.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_format.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udf_example_format.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION example_format AS 'org.apache.hadoop.hive.contrib.udf.example.UDFExampleFormat';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udf_row_sequence.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udf_row_sequence.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udf_row_sequence.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udf_row_sequence.q Tue Nov  5 07:01:32 2013
@@ -4,7 +4,7 @@
 
 drop temporary function row_sequence;
 
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 create temporary function row_sequence as 
 'org.apache.hadoop.hive.contrib.udf.UDFRowSequence';

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udtf_explode2.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udtf_explode2.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udtf_explode2.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udtf_explode2.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION explode2 AS 'org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFExplode2';
 

Modified: hive/branches/tez/contrib/src/test/queries/clientpositive/udtf_output_on_close.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/queries/clientpositive/udtf_output_on_close.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/queries/clientpositive/udtf_output_on_close.q (original)
+++ hive/branches/tez/contrib/src/test/queries/clientpositive/udtf_output_on_close.q Tue Nov  5 07:01:32 2013
@@ -1,4 +1,4 @@
-add jar ${system:build.dir}/hive-contrib-${system:hive.version}.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;
 
 CREATE TEMPORARY FUNCTION udtfCount2 AS 'org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFCount2';
 

Modified: hive/branches/tez/contrib/src/test/results/clientpositive/serde_regex.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/results/clientpositive/serde_regex.q.out?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/results/clientpositive/serde_regex.q.out (original)
+++ hive/branches/tez/contrib/src/test/results/clientpositive/serde_regex.q.out Tue Nov  5 07:01:32 2013
@@ -92,16 +92,16 @@ WITH SERDEPROPERTIES (
 STORED AS TEXTFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: default@serde_regex
-PREHOOK: query: LOAD DATA LOCAL INPATH "../data/files/apache.access.log" INTO TABLE serde_regex
+PREHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex
 PREHOOK: type: LOAD
 PREHOOK: Output: default@serde_regex
-POSTHOOK: query: LOAD DATA LOCAL INPATH "../data/files/apache.access.log" INTO TABLE serde_regex
+POSTHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex
 POSTHOOK: type: LOAD
 POSTHOOK: Output: default@serde_regex
-PREHOOK: query: LOAD DATA LOCAL INPATH "../data/files/apache.access.2.log" INTO TABLE serde_regex
+PREHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex
 PREHOOK: type: LOAD
 PREHOOK: Output: default@serde_regex
-POSTHOOK: query: LOAD DATA LOCAL INPATH "../data/files/apache.access.2.log" INTO TABLE serde_regex
+POSTHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex
 POSTHOOK: type: LOAD
 POSTHOOK: Output: default@serde_regex
 PREHOOK: query: SELECT * FROM serde_regex ORDER BY time

Modified: hive/branches/tez/contrib/src/test/results/clientpositive/serde_s3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/results/clientpositive/serde_s3.q.out?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/results/clientpositive/serde_s3.q.out (original)
+++ hive/branches/tez/contrib/src/test/results/clientpositive/serde_s3.q.out Tue Nov  5 07:01:32 2013
@@ -32,10 +32,10 @@ totaltime           	int                
 turnaroundtime      	int                 	from deserializer   
 referer             	string              	from deserializer   
 useragent           	string              	from deserializer   
-PREHOOK: query: LOAD DATA LOCAL INPATH '../contrib/data/files/s3.log' INTO TABLE s3log
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../contrib/data/files/s3.log' INTO TABLE s3log
 PREHOOK: type: LOAD
 PREHOOK: Output: default@s3log
-POSTHOOK: query: LOAD DATA LOCAL INPATH '../contrib/data/files/s3.log' INTO TABLE s3log
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../contrib/data/files/s3.log' INTO TABLE s3log
 POSTHOOK: type: LOAD
 POSTHOOK: Output: default@s3log
 PREHOOK: query: SELECT a.* FROM s3log a

Modified: hive/branches/tez/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out (original)
+++ hive/branches/tez/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out Tue Nov  5 07:01:32 2013
@@ -12,7 +12,7 @@ FROM (
   FROM src
   SELECT TRANSFORM(src.key, src.value) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDWRITER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordWriter'
-  USING 'python ../data/scripts/cat.py'
+  USING 'python ../../data/scripts/cat.py'
   AS (tkey, tvalue) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader'
 ) tmap
@@ -23,14 +23,14 @@ FROM (
   FROM src
   SELECT TRANSFORM(src.key, src.value) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDWRITER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordWriter'
-  USING 'python ../data/scripts/cat.py'
+  USING 'python ../../data/scripts/cat.py'
   AS (tkey, tvalue) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader'
 ) tmap
 INSERT OVERWRITE TABLE dest1 SELECT tkey, tvalue
 POSTHOOK: type: QUERY
 ABSTRACT SYNTAX TREE:
-  (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TRANSFORM (TOK_EXPLIST (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL src) value)) (TOK_SERDE (TOK_SERDENAME 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe')) (TOK_RECORDWRITER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordWriter') 'python ../data/scripts/cat.py' (TOK_SERDE (TOK_SERDENAME 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe')) (TOK_RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader') (TOK_ALIASLIST tkey tvalue)))))) tmap)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest1))) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL tkey)) (TOK_SELEXPR (TOK_TABLE_OR_COL tvalue)))))
+  (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TRANSFORM (TOK_EXPLIST (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL src) value)) (TOK_SERDE (TOK_SERDENAME 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe')) (TOK_RECORDWRITER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordWriter') 'python ../../data/scripts/cat.py' (TOK_SERDE (TOK_SERDENAME 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe')) (TOK_RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader') (TOK_ALIASLIST tkey tvalue)))))) tmap)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest1))) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL tkey)) (TOK_SELEXPR (TOK_TABLE_OR_COL tvalue)))))
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -57,7 +57,7 @@ STAGE PLANS:
                     type: string
               outputColumnNames: _col0, _col1
               Transform Operator
-                command: python ../data/scripts/cat.py
+                command: python ../../data/scripts/cat.py
                 output info:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -139,7 +139,7 @@ PREHOOK: query: FROM (
   FROM src
   SELECT TRANSFORM(src.key, src.value) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDWRITER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordWriter'
-  USING 'python ../data/scripts/cat.py'
+  USING 'python ../../data/scripts/cat.py'
   AS (tkey, tvalue) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader'
 ) tmap
@@ -151,7 +151,7 @@ POSTHOOK: query: FROM (
   FROM src
   SELECT TRANSFORM(src.key, src.value) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDWRITER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordWriter'
-  USING 'python ../data/scripts/cat.py'
+  USING 'python ../../data/scripts/cat.py'
   AS (tkey, tvalue) ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe'
   RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader'
 ) tmap

Modified: hive/branches/tez/data/files/symlink1.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/data/files/symlink1.txt?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/data/files/symlink1.txt (original)
+++ hive/branches/tez/data/files/symlink1.txt Tue Nov  5 07:01:32 2013
@@ -1,2 +1,2 @@
-../data/files/T1.txt
-../data/files/T3.txt
+../../data/files/T1.txt
+../../data/files/T3.txt

Modified: hive/branches/tez/data/files/symlink2.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/data/files/symlink2.txt?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/data/files/symlink2.txt (original)
+++ hive/branches/tez/data/files/symlink2.txt Tue Nov  5 07:01:32 2013
@@ -1 +1 @@
-../data/files/T2.txt
+../../data/files/T2.txt

Modified: hive/branches/tez/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsUtils.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsUtils.java (original)
+++ hive/branches/tez/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStatsUtils.java Tue Nov  5 07:01:32 2013
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hive.ql.stats.StatsSetupConst;
+import org.apache.hadoop.hive.common.StatsSetupConst;
 
 
 

Modified: hive/branches/tez/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q
URL: http://svn.apache.org/viewvc/hive/branches/tez/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q (original)
+++ hive/branches/tez/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q Tue Nov  5 07:01:32 2013
@@ -16,11 +16,11 @@ STORED BY 'org.apache.hadoop.hive.hbase.
 WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string")
 TBLPROPERTIES ("hbase.table.name" = "hbase_table_0");
 
-dfs -ls ../build/ql/tmp/hbase/hbase_table_0;
+dfs -ls target/tmp/hbase/hbase_table_0;
 
 DROP DATABASE IF EXISTS hbaseDB CASCADE;
 
-dfs -ls ../build/ql/tmp/hbase/hbase_table_0;
+dfs -ls target/tmp/hbase/hbase/hbase_table_0;
 
 
 

Modified: hive/branches/tez/hbase-handler/src/test/results/positive/hbase_stats.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/hbase-handler/src/test/results/positive/hbase_stats.q.out?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hbase-handler/src/test/results/positive/hbase_stats.q.out (original)
+++ hive/branches/tez/hbase-handler/src/test/results/positive/hbase_stats.q.out Tue Nov  5 07:01:32 2013
@@ -43,7 +43,6 @@ Retention:          	0                  
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
 	numFiles            	1                   
-	numPartitions       	0                   
 	numRows             	500                 
 	rawDataSize         	5312                
 	totalSize           	5812                
@@ -175,11 +174,6 @@ Retention:          	0                  
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	numFiles            	3                   
-	numPartitions       	3                   
-	numRows             	1500                
-	rawDataSize         	15936               
-	totalSize           	17436               
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -345,11 +339,6 @@ Retention:          	0                  
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	numFiles            	3                   
-	numPartitions       	3                   
-	numRows             	1500                
-	rawDataSize         	15936               
-	totalSize           	17436               
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 

Modified: hive/branches/tez/hbase-handler/src/test/results/positive/hbase_stats2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/tez/hbase-handler/src/test/results/positive/hbase_stats2.q.out?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hbase-handler/src/test/results/positive/hbase_stats2.q.out (original)
+++ hive/branches/tez/hbase-handler/src/test/results/positive/hbase_stats2.q.out Tue Nov  5 07:01:32 2013
@@ -43,7 +43,6 @@ Retention:          	0                  
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
 	numFiles            	1                   
-	numPartitions       	0                   
 	numRows             	500                 
 	rawDataSize         	5312                
 	totalSize           	5812                
@@ -175,11 +174,6 @@ Retention:          	0                  
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	numFiles            	3                   
-	numPartitions       	3                   
-	numRows             	1500                
-	rawDataSize         	15936               
-	totalSize           	17436               
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -345,11 +339,6 @@ Retention:          	0                  
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	numFiles            	3                   
-	numPartitions       	3                   
-	numRows             	1500                
-	rawDataSize         	15936               
-	totalSize           	17436               
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatMultiOutputFormat.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatMultiOutputFormat.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatMultiOutputFormat.java Tue Nov  5 07:01:32 2013
@@ -108,6 +108,7 @@ public class TestHCatMultiOutputFormat {
         HiveMetaStore.main(new String[]{"-v", "-p", msPort, "--hiveconf", warehouseConf});
       } catch (Throwable t) {
         System.err.println("Exiting. Got exception from metastore: " + t.getMessage());
+        t.printStackTrace();
       }
     }
 
@@ -163,13 +164,14 @@ public class TestHCatMultiOutputFormat {
 
   @BeforeClass
   public static void setup() throws Exception {
-    String testDir = System.getProperty("test.data.dir", "./");
+    System.clearProperty("mapred.job.tracker");
+    String testDir = System.getProperty("test.tmp.dir", "./");
     testDir = testDir + "/test_multitable_" + Math.abs(new Random().nextLong()) + "/";
     workDir = new File(new File(testDir).getCanonicalPath());
     FileUtil.fullyDelete(workDir);
     workDir.mkdirs();
 
-    warehousedir = new Path(workDir + "/warehouse");
+    warehousedir = new Path(System.getProperty("test.warehouse.dir"));
 
     // Run hive metastore server
     t = new Thread(new RunMS());
@@ -186,9 +188,10 @@ public class TestHCatMultiOutputFormat {
     mrCluster = new MiniMRCluster(1, fs.getUri().toString(), 1, null, null,
       new JobConf(conf));
     mrConf = mrCluster.createJobConf();
-    fs.mkdirs(warehousedir);
 
     initializeSetup();
+
+    warehousedir.getFileSystem(hiveConf).mkdirs(warehousedir);
   }
 
   private static void initializeSetup() throws Exception {
@@ -251,14 +254,15 @@ public class TestHCatMultiOutputFormat {
     tbl.setPartitionKeys(ColumnHolder.partitionCols);
 
     hmsc.createTable(tbl);
-    FileSystem fs = FileSystem.get(mrConf);
-    fs.setPermission(new Path(warehousedir, tableName), new FsPermission(tablePerm));
+    Path path = new Path(warehousedir, tableName);
+    FileSystem fs = path.getFileSystem(hiveConf);
+    fs.setPermission(path, new FsPermission(tablePerm));
   }
 
   @AfterClass
   public static void tearDown() throws IOException {
     FileUtil.fullyDelete(workDir);
-    FileSystem fs = FileSystem.get(mrConf);
+    FileSystem fs = warehousedir.getFileSystem(hiveConf);
     if (fs.exists(warehousedir)) {
       fs.delete(warehousedir, true);
     }
@@ -367,14 +371,14 @@ public class TestHCatMultiOutputFormat {
    * @throws Exception if any error occurs
    */
   private List<String> getTableData(String table, String database) throws Exception {
-    HiveConf conf = new HiveConf();
-    conf.addResource("hive-site.xml");
     ArrayList<String> results = new ArrayList<String>();
     ArrayList<String> temp = new ArrayList<String>();
-    Hive hive = Hive.get(conf);
+    Hive hive = Hive.get(hiveConf);
     org.apache.hadoop.hive.ql.metadata.Table tbl = hive.getTable(database, table);
     FetchWork work;
-    if (!tbl.getPartCols().isEmpty()) {
+    if (tbl.getPartCols().isEmpty()) {
+      work = new FetchWork(tbl.getDataLocation().toString(), Utilities.getTableDesc(tbl));
+    } else {
       List<Partition> partitions = hive.getPartitions(tbl);
       List<PartitionDesc> partDesc = new ArrayList<PartitionDesc>();
       List<String> partLocs = new ArrayList<String>();
@@ -384,12 +388,10 @@ public class TestHCatMultiOutputFormat {
       }
       work = new FetchWork(partLocs, partDesc, Utilities.getTableDesc(tbl));
       work.setLimit(100);
-    } else {
-      work = new FetchWork(tbl.getDataLocation().toString(), Utilities.getTableDesc(tbl));
     }
     FetchTask task = new FetchTask();
     task.setWork(work);
-    task.initialize(conf, null, null);
+    task.initialize(hiveConf, null, null);
     task.fetch(temp);
     for (String str : temp) {
       results.add(str.replace("\t", ","));

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitionPublish.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitionPublish.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitionPublish.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitionPublish.java Tue Nov  5 07:01:32 2013
@@ -51,6 +51,7 @@ import org.apache.hadoop.mapred.MiniMRCl
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.util.Shell;
 import org.apache.hcatalog.NoExitSecurityManager;
 import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
 import org.apache.hcatalog.data.DefaultHCatRecord;
@@ -74,10 +75,10 @@ public class TestHCatPartitionPublish {
   private static HiveConf hcatConf;
   private static HiveMetaStoreClient msc;
   private static SecurityManager securityManager;
+  private static Configuration conf = new Configuration(true);
 
   @BeforeClass
   public static void setup() throws Exception {
-    Configuration conf = new Configuration(true);
     conf.set("yarn.scheduler.capacity.root.queues", "default");
     conf.set("yarn.scheduler.capacity.root.default.capacity", "100");
 
@@ -150,10 +151,13 @@ public class TestHCatPartitionPublish {
     Assert.assertEquals(0, ptns.size());
     Table table = msc.getTable(dbName, tableName);
     Assert.assertTrue(table != null);
-    // Also make sure that the directory has been deleted in the table
-    // location.
-    Assert.assertFalse(fs.exists(new Path(table.getSd().getLocation()
-        + "/part1=p1value1/part0=p0value1")));
+    // In Windows, we cannot remove the output directory when job fail. See
+    // FileOutputCommitterContainer.abortJob
+    if (!Shell.WINDOWS) {
+      Path path = new Path(table.getSd().getLocation()
+          + "/part1=p1value1/part0=p0value1");
+      Assert.assertFalse(path.getFileSystem(conf).exists(path));
+    }
   }
 
   void runMRCreateFail(

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestMultiOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestMultiOutputFormat.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestMultiOutputFormat.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestMultiOutputFormat.java Tue Nov  5 07:01:32 2013
@@ -88,7 +88,7 @@ public class TestMultiOutputFormat {
   }
 
   private static void createWorkDir() throws IOException {
-    String testDir = System.getProperty("test.data.dir", "./");
+    String testDir = System.getProperty("test.tmp.dir", "./");
     testDir = testDir + "/test_multiout_" + Math.abs(new Random().nextLong()) + "/";
     workDir = new File(new File(testDir).getCanonicalPath());
     FileUtil.fullyDelete(workDir);

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java Tue Nov  5 07:01:32 2013
@@ -68,7 +68,7 @@ public class TestRCFileMapReduceInputFor
   static {
     try {
       fs = FileSystem.getLocal(conf);
-      Path dir = new Path(System.getProperty("test.data.dir", ".") + "/mapred");
+      Path dir = new Path(System.getProperty("test.tmp.dir", ".") + "/mapred");
       file = new Path(dir, "test_rcfile");
       fs.delete(dir, true);
       // the SerDe part is from TestLazySimpleSerDe
@@ -203,7 +203,7 @@ public class TestRCFileMapReduceInputFor
   private void writeThenReadByRecordReader(int intervalRecordCount,
                        int writeCount, int splitNumber, long maxSplitSize, CompressionCodec codec)
     throws IOException, InterruptedException {
-    Path testDir = new Path(System.getProperty("test.data.dir", ".")
+    Path testDir = new Path(System.getProperty("test.tmp.dir", ".")
       + "/mapred/testsmallfirstsplit");
     Path testFile = new Path(testDir, "test_rcfile");
     fs.delete(testFile, true);

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/MiniCluster.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/MiniCluster.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/MiniCluster.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/MiniCluster.java Tue Nov  5 07:01:32 2013
@@ -62,7 +62,9 @@ public class MiniCluster {
       Configuration config = new Configuration();
 
       // Builds and starts the mini dfs and mapreduce clusters
-      System.setProperty("hadoop.log.dir", ".");
+      if(System.getProperty("hadoop.log.dir") == null) {
+        System.setProperty("hadoop.log.dir", "target/tmp/logs/");
+      }
       m_dfs = new MiniDFSCluster(config, dataNodes, true, null);
 
       m_fileSys = m_dfs.getFileSystem();

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java Tue Nov  5 07:01:32 2013
@@ -105,6 +105,7 @@ public class TestHCatMultiOutputFormat {
         HiveMetaStore.main(new String[]{"-v", "-p", msPort, "--hiveconf", warehouseConf});
       } catch (Throwable t) {
         System.err.println("Exiting. Got exception from metastore: " + t.getMessage());
+        t.printStackTrace();
       }
     }
 
@@ -160,13 +161,14 @@ public class TestHCatMultiOutputFormat {
 
   @BeforeClass
   public static void setup() throws Exception {
-    String testDir = System.getProperty("test.data.dir", "./");
+    System.clearProperty("mapred.job.tracker");
+    String testDir = System.getProperty("test.tmp.dir", "./");
     testDir = testDir + "/test_multitable_" + Math.abs(new Random().nextLong()) + "/";
     workDir = new File(new File(testDir).getCanonicalPath());
     FileUtil.fullyDelete(workDir);
     workDir.mkdirs();
 
-    warehousedir = new Path(workDir + "/warehouse");
+    warehousedir = new Path(System.getProperty("test.warehouse.dir"));
 
     // Run hive metastore server
     t = new Thread(new RunMS());
@@ -183,9 +185,10 @@ public class TestHCatMultiOutputFormat {
     mrCluster = new MiniMRCluster(1, fs.getUri().toString(), 1, null, null,
       new JobConf(conf));
     mrConf = mrCluster.createJobConf();
-    fs.mkdirs(warehousedir);
 
     initializeSetup();
+
+    warehousedir.getFileSystem(conf).mkdirs(warehousedir);
   }
 
   private static void initializeSetup() throws Exception {
@@ -248,14 +251,15 @@ public class TestHCatMultiOutputFormat {
     tbl.setPartitionKeys(ColumnHolder.partitionCols);
 
     hmsc.createTable(tbl);
-    FileSystem fs = FileSystem.get(mrConf);
-    fs.setPermission(new Path(warehousedir, tableName), new FsPermission(tablePerm));
+    Path path = new Path(warehousedir, tableName);
+    FileSystem fs = path.getFileSystem(hiveConf);
+    fs.setPermission(path, new FsPermission(tablePerm));
   }
 
   @AfterClass
   public static void tearDown() throws IOException {
     FileUtil.fullyDelete(workDir);
-    FileSystem fs = FileSystem.get(mrConf);
+    FileSystem fs = warehousedir.getFileSystem(hiveConf);
     if (fs.exists(warehousedir)) {
       fs.delete(warehousedir, true);
     }
@@ -312,7 +316,8 @@ public class TestHCatMultiOutputFormat {
     Assert.assertEquals("Comparing output of table " +
       tableNames[0] + " is not correct", outputs.get(0), "a,a,1,ag");
     Assert.assertEquals("Comparing output of table " +
-      tableNames[1] + " is not correct", outputs.get(1), "a,1,ag");
+      tableNames[1] + " is not correct", outputs.get(1),
+      "a,1,ag");
     Assert.assertEquals("Comparing output of table " +
       tableNames[2] + " is not correct", outputs.get(2), "a,a,extra,1,ag");
 

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java Tue Nov  5 07:01:32 2013
@@ -74,15 +74,15 @@ public class TestHCatPartitionPublish {
   private static HiveConf hcatConf;
   private static HiveMetaStoreClient msc;
   private static SecurityManager securityManager;
+  private static Configuration conf = new Configuration(true);
 
   @BeforeClass
   public static void setup() throws Exception {
-    String testDir = System.getProperty("test.data.dir", "./");
+    String testDir = System.getProperty("test.tmp.dir", "./");
     testDir = testDir + "/test_hcat_partitionpublish_" + Math.abs(new Random().nextLong()) + "/";
     File workDir = new File(new File(testDir).getCanonicalPath());
     FileUtil.fullyDelete(workDir);
     workDir.mkdirs();
-    Configuration conf = new Configuration(true);
     conf.set("yarn.scheduler.capacity.root.queues", "default");
     conf.set("yarn.scheduler.capacity.root.default.capacity", "100");
 
@@ -158,8 +158,9 @@ public class TestHCatPartitionPublish {
     // In Windows, we cannot remove the output directory when job fail. See
     // FileOutputCommitterContainer.abortJob
     if (!Shell.WINDOWS) {
-      Assert.assertFalse(fs.exists(new Path(table.getSd().getLocation()
-          + "/part1=p1value1/part0=p0value1")));
+      Path path = new Path(table.getSd().getLocation()
+          + "/part1=p1value1/part0=p0value1");
+      Assert.assertFalse(path.getFileSystem(conf).exists(path));
     }
   }
 

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestMultiOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestMultiOutputFormat.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestMultiOutputFormat.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestMultiOutputFormat.java Tue Nov  5 07:01:32 2013
@@ -85,7 +85,7 @@ public class TestMultiOutputFormat {
   }
 
   private static void createWorkDir() throws IOException {
-    String testDir = System.getProperty("test.data.dir", "./");
+    String testDir = System.getProperty("test.tmp.dir", "./");
     testDir = testDir + "/test_multiout_" + Math.abs(new Random().nextLong()) + "/";
     workDir = new File(new File(testDir).getCanonicalPath());
     FileUtil.fullyDelete(workDir);

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java Tue Nov  5 07:01:32 2013
@@ -68,7 +68,7 @@ public class TestRCFileMapReduceInputFor
   static {
     try {
       fs = FileSystem.getLocal(conf);
-      Path dir = new Path(System.getProperty("test.data.dir", ".") + "/mapred");
+      Path dir = new Path(System.getProperty("test.tmp.dir", ".") + "/mapred");
       file = new Path(dir, "test_rcfile");
       fs.delete(dir, true);
       // the SerDe part is from TestLazySimpleSerDe
@@ -203,7 +203,7 @@ public class TestRCFileMapReduceInputFor
   private void writeThenReadByRecordReader(int intervalRecordCount,
                        int writeCount, int splitNumber, long maxSplitSize, CompressionCodec codec)
     throws IOException, InterruptedException {
-    Path testDir = new Path(System.getProperty("test.data.dir", ".")
+    Path testDir = new Path(System.getProperty("test.tmp.dir", ".")
       + "/mapred/testsmallfirstsplit");
     Path testFile = new Path(testDir, "test_rcfile");
     fs.delete(testFile, true);

Modified: hive/branches/tez/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm Tue Nov  5 07:01:32 2013
@@ -647,19 +647,16 @@ sub compare
       foreach my $key (keys %$json_matches) {
         my $regex_expected_value = $json_matches->{$key};
         my $path = JSON::Path->new($key);
-        my $value; 
-        # when filter_job_status is defined 
-        if (defined $testCmd->{'filter_job_status'}) {
-	        # decode $testResult->{'body'} to an array of hash
-	        my $body = decode_json $testResult->{'body'};
-	        # in the tests, we run this case with jobName = "PigLatin:loadstore.pig"
-	        # filter $body to leave only records with this jobName
-	        my @filtered_body = grep {($_->{detail}{profile}{jobName} eq "PigLatin:loadstore.pig")}  @$body;
-			my @sorted_filtered_body = sort { $a->{id} cmp $b->{id} } @filtered_body;
-        	$value = $path->value(\@sorted_filtered_body);
+
+        # decode $testResult->{'body'} to an array of hash
+        my $body = decode_json $testResult->{'body'};
+        my @sorted_body;
+        if (ref @$body[0] eq 'HASH') {
+          @sorted_body = sort { $a->{id} cmp $b->{id} } @$body;
         } else {
-        	$value = $path->value($testResult->{'body'});
+          @sorted_body = sort { $a cmp $b } @$body;
         }
+        my $value = $path->value(\@sorted_body);
         
         if ($value !~ /$regex_expected_value/s) {
           print $log "$0::$subName INFO check failed:"

Modified: hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobstatus.conf
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobstatus.conf?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobstatus.conf (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobstatus.conf Tue Nov  5 07:01:32 2013
@@ -89,7 +89,6 @@ $cfg = 
      'json_path' => {'$[-1:].detail.status.username' => ':UNAME_OTHER:', '$[-2:].detail.status.username' => ':UNAME_OTHER:', '$[-3:].detail.status.username' => ':UNAME:',
                      '$[-4:].detail.status.username' => ':UNAME:', '$[-5:].detail.status.username' => ':UNAME_OTHER:', '$[-6:].detail.status.username' => ':UNAME_OTHER:'},
      'status_code' => 200,
-     'filter_job_status' => 1,	
     },
     {
      # GET jobs?user.name=UNAME_OTHER&fields=*, should get only jobs launched as UNAME_OTHER
@@ -134,11 +133,11 @@ $cfg = 
                      '$[-1:].detail.profile.jobId' => 'job_.*',
                      '$[-1:].detail.id' => 'job_.*',
                      '$[-1:].detail.parentId' => 'job_.*',
-                     '$[-1:].detail.percentComplete' => '100%',
-                     '$[-1:].detail.exitValue' => '0',
-                     '$[-1:].detail.user' => ':UNAME_OTHER:',
-                     '$[-1:].detail.callback' => '^.+$',
-                     '$[-1:].detail.completed' => 'done',
+                     '$[-2:].detail.exitValue' => '0',
+                     '$[-2:].detail.user' => ':UNAME_OTHER:',
+                     '$[-2:].detail.callback' => '^.+$',
+                     '$[-2:].detail.completed' => 'done',
+                     '$[-2:].detail.percentComplete' => '100%',
                     },
      'status_code' => 200,
     },
@@ -152,6 +151,36 @@ $cfg = 
      'json_path' => {'$[-1:]' => 'job_.*'},
      'status_code' => 200,
     },
+    {
+     # GET jobs?user.name=UNAME_OTHER&fields=*, get all the details of the oldest 2 jobs whose
+     # id is greater than job_0
+     'num' => 9,
+     'depends_on' => 'JOBS_1,JOBS_2,JOBS_3',
+     'method' => 'GET',
+     'url' => ':TEMPLETON_URL:/templeton/v1/jobs?user.name=:UNAME_OTHER:&fields=*&numrecords=2&jobid=job_0',
+     'format_header' => 'Content-Type: application/json',
+     'json_path' => {'$[-1:].id' => 'job_.*',
+                     '$[-1:].detail.status.jobId' => 'job_.*',
+                     '$[-1:].detail.status.runState' => '\\d+',
+                     '$[-1:].detail.status.jobId' => 'job_.*',
+                     '$[-1:].detail.status.jobComplete' => 'true',
+                     '$[-1:].detail.profile.user' => ':UNAME_OTHER:',
+                     '$[-1:].detail.profile.jobFile' => '^.+$',
+                     '$[-1:].detail.profile.url' => '^.+$',
+                     '$[-1:].detail.profile.queueName' => '^.+$',
+                     '$[-1:].detail.profile.jobID.id' => '\\d+',
+                     '$[-1:].detail.profile.jobID.jtIdentifier' => '\\d+',
+                     '$[-1:].detail.profile.jobId' => 'job_.*',
+                     '$[-1:].detail.id' => 'job_.*',
+                     '$[-1:].detail.parentId' => 'job_.*',
+                     '$[-1:].detail.percentComplete' => '100%',
+                     '$[-1:].detail.exitValue' => '0',
+                     '$[-1:].detail.user' => ':UNAME_OTHER:',
+                     '$[-1:].detail.callback' => '^.+$',
+                     '$[-1:].detail.completed' => 'done',
+                    },
+     'status_code' => 200,
+    },
 
    ]
   }

Modified: hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf Tue Nov  5 07:01:32 2013
@@ -441,10 +441,6 @@ $cfg = 
      'check_job_exit_value' => 1,
      'check_call_back' => 1,
     },
-
-   ]
-  },
-
     {
                                 #test add jar
      'num' => 11,
@@ -490,11 +486,8 @@ $cfg = 
      'check_job_exit_value' => 0,
      'check_call_back' => 1,
     },
-
-
-
-
-
+   ]
+  },
  ]
 },
   ;

Modified: hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission_streaming.conf
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission_streaming.conf?rev=1538880&r1=1538879&r2=1538880&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission_streaming.conf (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission_streaming.conf Tue Nov  5 07:01:32 2013
@@ -65,7 +65,7 @@ $cfg = 
                                 #results
      'status_code' => 200,
      'check_job_created' => 1,
-     'check_job_complete' => 'FAILURE',
+     'check_job_complete' => 'SUCCESS',
      'check_job_exit_value' => 1,
      'check_call_back' => 1,
     },
@@ -102,7 +102,7 @@ $cfg = 
                                 #results
      'status_code' => 200,
      'check_job_created' => 1,
-     'check_job_complete' => 'FAILURE',
+     'check_job_complete' => 'SUCCESS',
      'check_logs' => { 'job_num' => '1' },
      'check_job_exit_value' => 1,
      'check_call_back' => 1,