You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2013/10/29 16:25:24 UTC

svn commit: r1536755 - in /hive/trunk: ./ cli/src/test/org/apache/hadoop/hive/cli/ common/src/java/conf/ common/src/java/org/apache/hadoop/hive/common/ common/src/java/org/apache/hadoop/hive/conf/ common/src/java/org/apache/hive/common/util/ common/src...

Author: brock
Date: Tue Oct 29 15:25:23 2013
New Revision: 1536755

URL: http://svn.apache.org/r1536755
Log:
HIVE-5676 - Cleanup test cases as done during mavenization (Brock Noland reviewed by Ashutosh Chauhan)

Added:
    hive/trunk/common/src/java/org/apache/hive/common/util/HiveTestUtils.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBridge.java
Modified:
    hive/trunk/.gitignore
    hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
    hive/trunk/common/src/java/conf/hive-log4j.properties
    hive/trunk/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java
    hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java
    hive/trunk/common/src/test/resources/hive-exec-log4j-test.properties
    hive/trunk/common/src/test/resources/hive-log4j-test.properties
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestPassProperties.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java
    hive/trunk/ql/src/java/conf/hive-exec-log4j.properties
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataPrettyFormatUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java
    hive/trunk/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java
    hive/trunk/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java

Modified: hive/trunk/.gitignore
URL: http://svn.apache.org/viewvc/hive/trunk/.gitignore?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/.gitignore (original)
+++ hive/trunk/.gitignore Tue Oct 29 15:25:23 2013
@@ -13,7 +13,8 @@ common/src/gen
 *.iml
 *.ipr
 *.iws
-ql/derby.log
 derby.log
+datanucleus.log
 .arc
-ql/TempStatsStore
+TempStatsStore/
+target/

Modified: hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java (original)
+++ hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java Tue Oct 29 15:25:23 2013
@@ -156,23 +156,22 @@ public class TestCliDriverMethods extend
       historyFile.delete();
     }
     HiveConf configuration = new HiveConf();
-    CliSessionState ss = new CliSessionState(configuration);
-    CliSessionState.start(ss);
-    String[] args = {};
+    configuration.setBoolVar(ConfVars.HIVE_SESSION_HISTORY_ENABLED, true);
     PrintStream oldOut = System.out;
     ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
     System.setOut(new PrintStream(dataOut));
-
     PrintStream oldErr = System.err;
     ByteArrayOutputStream dataErr = new ByteArrayOutputStream();
     System.setErr(new PrintStream(dataErr));
-
+    CliSessionState ss = new CliSessionState(configuration);
+    CliSessionState.start(ss);
+    String[] args = {};
 
     try {
       new FakeCliDriver().run(args);
-      assertTrue(dataOut.toString().contains("test message"));
-      assertTrue(dataErr.toString().contains("Hive history file="));
-      assertTrue(dataErr.toString().contains("File: fakeFile is not a file."));
+      assertTrue(dataOut.toString(), dataOut.toString().contains("test message"));
+      assertTrue(dataErr.toString(), dataErr.toString().contains("Hive history file="));
+      assertTrue(dataErr.toString(), dataErr.toString().contains("File: fakeFile is not a file."));
       dataOut.reset();
       dataErr.reset();
 

Modified: hive/trunk/common/src/java/conf/hive-log4j.properties
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/conf/hive-log4j.properties?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/common/src/java/conf/hive-log4j.properties (original)
+++ hive/trunk/common/src/java/conf/hive-log4j.properties Tue Oct 29 15:25:23 2013
@@ -17,7 +17,7 @@
 # Define some default values that can be overridden by system properties
 hive.log.threshold=ALL
 hive.root.logger=INFO,DRFA
-hive.log.dir=/tmp/${user.name}
+hive.log.dir=${java.io.tmpdir}/${user.name}
 hive.log.file=hive.log
 
 # Define the root logger to the system property "hadoop.root.logger".

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/common/LogUtils.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/common/LogUtils.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/common/LogUtils.java Tue Oct 29 15:25:23 2013
@@ -89,8 +89,11 @@ public class LogUtils {
         // property speficied file found in local file system
         // use the specified file
         if (confVarName == HiveConf.ConfVars.HIVE_EXEC_LOG4J_FILE) {
-          System.setProperty(HiveConf.ConfVars.HIVEQUERYID.toString(),
-            HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID));
+          String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID);
+          if(queryId == null || (queryId = queryId.trim()).isEmpty()) {
+            queryId = "unknown-" + System.currentTimeMillis();
+          }
+          System.setProperty(HiveConf.ConfVars.HIVEQUERYID.toString(), queryId);
         }
         LogManager.resetConfiguration();
         PropertyConfigurator.configure(log4jFileName);

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Oct 29 15:25:23 2013
@@ -749,7 +749,7 @@ public class HiveConf extends Configurat
     // Number of async threads
     HIVE_SERVER2_ASYNC_EXEC_THREADS("hive.server2.async.exec.threads", 50),
     // Number of seconds HiveServer2 shutdown will wait for async threads to terminate
-    HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT("hive.server2.async.exec.shutdown.timeout", 10),
+    HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT("hive.server2.async.exec.shutdown.timeout", 10L),
 
 
     // HiveServer2 auth configuration
@@ -964,12 +964,12 @@ public class HiveConf extends Configurat
   }
 
   public static int getIntVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Integer.class);
+    assert (var.valClass == Integer.class) : var.varname;
     return conf.getInt(var.varname, var.defaultIntVal);
   }
 
   public static void setIntVar(Configuration conf, ConfVars var, int val) {
-    assert (var.valClass == Integer.class);
+    assert (var.valClass == Integer.class) : var.varname;
     conf.setInt(var.varname, val);
   }
 
@@ -982,7 +982,7 @@ public class HiveConf extends Configurat
   }
 
   public static long getLongVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Long.class);
+    assert (var.valClass == Long.class) : var.varname;
     return conf.getLong(var.varname, var.defaultLongVal);
   }
 
@@ -991,7 +991,7 @@ public class HiveConf extends Configurat
   }
 
   public static void setLongVar(Configuration conf, ConfVars var, long val) {
-    assert (var.valClass == Long.class);
+    assert (var.valClass == Long.class) : var.varname;
     conf.setLong(var.varname, val);
   }
 
@@ -1004,7 +1004,7 @@ public class HiveConf extends Configurat
   }
 
   public static float getFloatVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Float.class);
+    assert (var.valClass == Float.class) : var.varname;
     return conf.getFloat(var.varname, var.defaultFloatVal);
   }
 
@@ -1013,7 +1013,7 @@ public class HiveConf extends Configurat
   }
 
   public static void setFloatVar(Configuration conf, ConfVars var, float val) {
-    assert (var.valClass == Float.class);
+    assert (var.valClass == Float.class) : var.varname;
     ShimLoader.getHadoopShims().setFloatConf(conf, var.varname, val);
   }
 
@@ -1026,7 +1026,7 @@ public class HiveConf extends Configurat
   }
 
   public static boolean getBoolVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Boolean.class);
+    assert (var.valClass == Boolean.class) : var.varname;
     return conf.getBoolean(var.varname, var.defaultBoolVal);
   }
 
@@ -1035,7 +1035,7 @@ public class HiveConf extends Configurat
   }
 
   public static void setBoolVar(Configuration conf, ConfVars var, boolean val) {
-    assert (var.valClass == Boolean.class);
+    assert (var.valClass == Boolean.class) : var.varname;
     conf.setBoolean(var.varname, val);
   }
 
@@ -1048,7 +1048,7 @@ public class HiveConf extends Configurat
   }
 
   public static String getVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == String.class);
+    assert (var.valClass == String.class) : var.varname;
     return conf.get(var.varname, var.defaultVal);
   }
 
@@ -1057,7 +1057,7 @@ public class HiveConf extends Configurat
   }
 
   public static void setVar(Configuration conf, ConfVars var, String val) {
-    assert (var.valClass == String.class);
+    assert (var.valClass == String.class) : var.varname;
     conf.set(var.varname, val);
   }
 

Added: hive/trunk/common/src/java/org/apache/hive/common/util/HiveTestUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hive/common/util/HiveTestUtils.java?rev=1536755&view=auto
==============================================================================
--- hive/trunk/common/src/java/org/apache/hive/common/util/HiveTestUtils.java (added)
+++ hive/trunk/common/src/java/org/apache/hive/common/util/HiveTestUtils.java Tue Oct 29 15:25:23 2013
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.common.util;
+
+import java.net.URL;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience;
+import org.apache.hadoop.hive.common.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HiveTestUtils {
+
+  public static String getFileFromClasspath(String name) {
+    URL url = ClassLoader.getSystemResource(name);
+    if (url == null) {
+      throw new IllegalArgumentException("Could not find " + name);
+    }
+    return url.getPath();
+  }
+}

Modified: hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java (original)
+++ hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java Tue Oct 29 15:25:23 2013
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.conf;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.common.util.HiveTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -33,8 +34,7 @@ import org.junit.Test;
 public class TestHiveConf {
   @Test
   public void testHiveSitePath() throws Exception {
-    String expectedPath =
-        new Path(System.getProperty("test.build.resources") + "/hive-site.xml").toUri().getPath();
+    String expectedPath = HiveTestUtils.getFileFromClasspath("hive-site.xml");
     Assert.assertEquals(expectedPath, new HiveConf().getHiveSiteLocation().getPath());
   }
 

Modified: hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java (original)
+++ hive/trunk/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java Tue Oct 29 15:25:23 2013
@@ -24,7 +24,8 @@ import java.io.InputStreamReader;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.common.LogUtils;
-import org.apache.hadoop.hive.common.LogUtils.LogInitializationException;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.common.util.HiveTestUtils;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 
 /**
@@ -43,53 +44,34 @@ public class TestHiveLogging extends Tes
     process = null;
   }
 
-  private void configLog(String hiveLog4jTest, String hiveExecLog4jTest) {
-    System.setProperty(ConfVars.HIVE_LOG4J_FILE.varname,
-      System.getProperty("test.build.resources") + "/" + hiveLog4jTest);
-    System.setProperty(ConfVars.HIVE_EXEC_LOG4J_FILE.varname,
-      System.getProperty("test.build.resources") + "/" + hiveExecLog4jTest);
-
-    String expectedLog4jPath = System.getProperty("test.build.resources")
-      + "/" + hiveLog4jTest;
-    String expectedLog4jExecPath = System.getProperty("test.build.resources")
-      + "/" + hiveExecLog4jTest;
-
-    try {
-      LogUtils.initHiveLog4j();
-    } catch (LogInitializationException e) {
-    }
+  private void configLog(String hiveLog4jTest, String hiveExecLog4jTest) 
+  throws Exception {
+    String expectedLog4jTestPath = HiveTestUtils.getFileFromClasspath(hiveLog4jTest);
+    String expectedLog4jExecPath = HiveTestUtils.getFileFromClasspath(hiveExecLog4jTest);
+    System.setProperty(ConfVars.HIVE_LOG4J_FILE.varname, expectedLog4jTestPath);
+    System.setProperty(ConfVars.HIVE_EXEC_LOG4J_FILE.varname, expectedLog4jExecPath);
+
+    LogUtils.initHiveLog4j();
 
     HiveConf conf = new HiveConf();
-    assertEquals(expectedLog4jPath, conf.getVar(ConfVars.HIVE_LOG4J_FILE));
+    assertEquals(expectedLog4jTestPath, conf.getVar(ConfVars.HIVE_LOG4J_FILE));
     assertEquals(expectedLog4jExecPath, conf.getVar(ConfVars.HIVE_EXEC_LOG4J_FILE));
   }
 
-  private void runCmd(String cmd) {
-    try {
-      process = runTime.exec(cmd);
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    try {
-      process.waitFor();
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    }
+  private void runCmd(String cmd) throws Exception {
+    process = runTime.exec(cmd);
+    process.waitFor();
   }
 
-  private void getCmdOutput(String logFile) {
+  private void getCmdOutput(String logFile) throws Exception {
     boolean logCreated = false;
     BufferedReader buf = new BufferedReader(
       new InputStreamReader(process.getInputStream()));
     String line = "";
-    try {
-      while((line = buf.readLine()) != null) {
-        if (line.equals(logFile)) {
-          logCreated = true;
-        }
+    while((line = buf.readLine()) != null) {
+      if (line.equals(logFile)) {
+        logCreated = true;
       }
-    } catch (IOException e) {
-      e.printStackTrace();
     }
     assertEquals(true, logCreated);
   }
@@ -112,12 +94,12 @@ public class TestHiveLogging extends Tes
   }
 
   public void testHiveLogging() throws Exception {
-    // customized log4j config log file to be: /tmp/hiveLog4jTest.log
-    String customLogPath = "/tmp/";
+    // customized log4j config log file to be: /tmp/TestHiveLogging/hiveLog4jTest.log
+    String customLogPath = "/tmp/" + System.getProperty("user.name") + "-TestHiveLogging/";
     String customLogName = "hiveLog4jTest.log";
     String customLogFile = customLogPath + customLogName;
     String customCleanCmd = "rm -rf " + customLogFile;
-    String customFindCmd = "find /tmp -name " + customLogName;
+    String customFindCmd = "find " + customLogPath + " -name " + customLogName;
     RunTest(customCleanCmd, customFindCmd, customLogFile,
       "hive-log4j-test.properties", "hive-exec-log4j-test.properties");
   }

Modified: hive/trunk/common/src/test/resources/hive-exec-log4j-test.properties
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/test/resources/hive-exec-log4j-test.properties?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/common/src/test/resources/hive-exec-log4j-test.properties (original)
+++ hive/trunk/common/src/test/resources/hive-exec-log4j-test.properties Tue Oct 29 15:25:23 2013
@@ -1,6 +1,6 @@
 # Define some default values that can be overridden by system properties
 hive.root.logger=INFO,FA
-hive.log.dir=/tmp
+hive.log.dir=/tmp/${user.name}-TestHiveLogging
 hive.log.file=hiveExecLog4jTest.log
 
 # Define the root logger to the system property "hadoop.root.logger".

Modified: hive/trunk/common/src/test/resources/hive-log4j-test.properties
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/test/resources/hive-log4j-test.properties?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/common/src/test/resources/hive-log4j-test.properties (original)
+++ hive/trunk/common/src/test/resources/hive-log4j-test.properties Tue Oct 29 15:25:23 2013
@@ -1,6 +1,6 @@
 # Define some default values that can be overridden by system properties
 hive.root.logger=WARN,DRFA
-hive.log.dir=/tmp
+hive.log.dir=/tmp/${user.name}-TestHiveLogging
 hive.log.file=hiveLog4jTest.log
 
 # Define the root logger to the system property "hadoop.root.logger".

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java Tue Oct 29 15:25:23 2013
@@ -585,13 +585,6 @@ class FileOutputCommitterContainer exten
         }
       }
 
-      //      for (Entry<String,Map<String,String>> spec : partitionsDiscoveredByPath.entrySet()){
-      //        LOG.info("Partition "+ spec.getKey());
-      //        for (Entry<String,String> e : spec.getValue().entrySet()){
-      //          LOG.info(e.getKey() + "=>" +e.getValue());
-      //        }
-      //      }
-
       this.partitionsDiscovered = true;
     }
   }
@@ -652,7 +645,6 @@ class FileOutputCommitterContainer exten
       for(Partition ptn : partitionsToAdd){
         ptnInfos.add(InternalUtil.createPtnKeyValueMap(new Table(tableInfo.getTable()), ptn));
       }
-
       //Publish the new partition(s)
       if (dynamicPartitioningUsed && harProcessor.isEnabled() && (!partitionsToAdd.isEmpty())){
 
@@ -678,7 +670,7 @@ class FileOutputCommitterContainer exten
           throw e;
         }
 
-      }else{
+      } else {
         // no harProcessor, regular operation
         updateTableSchema(client, table, jobInfo.getOutputSchema());
         LOG.info("HAR not is not being used. The table {} has new partitions {}.", table.getTableName(), ptnInfos);

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestPassProperties.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestPassProperties.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestPassProperties.java (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestPassProperties.java Tue Oct 29 15:25:23 2013
@@ -51,7 +51,7 @@ import org.junit.Test;
  */
 public class TestPassProperties {
   private static final String TEST_DATA_DIR = System.getProperty("user.dir") +
-      "/build/test/data/" + TestSequenceFileReadWrite.class.getCanonicalName();
+      "/build/test/data/" + TestPassProperties.class.getCanonicalName();
   private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
   private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data";
 
@@ -96,7 +96,7 @@ public class TestPassProperties {
       conf.set("hive.metastore.uris", "thrift://no.such.machine:10888");
       conf.set("hive.metastore.local", "false");
       Job job = new Job(conf, "Write-hcat-seq-table");
-      job.setJarByClass(TestSequenceFileReadWrite.class);
+      job.setJarByClass(TestPassProperties.class);
 
       job.setMapperClass(Map.class);
       job.setOutputKeyClass(NullWritable.class);

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java Tue Oct 29 15:25:23 2013
@@ -48,7 +48,7 @@ import org.junit.Test;
 
 public class TestPassProperties {
   private static final String TEST_DATA_DIR = System.getProperty("user.dir") +
-      "/build/test/data/" + TestSequenceFileReadWrite.class.getCanonicalName();
+      "/build/test/data/" + TestPassProperties.class.getCanonicalName();
   private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
   private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data";
 
@@ -93,7 +93,7 @@ public class TestPassProperties {
       conf.set("hive.metastore.uris", "thrift://no.such.machine:10888");
       conf.set("hive.metastore.local", "false");
       Job job = new Job(conf, "Write-hcat-seq-table");
-      job.setJarByClass(TestSequenceFileReadWrite.class);
+      job.setJarByClass(TestPassProperties.class);
 
       job.setMapperClass(Map.class);
       job.setOutputKeyClass(NullWritable.class);

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Tue Oct 29 15:25:23 2013
@@ -334,9 +334,7 @@ public class HiveMetaStore extends Thrif
         } catch (Exception e) {
           // log exception, but ignore inability to start
           LOG.error("error in Metrics init: " + e.getClass().getName() + " "
-              + e.getMessage());
-          MetaStoreUtils.printStackTrace(e);
-
+              + e.getMessage(), e);
         }
       }
 
@@ -466,8 +464,7 @@ public class HiveMetaStore extends Thrif
         Metrics.startScope(function);
       } catch (IOException e) {
         LOG.debug("Exception when starting metrics scope"
-            + e.getClass().getName() + " " + e.getMessage());
-        MetaStoreUtils.printStackTrace(e);
+            + e.getClass().getName() + " " + e.getMessage(), e);
       }
       return function;
     }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Tue Oct 29 15:25:23 2013
@@ -76,21 +76,6 @@ public class MetaStoreUtils {
 
   public static final String DATABASE_WAREHOUSE_SUFFIX = ".db";
 
-  /**
-   * printStackTrace
-   *
-   * Helper function to print an exception stack trace to the log and not stderr
-   *
-   * @param e
-   *          the exception
-   *
-   */
-  static public void printStackTrace(Exception e) {
-    for (StackTraceElement s : e.getStackTrace()) {
-      LOG.error(s);
-    }
-  }
-
   public static Table createColumnsetSchema(String name, List<String> columns,
       List<String> partCols, Configuration conf) throws MetaException {
 
@@ -175,14 +160,17 @@ public class MetaStoreUtils {
   static public Deserializer getDeserializer(Configuration conf,
       Properties schema) throws MetaException {
     try {
-      Deserializer deserializer = ReflectionUtils.newInstance(conf.getClassByName(
-      schema.getProperty(serdeConstants.SERIALIZATION_LIB)).asSubclass(Deserializer.class), conf);
+      String clazzName = schema.getProperty(serdeConstants.SERIALIZATION_LIB);
+      if(clazzName == null) {
+        throw new IllegalStateException("Property " + serdeConstants.SERIALIZATION_LIB + " cannot be null");
+      }
+      Deserializer deserializer = ReflectionUtils.newInstance(conf.getClassByName(clazzName)
+          .asSubclass(Deserializer.class), conf);
       deserializer.initialize(conf, schema);
       return deserializer;
     } catch (Exception e) {
       LOG.error("error in initSerDe: " + e.getClass().getName() + " "
-          + e.getMessage());
-      MetaStoreUtils.printStackTrace(e);
+          + e.getMessage(), e);
       throw new MetaException(e.getClass().getName() + " " + e.getMessage());
     }
   }
@@ -221,8 +209,7 @@ public class MetaStoreUtils {
       throw e;
     } catch (Exception e) {
       LOG.error("error in initSerDe: " + e.getClass().getName() + " "
-          + e.getMessage());
-      MetaStoreUtils.printStackTrace(e);
+          + e.getMessage(), e);
       throw new MetaException(e.getClass().getName() + " " + e.getMessage());
     }
   }
@@ -258,8 +245,7 @@ public class MetaStoreUtils {
       throw e;
     } catch (Exception e) {
       LOG.error("error in initSerDe: " + e.getClass().getName() + " "
-          + e.getMessage());
-      MetaStoreUtils.printStackTrace(e);
+          + e.getMessage(), e);
       throw new MetaException(e.getClass().getName() + " " + e.getMessage());
     }
   }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java Tue Oct 29 15:25:23 2013
@@ -1732,7 +1732,7 @@ public class ObjectStore implements RawS
       LOG.debug("Done executing query for listMPartitions");
       pm.retrieveAll(mparts);
       success = commitTransaction();
-      LOG.debug("Done retrieving all objects for listMPartitions");
+      LOG.debug("Done retrieving all objects for listMPartitions " + mparts);
     } finally {
       if (!success) {
         rollbackTransaction();

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java Tue Oct 29 15:25:23 2013
@@ -19,15 +19,21 @@
 package org.apache.hadoop.hive.metastore;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 
-public class TestMarkPartitionRemote extends TestMarkPartition{
+public class TestMarkPartitionRemote extends TestMarkPartition {
 
   private static class RunMS implements Runnable {
 
+    private final int port;
+
+    public RunMS(int port) {
+      this.port = port;
+    }
     @Override
     public void run() {
       try {
-        HiveMetaStore.main(new String[] { "29111" });
+        HiveMetaStore.main(new String[] { String.valueOf(port) });
       } catch (Throwable e) {
         e.printStackTrace(System.err);
         assert false;
@@ -38,10 +44,11 @@ public class TestMarkPartitionRemote ext
   @Override
   protected void setUp() throws Exception {
     super.setUp();
-    Thread t = new Thread(new RunMS());
+    int port = MetaStoreUtils.findFreePort();
+    Thread t = new Thread(new RunMS(port));
     t.setDaemon(true);
     t.start();
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:29111");
+    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
     hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
     Thread.sleep(30000);
   }

Modified: hive/trunk/ql/src/java/conf/hive-exec-log4j.properties
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/conf/hive-exec-log4j.properties?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/java/conf/hive-exec-log4j.properties (original)
+++ hive/trunk/ql/src/java/conf/hive-exec-log4j.properties Tue Oct 29 15:25:23 2013
@@ -17,7 +17,8 @@
 # Define some default values that can be overridden by system properties
 hive.log.threshold=ALL
 hive.root.logger=INFO,FA
-hive.log.dir=/tmp/${user.name}
+hive.log.dir=${java.io.tmpdir}/${user.name}
+hive.query.id=hadoop
 hive.log.file=${hive.query.id}.log
 
 # Define the root logger to the system property "hadoop.root.logger".

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Tue Oct 29 15:25:23 2013
@@ -140,55 +140,45 @@ public class Driver implements CommandPr
 
   private String userName;
 
-  private boolean checkLockManager() {
+  private boolean checkConcurrency() throws SemanticException {
     boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
     if (!supportConcurrency) {
+      LOG.info("Concurrency mode is disabled, not creating a lock manager");
       return false;
     }
-    if ((hiveLockMgr == null)) {
-      try {
-        setLockManager();
-      } catch (SemanticException e) {
-        errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage();
-        SQLState = ErrorMsg.findSQLState(e.getMessage());
-        downstreamError = e;
-        console.printError(errorMessage, "\n"
-            + org.apache.hadoop.util.StringUtils.stringifyException(e));
-        return false;
-      }
-    }
+    createLockManager();
     // the reason that we set the lock manager for the cxt here is because each
     // query has its own ctx object. The hiveLockMgr is shared accross the
     // same instance of Driver, which can run multiple queries.
     ctx.setHiveLockMgr(hiveLockMgr);
-    return hiveLockMgr != null;
+    return true;
   }
 
-  private void setLockManager() throws SemanticException {
-    boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
-    if (supportConcurrency) {
-      String lockMgr = conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER);
-      if ((lockMgr == null) || (lockMgr.isEmpty())) {
-        throw new SemanticException(ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg());
-      }
-
-      try {
-        hiveLockMgr = (HiveLockManager) ReflectionUtils.newInstance(conf.getClassByName(lockMgr),
-            conf);
-        hiveLockMgr.setContext(new HiveLockManagerCtx(conf));
-      } catch (Exception e) {
-        // set hiveLockMgr to null just in case this invalid manager got set to
-        // next query's ctx.
-        if (hiveLockMgr != null) {
-          try {
-            hiveLockMgr.close();
-          } catch (LockException e1) {
-            //nothing can do here
-          }
-          hiveLockMgr = null;
+  private void createLockManager() throws SemanticException {
+    if (hiveLockMgr != null) {
+      return;
+    }
+    String lockMgr = conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER);
+    LOG.info("Creating lock manager of type " + lockMgr);
+    if ((lockMgr == null) || (lockMgr.isEmpty())) {
+      throw new SemanticException(ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg());
+    }
+    try {
+      hiveLockMgr = (HiveLockManager) ReflectionUtils.newInstance(conf.getClassByName(lockMgr),
+          conf);
+      hiveLockMgr.setContext(new HiveLockManagerCtx(conf));
+    } catch (Exception e1) {
+      // set hiveLockMgr to null just in case this invalid manager got set to
+      // next query's ctx.
+      if (hiveLockMgr != null) {
+        try {
+          hiveLockMgr.close();
+        } catch (LockException e2) {
+          //nothing can do here
         }
-        throw new SemanticException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + e.getMessage());
+        hiveLockMgr = null;
       }
+      throw new SemanticException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + e1.getMessage(), e1);
     }
   }
 
@@ -1009,7 +999,18 @@ public class Driver implements CommandPr
     }
 
     boolean requireLock = false;
-    boolean ckLock = checkLockManager();
+    boolean ckLock = false;
+    try {
+      ckLock = checkConcurrency();
+    } catch (SemanticException e) {
+      errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage();
+      SQLState = ErrorMsg.findSQLState(e.getMessage());
+      downstreamError = e;
+      console.printError(errorMessage, "\n"
+          + org.apache.hadoop.util.StringUtils.stringifyException(e));
+      ret = 10;
+      return new CommandProcessorResponse(ret, errorMessage, SQLState);
+    }
 
     if (ckLock) {
       boolean lockOnlyMapred = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_LOCK_MAPRED_ONLY);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java Tue Oct 29 15:25:23 2013
@@ -666,6 +666,12 @@ public class ExecDriver extends Task<Map
 
     boolean isSilent = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVESESSIONSILENT);
 
+    String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID, "").trim();
+    if(queryId.isEmpty()) {
+      queryId = "unknown-" + System.currentTimeMillis();
+    }
+    System.setProperty(HiveConf.ConfVars.HIVEQUERYID.toString(), queryId);
+
     if (noLog) {
       // If started from main(), and noLog is on, we should not output
       // any logs. To turn the log on, please set -Dtest.silent=false

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java Tue Oct 29 15:25:23 2013
@@ -61,7 +61,7 @@ public class MapRedTask extends ExecDriv
   static final String HIVE_DEBUG_RECURSIVE = "HIVE_DEBUG_RECURSIVE";
   static final String HIVE_MAIN_CLIENT_DEBUG_OPTS = "HIVE_MAIN_CLIENT_DEBUG_OPTS";
   static final String HIVE_CHILD_CLIENT_DEBUG_OPTS = "HIVE_CHILD_CLIENT_DEBUG_OPTS";
-  static final String[] HIVE_SYS_PROP = {"build.dir", "build.dir.hive"};
+  static final String[] HIVE_SYS_PROP = {"build.dir", "build.dir.hive", "hive.query.id"};
 
   private transient ContentSummary inputSummary = null;
   private transient boolean runningViaChild = false;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java Tue Oct 29 15:25:23 2013
@@ -87,7 +87,7 @@ public class MapredLocalTask extends Tas
   public static transient final Log l4j = LogFactory.getLog(MapredLocalTask.class);
   static final String HADOOP_MEM_KEY = "HADOOP_HEAPSIZE";
   static final String HADOOP_OPTS_KEY = "HADOOP_OPTS";
-  static final String[] HIVE_SYS_PROP = {"build.dir", "build.dir.hive"};
+  static final String[] HIVE_SYS_PROP = {"build.dir", "build.dir.hive", "hive.query.id"};
   public static MemoryMXBean memoryMXBean;
   private static final Log LOG = LogFactory.getLog(MapredLocalTask.class);
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataPrettyFormatUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataPrettyFormatUtils.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataPrettyFormatUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataPrettyFormatUtils.java Tue Oct 29 15:25:23 2013
@@ -25,8 +25,6 @@ import org.apache.commons.lang.StringEsc
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 
-import jline.Terminal;
-
 /**
  * This class provides methods to format the output of DESCRIBE PRETTY
  * in a human-readable way.
@@ -116,8 +114,8 @@ public final class MetaDataPrettyFormatU
       int columnsAlreadyConsumed, int prettyOutputNumCols) {
 
     if (prettyOutputNumCols == -1) {
-      Terminal terminal = Terminal.getTerminal();
-      prettyOutputNumCols = terminal.getTerminalWidth() - 1;
+      // XXX fixed to 80 to remove jline dep
+      prettyOutputNumCols = 80 - 1;
     }
 
     int commentNumCols = prettyOutputNumCols - columnsAlreadyConsumed;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java Tue Oct 29 15:25:23 2013
@@ -46,21 +46,41 @@ import org.apache.hadoop.hive.serde2.typ
  *
  */
 public class GenericUDFBridge extends GenericUDF implements Serializable {
+  private static final long serialVersionUID = 4994861742809511113L;
+
   /**
    * The name of the UDF.
    */
-  String udfName;
+  private String udfName;
 
   /**
    * Whether the UDF is an operator or not. This controls how the display string
    * is generated.
    */
-  boolean isOperator;
+  private boolean isOperator;
 
   /**
    * The underlying UDF class Name.
    */
-  String udfClassName;
+  private String udfClassName;
+
+  /**
+   * The underlying method of the UDF class.
+   */
+  private transient Method udfMethod;
+
+  /**
+   * Helper to convert the parameters before passing to udfMethod.
+   */
+  private transient ConversionHelper conversionHelper;
+  /**
+   * The actual udf object.
+   */
+  private transient UDF udf;
+  /**
+   * The non-deferred real arguments for method invocation.
+   */
+  private transient Object[] realArguments;
 
   /**
    * Create a new GenericUDFBridge object.
@@ -76,7 +96,7 @@ public class GenericUDFBridge extends Ge
     this.isOperator = isOperator;
     this.udfClassName = udfClassName;
   }
-
+ 
   // For Java serialization only
   public GenericUDFBridge() {
   }
@@ -113,24 +133,6 @@ public class GenericUDFBridge extends Ge
     }
   }
 
-  /**
-   * The underlying method of the UDF class.
-   */
-  transient Method udfMethod;
-
-  /**
-   * Helper to convert the parameters before passing to udfMethod.
-   */
-  transient ConversionHelper conversionHelper;
-  /**
-   * The actual udf object.
-   */
-  transient UDF udf;
-  /**
-   * The non-deferred real arguments for method invocation.
-   */
-  transient Object[] realArguments;
-
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
 

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestLocationQueries.java Tue Oct 29 15:25:23 2013
@@ -32,6 +32,14 @@ import java.util.regex.Pattern;
  *  ignored.
  */
 public class TestLocationQueries extends BaseTestQueries {
+
+  public TestLocationQueries() {
+    File logDirFile = new File(logDir);
+    if (!(logDirFile.exists() || logDirFile.mkdirs())) {
+      fail("Could not create " + logDir);
+    }
+  }
+
   /**
    * Our own checker - validate the location of the partition.
    */

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/TestMTQueries.java Tue Oct 29 15:25:23 2013
@@ -25,6 +25,13 @@ import java.io.File;
  */
 public class TestMTQueries extends BaseTestQueries {
 
+  public TestMTQueries() {
+    File logDirFile = new File(logDir);
+    if (!(logDirFile.exists() || logDirFile.mkdirs())) {
+      fail("Could not create " + logDir);
+    }
+  }
+
   public void testMTQueries1() throws Exception {
     String[] testNames = new String[] {"join1.q", "join2.q", "groupby1.q",
         "groupby2.q", "join3.q", "input1.q", "input19.q"};

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java Tue Oct 29 15:25:23 2013
@@ -131,8 +131,7 @@ public class TestExecDriver extends Test
       }
 
     } catch (Throwable e) {
-      e.printStackTrace();
-      throw new RuntimeException("Encountered throwable");
+      throw new RuntimeException("Encountered throwable", e);
     }
   }
 
@@ -472,118 +471,70 @@ public class TestExecDriver extends Test
   public void testMapPlan1() throws Exception {
 
     LOG.info("Beginning testMapPlan1");
-
-    try {
-      populateMapPlan1(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src"));
-      executePlan();
-      fileDiff("lt100.txt.deflate", "mapplan1.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapPlan1(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src"));
+    executePlan();
+    fileDiff("lt100.txt.deflate", "mapplan1.out");
   }
 
   public void testMapPlan2() throws Exception {
 
     LOG.info("Beginning testMapPlan2");
-
-    try {
-      populateMapPlan2(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src"));
-      executePlan();
-      fileDiff("lt100.txt", "mapplan2.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapPlan2(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src"));
+    executePlan();
+    fileDiff("lt100.txt", "mapplan2.out");
   }
 
   public void testMapRedPlan1() throws Exception {
 
     LOG.info("Beginning testMapRedPlan1");
-
-    try {
-      populateMapRedPlan1(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("kv1.val.sorted.txt", "mapredplan1.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan1(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("kv1.val.sorted.txt", "mapredplan1.out");
   }
 
   public void testMapRedPlan2() throws Exception {
 
     LOG.info("Beginning testMapPlan2");
-
-    try {
-      populateMapRedPlan2(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("lt100.sorted.txt", "mapredplan2.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan2(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("lt100.sorted.txt", "mapredplan2.out");
   }
 
   public void testMapRedPlan3() throws Exception {
 
     LOG.info("Beginning testMapPlan3");
-
-    try {
-      populateMapRedPlan3(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"), db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src2"));
-      executePlan();
-      fileDiff("kv1kv2.cogroup.txt", "mapredplan3.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan3(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"), db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "src2"));
+    executePlan();
+    fileDiff("kv1kv2.cogroup.txt", "mapredplan3.out");
   }
 
   public void testMapRedPlan4() throws Exception {
 
     LOG.info("Beginning testMapPlan4");
-
-    try {
-      populateMapRedPlan4(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("kv1.string-sorted.txt", "mapredplan4.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan4(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("kv1.string-sorted.txt", "mapredplan4.out");
   }
 
   public void testMapRedPlan5() throws Exception {
 
     LOG.info("Beginning testMapPlan5");
-
-    try {
-      populateMapRedPlan5(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("kv1.string-sorted.txt", "mapredplan5.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan5(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("kv1.string-sorted.txt", "mapredplan5.out");
   }
 
   public void testMapRedPlan6() throws Exception {
 
     LOG.info("Beginning testMapPlan6");
-
-    try {
-      populateMapRedPlan6(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          "src"));
-      executePlan();
-      fileDiff("lt100.sorted.txt", "mapredplan6.out");
-    } catch (Throwable e) {
-      e.printStackTrace();
-      fail("Got Throwable");
-    }
+    populateMapRedPlan6(db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
+        "src"));
+    executePlan();
+    fileDiff("lt100.sorted.txt", "mapredplan6.out");
   }
 }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestMemoryManager.java Tue Oct 29 15:25:23 2013
@@ -51,7 +51,7 @@ public class TestMemoryManager {
     NullCallback callback = new NullCallback();
     long poolSize = mgr.getTotalMemoryPool();
     assertEquals(Math.round(ManagementFactory.getMemoryMXBean().
-        getHeapMemoryUsage().getMax() * 0.5f), poolSize);
+        getHeapMemoryUsage().getMax() * 0.5d), poolSize);
     assertEquals(1.0, mgr.getAllocationScale(), 0.00001);
     mgr.addWriter(new Path("p1"), 1000, callback);
     assertEquals(1.0, mgr.getAllocationScale(), 0.00001);

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBridge.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBridge.java?rev=1536755&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBridge.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFBridge.java Tue Oct 29 15:25:23 2013
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.junit.Test;
+
+public class TestGenericUDFBridge {
+
+  @Test(expected = UDFArgumentException.class)
+  public void testInvalidName() throws Exception {
+    GenericUDFBridge udf = new GenericUDFBridge("someudf", false, "not a class name");
+    udf.initialize(new ObjectInspector[0]);
+    udf.close();
+  }
+
+  @Test(expected = UDFArgumentException.class)
+  public void testNullName() throws Exception {
+    GenericUDFBridge udf = new GenericUDFBridge("someudf", false, null);
+    udf.initialize(new ObjectInspector[0]);
+    udf.close();
+  }
+}

Modified: hive/trunk/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java (original)
+++ hive/trunk/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java Tue Oct 29 15:25:23 2013
@@ -26,102 +26,72 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import javax.security.sasl.AuthenticationException;
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.SQLException;
 import java.util.HashMap;
 import java.util.Map;
 
 public class TestCustomAuthentication {
 
-  private static HiveServer2 hiveserver2 = null;
-
-  private static File configFile = null;
+  private static HiveServer2 hiveserver2;
+  private static HiveConf hiveConf;
+  private static byte[] hiveConfBackup;
 
   @BeforeClass
   public static void setUp() throws Exception {
-    createConfig();
-    startServer();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    stopServer();
-    removeConfig();
-  }
-
-  private static void startServer() throws Exception{
-
-    HiveConf hiveConf = new HiveConf();
+    hiveConf = new HiveConf();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    hiveConf.writeXml(baos);
+    baos.close();
+    hiveConfBackup = baos.toByteArray();
+    hiveConf.set("hive.server2.authentication", "CUSTOM");
+    hiveConf.set("hive.server2.custom.authentication.class",
+        "org.apache.hive.service.auth.TestCustomAuthentication$SimpleAuthenticationProviderImpl");
+    FileOutputStream fos = new FileOutputStream(new File(hiveConf.getHiveSiteLocation().toURI()));
+    hiveConf.writeXml(fos);
+    fos.close();
     hiveserver2 = new HiveServer2();
     hiveserver2.init(hiveConf);
     hiveserver2.start();
     Thread.sleep(1000);
     System.out.println("hiveServer2 start ......");
-
   }
 
-  private static void stopServer(){
-    try {
-      if (hiveserver2 != null) {
-        hiveserver2.stop();
-        hiveserver2 = null;
-      }
-      Thread.sleep(1000);
-    } catch (Exception e) {
-      e.printStackTrace();
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if(hiveConf != null && hiveConfBackup != null) {
+      FileOutputStream fos = new FileOutputStream(new File(hiveConf.getHiveSiteLocation().toURI()));
+      fos.write(hiveConfBackup);
+      fos.close();
+    }
+    if (hiveserver2 != null) {
+      hiveserver2.stop();
+      hiveserver2 = null;
     }
+    Thread.sleep(1000);
     System.out.println("hiveServer2 stop ......");
   }
 
-  private static void createConfig() throws Exception{
-
-    Configuration conf = new Configuration(false);
-    conf.set("hive.server2.authentication", "CUSTOM");
-    conf.set("hive.server2.custom.authentication.class",
-        "org.apache.hive.service.auth.TestCustomAuthentication$SimpleAuthenticationProviderImpl");
-
-    configFile = new File("../build/service/test/resources","hive-site.xml");
-
-    FileOutputStream out = new FileOutputStream(configFile);
-    conf.writeXml(out);
-  }
-
-  private static void removeConfig(){
-    try {
-      configFile.delete();
-    } catch (Exception e){
-      System.out.println(e.getMessage());
-    }
-  }
-
   @Test
-  public void testCustomAuthentication() throws Exception{
+  public void testCustomAuthentication() throws Exception {
 
     String url = "jdbc:hive2://localhost:10000/default";
+    Class.forName("org.apache.hive.jdbc.HiveDriver");
 
-    Exception exception = null;
-    try{
-      Class.forName("org.apache.hive.jdbc.HiveDriver");
-      Connection connection =  DriverManager.getConnection(url, "wronguser", "pwd");
-      connection.close();
-    } catch (Exception e){
-      exception = e;
-    }
-
-    Assert.assertNotNull(exception);
-
-    exception = null;
-    try{
-      Class.forName("org.apache.hive.jdbc.HiveDriver");
-      Connection connection =  DriverManager.getConnection(url, "hiveuser", "hive");
-      connection.close();
-    } catch (Exception e){
-      exception = e;
+    try {
+      DriverManager.getConnection(url, "wronguser", "pwd");
+      Assert.fail("Expected Exception");
+    } catch(SQLException e) {
+      Assert.assertNotNull(e.getMessage());
+      Assert.assertTrue(e.getMessage(), e.getMessage().contains("Peer indicated failure: Error validating the login"));
     }
 
-    Assert.assertNull(exception);
+    Connection connection = DriverManager.getConnection(url, "hiveuser", "hive");
+    connection.close();
 
     System.out.println(">>> PASSED testCustomAuthentication");
   }

Modified: hive/trunk/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java?rev=1536755&r1=1536754&r2=1536755&view=diff
==============================================================================
--- hive/trunk/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java (original)
+++ hive/trunk/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java Tue Oct 29 15:25:23 2013
@@ -88,7 +88,6 @@ public class Hadoop23Shims extends Hadoo
 
   @Override
   public JobTrackerState getJobTrackerState(ClusterStatus clusterStatus) throws Exception {
-    JobTrackerState state;
     switch (clusterStatus.getJobTrackerStatus()) {
     case INITIALIZING:
       return JobTrackerState.INITIALIZING;
@@ -212,19 +211,16 @@ public class Hadoop23Shims extends Hadoo
     public void shutdown() throws IOException {
       mr.shutdown();
     }
-    
+
     @Override
     public void setupConfiguration(Configuration conf) {
       JobConf jConf = mr.createJobConf();
       for (Map.Entry<String, String> pair: jConf) {
-	//System.out.println("XXX Var: "+pair.getKey() +"="+pair.getValue());
-        //if (conf.get(pair.getKey()) == null) {
-          conf.set(pair.getKey(), pair.getValue());
-	  //}
+        conf.set(pair.getKey(), pair.getValue());
       }
     }
   }
-  
+
   // Don't move this code to the parent class. There's a binary
   // incompatibility between hadoop 1 and 2 wrt MiniDFSCluster and we
   // need to have two different shim classes even though they are