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

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

Author: gunther
Date: Fri Oct  4 23:35:50 2013
New Revision: 1529347

URL: http://svn.apache.org/r1529347
Log:
Merge latest trunk into branch. (Gunther Hagleitner)

Added:
    hive/branches/tez/data/files/orc_create_people.txt
      - copied unchanged from r1529345, hive/trunk/data/files/orc_create_people.txt
    hive/branches/tez/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java
      - copied unchanged from r1529345, hive/trunk/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java
      - copied unchanged from r1529345, hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/AbstractFilterStringColLikeStringScalar.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStringColRegExpStringScalar.java
      - copied unchanged from r1529345, hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStringColRegExpStringScalar.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/io/orc/BinaryColumnStatistics.java
      - copied unchanged from r1529345, hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/BinaryColumnStatistics.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/
      - copied from r1529345, hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/
    hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSerDeStats.java
      - copied unchanged from r1529345, hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSerDeStats.java
    hive/branches/tez/ql/src/test/queries/clientpositive/alter_varchar2.q
      - copied unchanged from r1529345, hive/trunk/ql/src/test/queries/clientpositive/alter_varchar2.q
    hive/branches/tez/ql/src/test/queries/clientpositive/stats_partscan_1_23.q
      - copied unchanged from r1529345, hive/trunk/ql/src/test/queries/clientpositive/stats_partscan_1_23.q
    hive/branches/tez/ql/src/test/results/clientpositive/alter_varchar2.q.out
      - copied unchanged from r1529345, hive/trunk/ql/src/test/results/clientpositive/alter_varchar2.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/stats_partscan_1_23.q.out
      - copied unchanged from r1529345, hive/trunk/ql/src/test/results/clientpositive/stats_partscan_1_23.q.out
    hive/branches/tez/serde/src/test/org/apache/hadoop/hive/serde2/io/TestHiveVarcharWritable.java
      - copied unchanged from r1529345, hive/trunk/serde/src/test/org/apache/hadoop/hive/serde2/io/TestHiveVarcharWritable.java
Removed:
    hive/branches/tez/hcatalog/core/src/main/java/org/apache/hive/hcatalog/security/HdfsAuthorizationProvider.java
    hive/branches/tez/hcatalog/core/src/main/java/org/apache/hive/hcatalog/security/StorageDelegationAuthorizationProvider.java
    hive/branches/tez/hcatalog/core/src/test/java/org/apache/hive/hcatalog/security/TestHdfsAuthorizationProvider.java
    hive/branches/tez/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/HBaseReadWrite.java
Modified:
    hive/branches/tez/   (props changed)
    hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
    hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
    hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
    hive/branches/tez/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java
    hive/branches/tez/bin/hive
    hive/branches/tez/build-common.xml
    hive/branches/tez/build.xml
    hive/branches/tez/common/build.xml
    hive/branches/tez/common/src/test/org/apache/hadoop/hive/common/type/TestHiveVarchar.java
    hive/branches/tez/hbase-handler/src/test/templates/TestHBaseCliDriver.vm
    hive/branches/tez/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm
    hive/branches/tez/hcatalog/build-support/ant/build-common.xml
    hive/branches/tez/hcatalog/build.xml
    hive/branches/tez/hcatalog/conf/proto-hive-site.xml
    hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java
    hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java
    hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java
    hive/branches/tez/hcatalog/pom.xml
    hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf
    hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/pig.conf
    hive/branches/tez/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java
    hive/branches/tez/hcatalog/src/test/e2e/templeton/README.txt
    hive/branches/tez/hcatalog/src/test/e2e/templeton/build.xml
    hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission2.conf
    hive/branches/tez/hcatalog/storage-handlers/hbase/build.xml
    hive/branches/tez/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java
    hive/branches/tez/ivy/ivysettings.xml
    hive/branches/tez/ivy/libraries.properties
    hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java
    hive/branches/tez/metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql
    hive/branches/tez/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql
    hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.12.0.oracle.sql
    hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql
    hive/branches/tez/metastore/scripts/upgrade/oracle/upgrade-0.11.0-to-0.12.0.oracle.sql
    hive/branches/tez/metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql
    hive/branches/tez/metastore/scripts/upgrade/postgres/hive-schema-0.12.0.postgres.sql
    hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java
    hive/branches/tez/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/io/orc/OrcProto.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStringColLikeStringScalar.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ColumnStatisticsImpl.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/io/orc/StringColumnStatistics.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContext.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContextImpl.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/PTFDesc.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/PTFDeserializer.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultAuthenticator.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NPath.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/Noop.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMap.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/TableFunctionEvaluator.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/TableFunctionResolver.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java
    hive/branches/tez/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto
    hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
    hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
    hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
    hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java
    hive/branches/tez/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
    hive/branches/tez/ql/src/test/queries/clientpositive/alter_varchar1.q
    hive/branches/tez/ql/src/test/queries/clientpositive/orc_create.q
    hive/branches/tez/ql/src/test/queries/clientpositive/stats_partscan_1.q
    hive/branches/tez/ql/src/test/queries/clientpositive/varchar_1.q
    hive/branches/tez/ql/src/test/queries/clientpositive/varchar_nested_types.q
    hive/branches/tez/ql/src/test/queries/clientpositive/varchar_udf1.q
    hive/branches/tez/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
    hive/branches/tez/ql/src/test/resources/orc-file-dump.out
    hive/branches/tez/ql/src/test/results/clientpositive/alter_varchar1.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/combine2.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/ctas.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/input12.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/input39.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_1.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_11.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_12.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_13.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_2.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_3.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_4.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_5.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_6.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_dml_9.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_query_multiskew_1.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_query_multiskew_2.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_query_oneskew_1.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/orc_create.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/skewjoin_union_remove_1.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/skewjoin_union_remove_2.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/stats_partscan_1.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/truncate_column_list_bucket.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/union_remove_10.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/union_remove_23.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/union_remove_4.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/union_remove_5.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/varchar_1.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/varchar_nested_types.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/varchar_udf1.q.out
    hive/branches/tez/ql/src/test/templates/TestBeeLineDriver.vm
    hive/branches/tez/ql/src/test/templates/TestCliDriver.vm
    hive/branches/tez/ql/src/test/templates/TestNegativeCliDriver.vm
    hive/branches/tez/ql/src/test/templates/TestParse.vm
    hive/branches/tez/ql/src/test/templates/TestParseNegative.vm
    hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveVarcharWritable.java
    hive/branches/tez/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorConverters.java
    hive/branches/tez/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java
    hive/branches/tez/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
    hive/branches/tez/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
    hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
    hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
    hive/branches/tez/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
    hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
    hive/branches/tez/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
    hive/branches/tez/testutils/ptest2/src/main/resources/batch-exec.vm
    hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java
    hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt
    hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt

Propchange: hive/branches/tez/
------------------------------------------------------------------------------
  Merged /hive/trunk:r1527868-1529345

Modified: hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java (original)
+++ hive/branches/tez/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java Fri Oct  4 23:35:50 2013
@@ -367,7 +367,7 @@ public class QTestGenTask extends Task {
       
       Collections.sort(qFiles);
       for (File qFile : qFiles) {
-        qFilesMap.put(qFile.getName(), getEscapedCanonicalPath(qFile));
+        qFilesMap.put(qFile.getName(), relativePath(hiveRootDir, qFile));
       }
 
       // Make sure the output directory exists, if it doesn't
@@ -418,12 +418,12 @@ public class QTestGenTask extends Task {
       // For each of the qFiles generate the test
       VelocityContext ctx = new VelocityContext();
       ctx.put("className", className);
-      ctx.put("hiveRootDir", getEscapedCanonicalPath(hiveRootDir));
-      ctx.put("queryDir", getEscapedCanonicalPath(queryDir));
+      ctx.put("hiveRootDir", escapePath(hiveRootDir.getCanonicalPath()));
+      ctx.put("queryDir", relativePath(hiveRootDir, queryDir));
       ctx.put("qfiles", qFiles);
       ctx.put("qfilesMap", qFilesMap);
-      ctx.put("resultsDir", getEscapedCanonicalPath(resultsDir));
-      ctx.put("logDir", getEscapedCanonicalPath(logDir));
+      ctx.put("resultsDir", relativePath(hiveRootDir, resultsDir));
+      ctx.put("logDir", relativePath(hiveRootDir, logDir));
       ctx.put("clusterMode", clusterMode);
       ctx.put("hadoopVersion", hadoopVersion);
 
@@ -447,8 +447,10 @@ public class QTestGenTask extends Task {
       throw new BuildException("Generation failed", e);
     }
   }
-  
-  private static String getEscapedCanonicalPath(File file) throws IOException {
+  private String relativePath(File hiveRootDir, File file) {
+    return escapePath(hiveRootDir.toURI().relativize(file.toURI()).getPath());
+  }  
+  private static String escapePath(String path) {
     if (System.getProperty("os.name").toLowerCase().startsWith("win")) {
       // Escape the backward slash in CanonicalPath if the unit test runs on windows
       // e.g. dir.getCanonicalPath() gets the absolute path of local
@@ -456,8 +458,8 @@ public class QTestGenTask extends Task {
       // in compiler error in windows. Reason : the canonical path contains backward
       // slashes "C:\temp\etc\" and it is not a valid string in Java
       // unless we escape the backward slashes.
-      return file.getCanonicalPath().replace("\\", "\\\\");
+      return path.replace("\\", "\\\\");
     }
-    return file.getCanonicalPath();
+    return path;
   }
 }

Modified: hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java (original)
+++ hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java Fri Oct  4 23:35:50 2013
@@ -22,7 +22,7 @@ import java.util.IllegalFormatException;
 public class HiveSchemaHelper {
   public static final String DB_DERBY = "derby";
   public static final String DB_MYSQL = "mysql";
-  public static final String DB_POSTGRACE = "postgrace";
+  public static final String DB_POSTGRACE = "postgres";
   public static final String DB_ORACLE = "oracle";
 
   public interface NestedScriptParser {
@@ -225,8 +225,8 @@ public class HiveSchemaHelper {
       if (!isNestedScript(dbCommand)) {
         throw new IllegalArgumentException("Not a nested script format " + dbCommand);
       }
-      // remove ending ';'
-      return dbCommand.replace(";", "");
+      // remove ending ';' and starting '@'
+      return dbCommand.replace(";", "").replace(ORACLE_NESTING_TOKEN, "");
     }
 
     @Override

Modified: hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java (original)
+++ hive/branches/tez/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java Fri Oct  4 23:35:50 2013
@@ -366,6 +366,7 @@ public class HiveSchemaTool {
       beeLine.getOpts().setSilent(true);
     }
     beeLine.getOpts().setAllowMultiLineCommand(false);
+    beeLine.getOpts().setIsolation("TRANSACTION_READ_COMMITTED");
     int status = beeLine.begin(argList.toArray(new String[0]), null);
     if (status != 0) {
       throw new IOException("Schema script failed, errorcode " + status);

Modified: hive/branches/tez/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java (original)
+++ hive/branches/tez/beeline/src/test/org/apache/hive/beeline/src/test/TestSchemaTool.java Fri Oct  4 23:35:50 2013
@@ -30,6 +30,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfo;
 import org.apache.hive.beeline.HiveSchemaHelper;
 import org.apache.hive.beeline.HiveSchemaHelper.NestedScriptParser;
 import org.apache.hive.beeline.HiveSchemaTool;
@@ -100,7 +101,8 @@ public class TestSchemaTool extends Test
    * @throws Exception
    */
   public void testSchemaInit() throws Exception {
-    schemaTool.doInit("0.12.0");
+    schemaTool.doInit(MetaStoreSchemaInfo.getHiveSchemaVersion());
+    schemaTool.verifySchemaVersion();
     }
 
   /**
@@ -321,6 +323,51 @@ public class TestSchemaTool extends Test
     assertEquals(expectedSQL, flattenedSql);
   }
 
+  /**
+   * Test nested script formatting
+   * @throws Exception
+   */
+  public void testNestedScriptsForOracle() throws Exception {
+    String childTab1 = "childTab1";
+    String childTab2 = "childTab2";
+    String parentTab = "fooTab";
+
+    String childTestScript1[] = {
+      "-- this is a comment ",
+      "DROP TABLE IF EXISTS " + childTab1 + ";",
+      "CREATE TABLE " + childTab1 + "(id INTEGER);",
+      "DROP TABLE " + childTab1 + ";"
+    };
+    String childTestScript2[] = {
+        "-- this is a comment",
+        "DROP TABLE IF EXISTS " + childTab2 + ";",
+        "CREATE TABLE " + childTab2 + "(id INTEGER);",
+        "-- this is also a comment",
+        "DROP TABLE " + childTab2 + ";"
+    };
+
+    String parentTestScript[] = {
+        " -- this is a comment",
+        "DROP TABLE IF EXISTS " + parentTab + ";",
+        " -- this is another comment ",
+        "CREATE TABLE " + parentTab + "(id INTEGER);",
+        "@" + generateTestScript(childTestScript1).getName() + ";",
+        "DROP TABLE " + parentTab + ";",
+        "@" + generateTestScript(childTestScript2).getName() + ";",
+        "--ending comment ",
+      };
+
+    File testScriptFile = generateTestScript(parentTestScript);
+    String flattenedSql = HiveSchemaTool.buildCommand(
+        HiveSchemaHelper.getDbCommandParser("oracle"),
+        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    assertFalse(flattenedSql.contains("@"));
+    assertFalse(flattenedSql.contains("comment"));
+    assertTrue(flattenedSql.contains(childTab1));
+    assertTrue(flattenedSql.contains(childTab2));
+    assertTrue(flattenedSql.contains(parentTab));
+  }
+
   private File generateTestScript(String [] stmts) throws IOException {
     File testScriptFile = File.createTempFile("schematest", ".sql");
     testScriptFile.deleteOnExit();

Modified: hive/branches/tez/bin/hive
URL: http://svn.apache.org/viewvc/hive/branches/tez/bin/hive?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/bin/hive (original)
+++ hive/branches/tez/bin/hive Fri Oct  4 23:35:50 2013
@@ -116,8 +116,7 @@ elif [ "${HIVE_AUX_JARS_PATH}" != "" ]; 
       HIVE_AUX_JARS_PATH=`echo $HIVE_AUX_JARS_PATH | sed 's/;/,/g'`
   fi
   AUX_CLASSPATH=${HIVE_AUX_JARS_PATH}
-  AUX_PARAM=file://${HIVE_AUX_JARS_PATH}
-  AUX_PARAM=`echo $AUX_PARAM | sed 's/:/,file:\/\//g'`
+  AUX_PARAM="file://$(echo ${HIVE_AUX_JARS_PATH} | sed 's/:/,file:\/\//g')"
 fi
 
 # adding jars from auxlib directory

Modified: hive/branches/tez/build-common.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/build-common.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/build-common.xml (original)
+++ hive/branches/tez/build-common.xml Fri Oct  4 23:35:50 2013
@@ -418,6 +418,10 @@
   <!-- target to run the tests -->
   <target name="test"
     depends="test-conditions,gen-test,compile-test,test-jar,test-init">
+    <antcall target="testonly" />
+  </target>
+
+  <target name="testonly" depends="test-conditions,test-init">
     <echo message="Project: ${ant.project.name}"/>
     <property name="hadoop.testcp" refid="test.classpath"/>
     <if>
@@ -488,6 +492,7 @@
       <sysproperty key="hadoop.bin.path" value="${test.hadoop.bin.path}${junit.script.extension}"/>
       <sysproperty key="test.concurrency.num.threads" value="${test.concurrency.num.threads}"/>
       <sysproperty key="hive.home" value="${hive.root}/build/dist"/>
+      <sysproperty key="qfile" value="${qfile}"/>
       <jvmarg line="${junit.jvm.args}"/>
 
       <classpath refid="test.local.classpath"/>

Modified: hive/branches/tez/build.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/build.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/build.xml (original)
+++ hive/branches/tez/build.xml Fri Oct  4 23:35:50 2013
@@ -347,23 +347,38 @@
     <echo message="Project: ${ant.project.name}"/>
     <iterate target="compile-test" iterate="${iterate.hive.tests}"/>
   </target>
-  
+
   <target name="test" depends="clean-test,jar-test" description="Run tests">
+    <property name="test.target.name" value="test"/>
+    <antcall target="testonly" />
+  </target>
+
+
+
+  <!-- target to run the tests -->
+  <target name="testonly">
+    <if>
+      <not><isset property="test.target.name"/></not>
+      <then><property name="test.target.name" value="testonly"/></then>
+    </if>
     <echo message="Project: ${ant.project.name}"/>
     <if>
+      <or>
       <isset property="module"/>
+      <equals arg1="${test.target.name}" arg2="testonly"/>
+      </or>
       <else>
         <antcall target="test-shims">
           <param name="hadoop.version.ant-internal" value="${hadoop.security.version}" />
         </antcall>
       </else>
     </if>
-    <condition property="target.module" value="${module}" else="${iterate.hive.test}">
+    <condition property="target.module" value="${module}" else="${iterate.hive.tests}">
       <isset property="module"/>
     </condition>
     <for keepgoing="${test.continue.on.failure}" list="${target.module}" param="module">
       <sequential>
-        <ant antfile="@{module}/build.xml" target="test" inheritAll="false" inheritRefs="true">
+        <ant antfile="@{module}/build.xml" target="${test.target.name}" inheritAll="false" inheritRefs="true">
           <property name="build.dir.hive" location="${build.dir.hive}"/>
           <property name="is-offline" value="${is-offline}"/>
         </ant>
@@ -549,6 +564,7 @@
         <exclude name="**/*high-scale-lib-*"/>
         <exclude name="**/hamcrest-core-*jar"/>
         <exclude name="**/junit*.jar"/>
+        <exclude name="**/kryo*.jar"/>
         <exclude name="**/asm*.jar"/>
         <exclude name="**/mockito*.jar"/>
         <exclude name="**/velocity*.jar"/>

Modified: hive/branches/tez/common/build.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/common/build.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/common/build.xml (original)
+++ hive/branches/tez/common/build.xml Fri Oct  4 23:35:50 2013
@@ -54,6 +54,14 @@ to call at top-level: ant deploy-contrib
   <!-- target to run the tests -->
   <target name="test"
   	depends="test-conditions,gen-test,compile-test,test-jar,test-init">
+    <antcall target="testonly" />
+  </target>
+
+
+
+  <!-- target to run the tests -->
+  <target name="testonly"
+    depends="test-conditions,test-init">
     <echo message="Project: ${ant.project.name}"/>
     <junit showoutput="${test.output}" printsummary="yes" haltonfailure="no"
            fork="yes" maxmemory="512m" dir="${basedir}" timeout="${test.junit.timeout}"

Modified: hive/branches/tez/common/src/test/org/apache/hadoop/hive/common/type/TestHiveVarchar.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/common/src/test/org/apache/hadoop/hive/common/type/TestHiveVarchar.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/common/src/test/org/apache/hadoop/hive/common/type/TestHiveVarchar.java (original)
+++ hive/branches/tez/common/src/test/org/apache/hadoop/hive/common/type/TestHiveVarchar.java Fri Oct  4 23:35:50 2013
@@ -106,6 +106,22 @@ public class TestHiveVarchar extends Tes
         assertEquals(strLen, enforcedString.codePointCount(0, enforcedString.length()));
       }
     }
+
+    HiveVarchar vc1 = new HiveVarchar("0123456789", 10);
+    assertEquals(10, vc1.getCharacterLength());
+
+    // Changing string value; getCharacterLength() should update accordingly
+    vc1.setValue("012345678901234");
+    assertEquals(15, vc1.getCharacterLength());
+
+    vc1.setValue("01234", -1);
+    assertEquals(5, vc1.getCharacterLength());
+
+    vc1.setValue(new HiveVarchar("0123456789", -1));
+    assertEquals(10, vc1.getCharacterLength());
+
+    vc1.setValue(new HiveVarchar("01234", -1), -1);
+    assertEquals(5, vc1.getCharacterLength());
   }
 
   public void testComparison() throws Exception {

Modified: hive/branches/tez/hbase-handler/src/test/templates/TestHBaseCliDriver.vm
URL: http://svn.apache.org/viewvc/hive/branches/tez/hbase-handler/src/test/templates/TestHBaseCliDriver.vm?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hbase-handler/src/test/templates/TestHBaseCliDriver.vm (original)
+++ hive/branches/tez/hbase-handler/src/test/templates/TestHBaseCliDriver.vm Fri Oct  4 23:35:50 2013
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.session
 
 public class $className extends TestCase {
 
+  private static final String HIVE_ROOT = HBaseQTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
   private HBaseQTestUtil qt;
   private HBaseTestSetup setup;
 
@@ -42,16 +43,13 @@ public class $className extends TestCase
   @Override
   protected void setUp() {
     try {
-      boolean miniMR = false;
-      if ("$clusterMode".equals("miniMR")) {
-        miniMR = true;
-      }
-      qt = new HBaseQTestUtil("$resultsDir", "$logDir", miniMR, setup);
+      boolean miniMR = "$clusterMode".equals("miniMR");
+      qt = new HBaseQTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR, setup);
 
     } catch (Exception e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.flush();
+      System.err.flush();
       fail("Unexpected exception in setup");
     }
   }
@@ -62,21 +60,33 @@ public class $className extends TestCase
       qt.shutdown();
     }
     catch (Exception e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.flush();
+      System.err.flush();
       fail("Unexpected exception in tearDown");
     }
   }
 
   public static Test suite() {
+    Set<String> qFilesToExecute = new HashSet<String>();
+    String qFiles = System.getProperty("qfile", "").trim();
+    if(!qFiles.isEmpty()) {
+      for(String qFile : qFiles.split(",")) {
+        qFile = qFile.trim();
+        if(!qFile.isEmpty()) {
+          qFilesToExecute.add(qFile);
+        }
+      }
+    }
     TestSuite suite = new TestSuite();
     HBaseTestSetup setup = new HBaseTestSetup(suite);
 #foreach ($qf in $qfiles)
   #set ($fname = $qf.getName())
   #set ($eidx = $fname.indexOf('.'))
   #set ($tname = $fname.substring(0, $eidx))
-    suite.addTest(new $className("testCliDriver_$tname", setup));
+    if(qFilesToExecute.isEmpty() || qFilesToExecute.contains("$fname")) {
+      suite.addTest(new $className("testCliDriver_$tname", setup));
+    }
 #end
     return setup;
   }
@@ -85,9 +95,9 @@ public class $className extends TestCase
   #set ($fname = $qf.getName())
   #set ($eidx = $fname.indexOf('.'))
   #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qf.getCanonicalPath().replaceAll("\\","\\\\"))
+  #set ($fpath = $qfilesMap.get($fname))
   public void testCliDriver_$tname() throws Exception {
-    runTest("$tname", "$fname", "$fpath");
+    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
   }
 
 #end
@@ -95,12 +105,12 @@ public class $className extends TestCase
   private void runTest(String tname, String fname, String fpath) throws Exception {
     long startTime = System.currentTimeMillis();
     try {
-      System.out.println("Begin query: " + fname);
+      System.err.println("Begin query: " + fname);
 
       qt.addFile(fpath);
 
       if (qt.shouldBeSkipped(fname)) {
-        System.out.println("Test " + fname + " skipped");
+        System.err.println("Test " + fname + " skipped");
         return;
       }
 
@@ -118,15 +128,15 @@ public class $className extends TestCase
       qt.clearPostTestEffects();
 
     } catch (Throwable e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.println("Failed query: " + fname);
-      System.out.flush();
+      System.err.println("Failed query: " + fname);
+      System.err.flush();
       fail("Unexpected exception");
     }
 
     long elapsedTime = System.currentTimeMillis() - startTime;
-    System.out.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
     assertTrue("Test passed", true);
   }
 }

Modified: hive/branches/tez/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm
URL: http://svn.apache.org/viewvc/hive/branches/tez/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm (original)
+++ hive/branches/tez/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm Fri Oct  4 23:35:50 2013
@@ -12,6 +12,7 @@ import org.apache.hadoop.hive.hbase.HBas
 
 public class $className extends TestCase {
 
+  private static final String HIVE_ROOT = HBaseQTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
   private HBaseQTestUtil qt;
   private HBaseTestSetup setup;
 
@@ -24,17 +25,14 @@ public class $className extends TestCase
   @Override
   protected void setUp() {
     try {
-      boolean miniMR = false;
-      if ("$clusterMode".equals("miniMR")) {
-        miniMR = true;
-      }
+      boolean miniMR = "$clusterMode".equals("miniMR");
 
-      qt = new HBaseQTestUtil("$resultsDir", "$logDir", miniMR, setup);
+      qt = new HBaseQTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR, setup);
 
     } catch (Exception e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.flush();
+      System.err.flush();
       fail("Unexpected exception in setup");
     }
   }
@@ -45,21 +43,33 @@ public class $className extends TestCase
       qt.shutdown();
     }
     catch (Exception e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.flush();
+      System.err.flush();
       fail("Unexpected exception in tearDown");
     }
   }
 
   public static Test suite() {
+    Set<String> qFilesToExecute = new HashSet<String>();
+    String qFiles = System.getProperty("qfile", "").trim();
+    if(!qFiles.isEmpty()) {
+      for(String qFile : qFiles.split(",")) {
+        qFile = qFile.trim();
+        if(!qFile.isEmpty()) {
+          qFilesToExecute.add(qFile);
+        }
+      }
+    }
     TestSuite suite = new TestSuite();
     HBaseTestSetup setup = new HBaseTestSetup(suite);
 #foreach ($qf in $qfiles)
   #set ($fname = $qf.getName())
   #set ($eidx = $fname.indexOf('.'))
   #set ($tname = $fname.substring(0, $eidx))
-    suite.addTest(new $className("testCliDriver_$tname", setup));
+    if(qFilesToExecute.isEmpty() || qFilesToExecute.contains("$fname")) {
+      suite.addTest(new $className("testCliDriver_$tname", setup));
+    }
 #end
     return setup;
   }
@@ -68,9 +78,9 @@ public class $className extends TestCase
   #set ($fname = $qf.getName())
   #set ($eidx = $fname.indexOf('.'))
   #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qf.getCanonicalPath().replaceAll("\\","\\\\"))
+  #set ($fpath = $qfilesMap.get($fname))
   public void testCliDriver_$tname() throws Exception {
-    runTest("$tname", "$fname", "$fpath");
+    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
   }
 
 #end
@@ -78,12 +88,12 @@ public class $className extends TestCase
   private void runTest(String tname, String fname, String fpath) throws Exception {
     long startTime = System.currentTimeMillis();
     try {
-      System.out.println("Begin query: " + fname);
+      System.err.println("Begin query: " + fname);
 
       qt.addFile(fpath);
 
       if (qt.shouldBeSkipped(fname)) {
-        System.out.println("Test " + fname + " skipped");
+        System.err.println("Test " + fname + " skipped");
         return;
       }
 
@@ -101,15 +111,15 @@ public class $className extends TestCase
       qt.clearPostTestEffects();
 
     } catch (Throwable e) {
-      System.out.println("Exception: " + e.getMessage());
+      System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();
-      System.out.println("Failed query: " + fname);
-      System.out.flush();
+      System.err.println("Failed query: " + fname);
+      System.err.flush();
       fail("Unexpected exception");
     }
 
     long elapsedTime = System.currentTimeMillis() - startTime;
-    System.out.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
     assertTrue("Test passed", true);
   }
 }

Modified: hive/branches/tez/hcatalog/build-support/ant/build-common.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/build-support/ant/build-common.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/build-support/ant/build-common.xml (original)
+++ hive/branches/tez/hcatalog/build-support/ant/build-common.xml Fri Oct  4 23:35:50 2013
@@ -93,6 +93,10 @@
   </target>
 
   <target name="test" depends="jar, compile-test">
+    <antcall target="testonly" />
+  </target>
+
+  <target name="testonly">
     <echo message="${ant.project.name}"/>
     <_junit srcDir="${basedir}/src/test/java"/>
   </target>

Modified: hive/branches/tez/hcatalog/build.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/build.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/build.xml (original)
+++ hive/branches/tez/hcatalog/build.xml Fri Oct  4 23:35:50 2013
@@ -92,39 +92,47 @@
     -->
 
     <target name="gen-test" description="Generate tests, a no-op for hcat"/>
-
     <target name="test" depends="jar" description="run unit tests">
+      <property name="test.target.name" value="test"/>
+      <antcall target="testonly" />
+      <!-- One checkstyle run for the whole repo. Runs after junit tests
+      to piggyback on resolved jars. -->
+      <path id="checkstyle.class.path">
+          <fileset dir="core/build/lib/test"/>
+      </path>
+      <antcall target="checkstyle" inheritRefs="true"/>
+    </target>
+
+    <target name="testonly">
+        <if>
+          <not><isset property="test.target.name"/></not>
+          <then><property name="test.target.name" value="testonly"/></then>
+        </if>
         <mkdir dir="${test.result.dir}"/>
         <!-- Placed in a parallel structure so that the tests keep going
              even if some fail.  Otherwise a failure in one of the earlier ant
              call terminates the target and the rest do not run.  -->
         <parallel threadCount="1">
-            <ant target="test" dir="core" inheritAll="false">
+            <ant target="${test.target.name}" dir="core" inheritAll="false">
                 <property name="test.result.dir" location="${test.result.dir}"/>
             </ant>
-            <ant target="test" dir="hcatalog-pig-adapter" inheritAll="false">
+            <ant target="${test.target.name}" dir="hcatalog-pig-adapter" inheritAll="false">
                 <property name="test.result.dir" location="${test.result.dir}"/>
             </ant>
-            <ant target="test" dir="server-extensions" inheritAll="false">
+            <ant target="${test.target.name}" dir="server-extensions" inheritAll="false">
                 <property name="test.result.dir" location="${test.result.dir}"/>
             </ant>
-            <ant target="test" dir="webhcat/svr" inheritAll="false">
+            <ant target="${test.target.name}" dir="webhcat/svr" inheritAll="false">
                 <property name="test.result.dir" location="${test.result.dir}"/>
                 <property name="env.HIVE_HOME" value="${hive.dist.dir}"/>
                 <property name="env.HCAT_PREFIX" value="${hive.dist.dir}/hcatalog"/>
             </ant>
-            <ant target="test" dir="webhcat/java-client" inheritAll="false">
+            <ant target="${test.target.name}" dir="webhcat/java-client" inheritAll="false">
                 <property name="test.result.dir" location="${test.result.dir}"/>
             </ant>
-            <ant target="test" dir="storage-handlers/hbase" inheritAll="false">
+            <ant target="${test.target.name}" dir="storage-handlers/hbase" inheritAll="false">
                 <property name="test.result.dir" location="${test.result.dir}"/>
             </ant>
-            <!-- One checkstyle run for the whole repo. Runs after junit tests
-            to piggyback on resolved jars. -->
-            <path id="checkstyle.class.path">
-                <fileset dir="core/build/lib/test"/>
-            </path>
-            <antcall target="checkstyle" inheritRefs="true"/>
         </parallel>
     </target>
 

Modified: hive/branches/tez/hcatalog/conf/proto-hive-site.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/conf/proto-hive-site.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/conf/proto-hive-site.xml (original)
+++ hive/branches/tez/hcatalog/conf/proto-hive-site.xml Fri Oct  4 23:35:50 2013
@@ -114,7 +114,7 @@
 
 <property>
   <name>hive.security.authorization.manager</name>
-  <value>org.apache.hive.hcatalog.security.StorageDelegationAuthorizationProvider</value>
+  <value>org.apache.hcatalog.security.StorageDelegationAuthorizationProvider</value>
   <description>the hive client authorization manager class name.
   The user defined authorization class should implement interface org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider.
   HCatalog uses a model, where authorization checks are delegated to the storage layer (hdfs, hbase, ...).

Modified: hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java (original)
+++ hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java Fri Oct  4 23:35:50 2013
@@ -56,7 +56,7 @@ import org.apache.hadoop.security.UserGr
  * An AuthorizationProvider, which checks against the data access level permissions on HDFS.
  * It makes sense to eventually move this class to Hive, so that all hive users can
  * use this authorization model. 
- * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.HdfsAuthorizationProvider} instead
+ * @deprecated use {@link org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider}
  */
 public class HdfsAuthorizationProvider extends HiveAuthorizationProviderBase {
 

Modified: hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java (original)
+++ hive/branches/tez/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java Fri Oct  4 23:35:50 2013
@@ -42,7 +42,7 @@ import org.apache.hcatalog.mapreduce.HCa
 /**
  * A HiveAuthorizationProvider which delegates the authorization requests to 
  * the underlying AuthorizationProviders obtained from the StorageHandler.
- * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.StorageDelegationAuthorizationProvider} instead
+ * @deprecated 
  */
 public class StorageDelegationAuthorizationProvider extends HiveAuthorizationProviderBase {
 

Modified: hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java (original)
+++ hive/branches/tez/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java Fri Oct  4 23:35:50 2013
@@ -53,7 +53,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 /**
- * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.TestHdfsAuthorizationProvider} instead
+ * @deprecated 
  */
 public class TestHdfsAuthorizationProvider {
 
@@ -78,7 +78,7 @@ public class TestHdfsAuthorizationProvid
     conf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName());
     conf.setBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
     conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
-        StorageDelegationAuthorizationProvider.class.getCanonicalName());
+        StorageDelegationAuthorizationProvider.class.getName());
     conf.set("fs.pfile.impl", "org.apache.hadoop.fs.ProxyLocalFileSystem");
 
     whDir = System.getProperty("test.warehouse.dir", "/tmp/testhdfsauthorization_wh");

Modified: hive/branches/tez/hcatalog/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/pom.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/pom.xml (original)
+++ hive/branches/tez/hcatalog/pom.xml Fri Oct  4 23:35:50 2013
@@ -215,16 +215,6 @@
         <enabled>false</enabled>
       </snapshots>
     </repository>
-    <repository>
-      <id>sonatype-snapshots</id>
-      <url>https://oss.sonatype.org/content/repositories/snapshots/</url>
-      <releases>
-        <enabled>false</enabled>
-      </releases>
-      <snapshots>
-        <enabled>true</enabled>
-      </snapshots>
-    </repository>
   </repositories>
 
     <dependencies>

Modified: hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf Fri Oct  4 23:35:50 2013
@@ -225,9 +225,9 @@ jar :FUNCPATH:/testudf.jar org.apache.hi
                                 {
                                  'num' => 1
                                 ,'hcat_prep'=>q\drop table if exists hadoop_hbase_1;
-create table hadoop_hbase_1(key string, gpa string) STORED BY 'org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:gpa');\
+create table hadoop_hbase_1(key string, gpa string) STORED BY 'org.apache.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:gpa');\
                                 ,'hadoop' => q\
-jar :FUNCPATH:/testudf.jar org.apache.hive.hcatalog.utils.HBaseReadWrite -libjars :HCAT_JAR: :THRIFTSERVER: :INPATH:/studenttab10k hadoop_hbase_1 :OUTPATH:
+jar :FUNCPATH:/testudf.jar org.apache.hcatalog.utils.HBaseReadWrite -libjars :HCAT_JAR: :THRIFTSERVER: :INPATH:/studenttab10k hadoop_hbase_1 :OUTPATH:
 \,
                                 ,'sql' => q\select name, sum(gpa) from studenttab10k group by name;\
                                 ,'floatpostprocess' => 1

Modified: hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/pig.conf
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/pig.conf?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/pig.conf (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/hcatalog/tests/pig.conf Fri Oct  4 23:35:50 2013
@@ -319,15 +319,15 @@ store c into ':OUTPATH:';\
                                 {
                                  'num' => 1
                                 ,'hcat_prep'=>q\drop table if exists pig_hbase_1;
-create table pig_hbase_1(key string, age string, gpa string) STORED BY 'org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');\
+create table pig_hbase_1(key string, age string, gpa string) STORED BY 'org.apache.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');\
                                 ,'pig' => q\set hcat.hbase.output.bulkMode 'false'
 a = load ':INPATH:/studenttab10k' as (name:chararray, age:int, gpa:float);
 b = group a by name;
 c = foreach b generate group as name, AVG(a.age) as age, AVG(a.gpa) as gpa;
 d = foreach c generate name as key, (chararray)age, (chararray)gpa as gpa;
-store d into 'pig_hbase_1' using org.apache.hive.hcatalog.pig.HCatStorer();
+store d into 'pig_hbase_1' using org.apache.hcatalog.pig.HCatStorer();
 exec
-e = load 'pig_hbase_1' using org.apache.hive.hcatalog.pig.HCatLoader();
+e = load 'pig_hbase_1' using org.apache.hcatalog.pig.HCatLoader();
 store e into ':OUTPATH:';\,
                                 ,'result_table' => ['pig_hbase_1','?']
 				,'sql'   => [ 'select name, avg(cast(age as decimal(10,5))), avg(gpa) from studenttab10k group by name;', 'select name, avg(cast(age as decimal(10,5))), avg(gpa) from studenttab10k group by name;' ]
@@ -338,17 +338,17 @@ store e into ':OUTPATH:';\,
                                  # multiquery
                                  'num' => 2
                                 ,'hcat_prep'=>q\drop table if exists pig_hbase_2_1;
-create table pig_hbase_2_1(key string, age string, gpa string) STORED BY 'org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');
+create table pig_hbase_2_1(key string, age string, gpa string) STORED BY 'org.apache.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');
 drop table if exists pig_hbase_2_2;
-create table pig_hbase_2_2(key string, age string, gpa string) STORED BY 'org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');
+create table pig_hbase_2_2(key string, age string, gpa string) STORED BY 'org.apache.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');
 \
                                 ,'pig' => q\set hcat.hbase.output.bulkMode 'false'
 a = load ':INPATH:/studenttab10k' as (name:chararray, age:int, gpa:float);
 b = group a by name;
 c = foreach b generate group as name, AVG(a.age) as age, AVG(a.gpa) as gpa;
 d = foreach c generate name as key, (chararray)age, (chararray)gpa as gpa;
-store d into 'pig_hbase_2_1' using org.apache.hive.hcatalog.pig.HCatStorer();
-store d into 'pig_hbase_2_2' using org.apache.hive.hcatalog.pig.HCatStorer();\,
+store d into 'pig_hbase_2_1' using org.apache.hcatalog.pig.HCatStorer();
+store d into 'pig_hbase_2_2' using org.apache.hcatalog.pig.HCatStorer();\,
                                 ,'result_table' => ['pig_hbase_2_1','pig_hbase_2_2']
 				,'sql'   => [ 'select name, avg(cast(age as decimal(10,5))), avg(gpa) from studenttab10k group by name;', 'select name, avg(cast(age as decimal(10,5))), avg(gpa) from studenttab10k group by name;']
                                 ,'floatpostprocess' => 1

Modified: hive/branches/tez/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java Fri Oct  4 23:35:50 2013
@@ -48,7 +48,7 @@ import org.apache.hive.hcatalog.mapreduc
  * other columns. This is to simulate a typical operation in a map reduce
  * program to test that hcat hands the right data to the map reduce program
  *
- * Usage: hadoop jar org.apache.hive.hcatalog.utils.HBaseReadWrite -libjars
+ * Usage: hadoop jar org.apache.hcatalog.utils.HBaseReadWrite -libjars
  * &lt;hcat_jar&gt; * &lt;serveruri&gt; &lt;input_tablename&gt; &lt;output_tablename&gt; [filter]
  * If filter is given it will be provided as the partition to write to.
  */

Modified: hive/branches/tez/hcatalog/src/test/e2e/templeton/README.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/templeton/README.txt?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/templeton/README.txt (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/templeton/README.txt Fri Oct  4 23:35:50 2013
@@ -69,6 +69,7 @@ Setup
 2. Install perl and following perl modules  (cpan -i <MODULE_NAME>)
 * IPC::Run
 * JSON
+* JSON::Path
 * Data::Dump
 * Number::Compare
 * Text::Glob
@@ -86,15 +87,15 @@ Tips:
 3. Copy contents of src/test/e2e/templeton/inpdir to hdfs
 (e.g. ./bin/hadoop fs -put ~/dev/hive/hcatalog/src/test/e2e/templeton/inpdir/ webhcate2e)
 
-4. You will need to two jars in the same HDFS directory as the contents of inpdir.  piggybank.jar, which can
-be obtained from Pig.  The second is the hadoop-examples.jar, which can be obtained from your Hadoop distribution.
-This should be called hexamples.jar when it is uploaded to HDFS.
+4. You will need to copy three jars in the same HDFS directory as the contents of inpdir.  piggybank.jar, which can
+be obtained from Pig and the other two are obtained from your Hadoop distribution.
+For Hadoop 1.x you would need to upload hadoop-examples.jar twice to HDFS one as hclient.jar and other as hexamples.jar.
+For Hadoop 2.x you would need to upload hadoop-mapreduce-client-jobclient.jar to HDFS as hclient.jar and hadoop-mapreduce-examples.jar to HDFS as hexamples.jar. 
 Also see http://hive.apache.org/docs/hcat_r0.5.0/rest_server_install.html#Hadoop+Distributed+Cache for notes on
 additional JAR files to copy to HDFS.
 
 5. Make sure TEMPLETON_HOME evnironment variable is set
 
-
 6. hadoop/conf/core-site.xml should have items described in
 http://hive.apache.org/docs/hcat_r0.5.0/rest_server_install.html#Permissions
 

Modified: hive/branches/tez/hcatalog/src/test/e2e/templeton/build.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/templeton/build.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/templeton/build.xml (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/templeton/build.xml Fri Oct  4 23:35:50 2013
@@ -114,6 +114,7 @@
             <arg value="${basedir}/tests/ddl.conf"/>
             <arg value="${basedir}/tests/jobstatus.conf"/>
             <arg value="${basedir}/tests/jobsubmission.conf"/>
+            <arg value="${basedir}/tests/jobsubmission2.conf"/>
         </exec>
     </target>
 

Modified: hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission2.conf
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission2.conf?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission2.conf (original)
+++ hive/branches/tez/hcatalog/src/test/e2e/templeton/tests/jobsubmission2.conf Fri Oct  4 23:35:50 2013
@@ -46,7 +46,7 @@ $cfg = 
      'method' => 'POST',
      'url' => ':TEMPLETON_URL:/templeton/v1/mapreduce/jar',
      'post_options' => ['user.name=:UNAME:','arg=-mt', 'arg=642000',
-                             'jar=:INPDIR_HDFS:/hexamples.jar', 'class=sleep', 'statusdir=/tmp' ],
+                             'jar=:INPDIR_HDFS:/hclient.jar', 'class=sleep', 'statusdir=/tmp' ],
      'json_field_substr_match' => { 'id' => '\d+'},
      'status_code' => 200,
      'check_job_created' => 1,

Modified: hive/branches/tez/hcatalog/storage-handlers/hbase/build.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/storage-handlers/hbase/build.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/storage-handlers/hbase/build.xml (original)
+++ hive/branches/tez/hcatalog/storage-handlers/hbase/build.xml Fri Oct  4 23:35:50 2013
@@ -127,14 +127,18 @@
 
     <!-- Run the unit tests -->
     <target name="test" depends="compile-test">
+      <antcall target="testonly" />
+    </target>
+     
+    <target name="testonly">
         <sequential>
             <delete dir="${test.log.dir}"/>
-	        <delete dir="${test.tmp.dir}" />
+            <delete dir="${test.tmp.dir}" />
             <delete dir="${test.data.dir}" />
-            <mkdir dir="${test.log.dir}"/>
-	        <mkdir dir="${test.tmp.dir}" />
+            <mkdir dir="${test.log.dir}"/> 
+            <mkdir dir="${test.tmp.dir}" />
             <mkdir dir="${test.data.dir}" />
-	        <junit showoutput="${test.output}" printsummary="yes" haltonfailure="no"
+            <junit showoutput="${test.output}" printsummary="yes" haltonfailure="no"
                    fork="yes" maxmemory="512m" dir="${basedir}" timeout="${test.timeout}"
                    errorProperty="tests.failed" failureProperty="tests.failed">
                 <classpath>

Modified: hive/branches/tez/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java (original)
+++ hive/branches/tez/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java Fri Oct  4 23:35:50 2013
@@ -22,6 +22,8 @@ package org.apache.hcatalog.hbase;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.classification.InterfaceAudience;
+import org.apache.hadoop.hive.common.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -34,8 +36,11 @@ import org.apache.hadoop.hive.ql.securit
 /**
  * This class is an implementation of HiveAuthorizationProvider to provide
  * authorization functionality for HBase tables.
+ * @deprecated 
  */
-class HBaseAuthorizationProvider implements HiveAuthorizationProvider {
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HBaseAuthorizationProvider implements HiveAuthorizationProvider {
 
   @Override
   public Configuration getConf() {

Modified: hive/branches/tez/ivy/ivysettings.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/ivy/ivysettings.xml?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/ivy/ivysettings.xml (original)
+++ hive/branches/tez/ivy/ivysettings.xml Fri Oct  4 23:35:50 2013
@@ -47,10 +47,6 @@
              checkmodified="${ivy.checkmodified}" 
              changingPattern="${ivy.changingPattern}"/>
 
-    <ibiblio name="sonatype-snapshot" root="https://oss.sonatype.org/content/repositories/snapshots/" m2compatible="true"
-             checkmodified="${ivy.checkmodified}" 
-             changingPattern="${ivy.changingPattern}"/>
-
     <url name="datanucleus-repo" m2compatible="true">
       <artifact pattern="${datanucleus.repo}/[organisation]/[module]/[revision]/[module]-[revision](-[classifier]).[ext]"/>
     </url>
@@ -72,7 +68,6 @@
       <resolver ref="maven2"/>
       <resolver ref="datanucleus-repo"/>
       <resolver ref="sourceforge"/>
-      <resolver ref="sonatype-snapshot"/>
     </chain>
 
     <chain name="internal" dual="true">
@@ -82,13 +77,11 @@
       <resolver ref="maven2"/>
       <resolver ref="datanucleus-repo"/>
       <resolver ref="sourceforge"/>
-      <resolver ref="sonatype-snapshot"/>
     </chain>
 
     <chain name="external">
       <resolver ref="maven2"/>
       <resolver ref="datanucleus-repo"/>
-      <resolver ref="sonatype-snapshot"/>
     </chain>
 
   </resolvers>

Modified: hive/branches/tez/ivy/libraries.properties
URL: http://svn.apache.org/viewvc/hive/branches/tez/ivy/libraries.properties?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/ivy/libraries.properties (original)
+++ hive/branches/tez/ivy/libraries.properties Fri Oct  4 23:35:50 2013
@@ -54,7 +54,7 @@ jetty.version=6.1.26
 jline.version=0.9.94
 json.version=20090211
 junit.version=4.10
-kryo.version=2.22-SNAPSHOT
+kryo.version=2.22
 libfb303.version=0.9.0
 libthrift.version=0.9.0
 log4j.version=1.2.16

Modified: hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java (original)
+++ hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java Fri Oct  4 23:35:50 2013
@@ -20,6 +20,12 @@ package org.apache.hive.jdbc;
 
 import static org.apache.hadoop.hive.ql.exec.ExplainTask.EXPL_COLUMN_NAME;
 import static org.apache.hadoop.hive.ql.processors.SetProcessor.SET_COLUMN_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
@@ -39,8 +45,6 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.regex.Pattern;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
@@ -50,13 +54,17 @@ import org.apache.hive.service.cli.opera
 import org.apache.hive.service.cli.operation.ClassicTableTypeMapping.ClassicTableTypes;
 import org.apache.hive.service.cli.operation.HiveTableTypeMapping;
 import org.apache.hive.service.cli.operation.TableTypeMappingFactory.TableTypeMappings;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 
 /**
  * TestJdbcDriver2
  *
  */
-public class TestJdbcDriver2 extends TestCase {
+public class TestJdbcDriver2 {
   private static final String driverName = "org.apache.hive.jdbc.HiveDriver";
   private static final String tableName = "testHiveJdbcDriver_Table";
   private static final String tableComment = "Simple table";
@@ -72,10 +80,10 @@ public class TestJdbcDriver2 extends Tes
   private final Path dataFilePath;
   private final Path dataTypeDataFilePath;
   private Connection con;
-  private boolean standAloneServer = false;
+  private static boolean standAloneServer = false;
+  private static final float floatCompareDelta = 0.0001f;
 
-  public TestJdbcDriver2(String name) {
-    super(name);
+  public TestJdbcDriver2() {
     conf = new HiveConf(TestJdbcDriver2.class);
     String dataFileDir = conf.get("test.data.files").replace('\\', '/')
         .replace("c:", "");
@@ -85,19 +93,36 @@ public class TestJdbcDriver2 extends Tes
         .getProperty("test.service.standalone.server"));
   }
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @BeforeClass
+  public static void setUpBeforeClass() throws SQLException, ClassNotFoundException{
     Class.forName(driverName);
-    if (standAloneServer) {
-      // get connection
-      con = DriverManager.getConnection("jdbc:hive2://localhost:10000/default",
-          "", "");
-    } else {
-      con = DriverManager.getConnection("jdbc:hive2://", "", "");
+    Connection con1 = getConnection();
+
+    Statement stmt1 = con1.createStatement();
+    assertNotNull("Statement is null", stmt1);
+
+    stmt1.execute("set hive.support.concurrency = false");
+
+    DatabaseMetaData metadata = con1.getMetaData();
+
+    // Drop databases created by other test cases
+    ResultSet databaseRes = metadata.getSchemas();
+
+    while(databaseRes.next()){
+      String db = databaseRes.getString(1);
+      if(!db.equals("default")){
+        System.err.println("Dropping database " + db);
+        stmt1.execute("DROP DATABASE " + db + " CASCADE");
+      }
     }
-    assertNotNull("Connection is null", con);
-    assertFalse("Connection should not be closed", con.isClosed());
+    stmt1.close();
+    con1.close();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    con = getConnection();
+
     Statement stmt = con.createStatement();
     assertNotNull("Statement is null", stmt);
 
@@ -110,7 +135,6 @@ public class TestJdbcDriver2 extends Tes
       fail(ex.toString());
     }
 
-    ResultSet res;
     // create table
     stmt.execute("create table " + tableName
         + " (under_col int comment 'the under column', value string) comment '"
@@ -179,10 +203,23 @@ public class TestJdbcDriver2 extends Tes
         +"' as select * from "+ tableName);
   }
 
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
+  private static Connection getConnection() throws SQLException {
+    Connection con1;
+    if (standAloneServer) {
+      // get connection
+      con1 = DriverManager.getConnection("jdbc:hive2://localhost:10000/default",
+          "", "");
+    } else {
+      con1 = DriverManager.getConnection("jdbc:hive2://", "", "");
+    }
+    assertNotNull("Connection is null", con1);
+    assertFalse("Connection should not be closed", con1.isClosed());
 
+    return con1;
+  }
+
+  @After
+  public void tearDown() throws Exception {
     // drop table
     Statement stmt = con.createStatement();
     assertNotNull("Statement is null", stmt);
@@ -205,6 +242,7 @@ public class TestJdbcDriver2 extends Tes
         expectedException);
   }
 
+  @Test
   public void testBadURL() throws Exception {
     checkBadUrl("jdbc:hive2://localhost:10000;principal=test");
     checkBadUrl("jdbc:hive2://localhost:10000;" +
@@ -223,6 +261,7 @@ public class TestJdbcDriver2 extends Tes
     }
   }
 
+  @Test
   public void testDataTypes2() throws Exception {
     Statement stmt = con.createStatement();
 
@@ -238,6 +277,8 @@ public class TestJdbcDriver2 extends Tes
     }
 
   }
+
+  @Test
   public void testErrorDiag() throws SQLException {
     Statement stmt = con.createStatement();
 
@@ -268,6 +309,7 @@ public class TestJdbcDriver2 extends Tes
    * verify 'explain ...' resultset
    * @throws SQLException
    */
+  @Test
   public void testExplainStmt() throws SQLException {
     Statement stmt = con.createStatement();
 
@@ -276,12 +318,15 @@ public class TestJdbcDriver2 extends Tes
             "c1*2, sentences(null, null, null) as b from " + dataTypeTableName + " limit 1");
 
     ResultSetMetaData md = res.getMetaData();
-    assertEquals(md.getColumnCount(), 1); // only one result column
-    assertEquals(md.getColumnLabel(1), EXPL_COLUMN_NAME); // verify the column name
+    // only one result column
+    assertEquals(md.getColumnCount(), 1);
+    // verify the column name
+    assertEquals(md.getColumnLabel(1), EXPL_COLUMN_NAME);
     //verify that there is data in the resultset
     assertTrue("Nothing returned explain", res.next());
   }
 
+  @Test
   public void testPrepareStatement() {
 
     String sql = "from (select count(1) from "
@@ -398,6 +443,7 @@ public class TestJdbcDriver2 extends Tes
    * of PreparedStatement interface
    * @throws Exception
    */
+  @Test
   public void testExecutePreparedStatement() throws Exception {
     String key = "testKey";
     String val1 = "val1";
@@ -433,11 +479,13 @@ public class TestJdbcDriver2 extends Tes
     assertEquals("Conf value should be set by execute()", expectedVal, result);
   }
 
+  @Test
   public final void testSelectAll() throws Exception {
     doTestSelectAll(tableName, -1, -1); // tests not setting maxRows (return all)
     doTestSelectAll(tableName, 0, -1); // tests setting maxRows to 0 (return all)
   }
 
+  @Test
   public final void testSelectAllPartioned() throws Exception {
     doTestSelectAll(partitionedTableName, -1, -1); // tests not setting maxRows
     // (return all)
@@ -445,14 +493,17 @@ public class TestJdbcDriver2 extends Tes
     // (return all)
   }
 
+  @Test
   public final void testSelectAllMaxRows() throws Exception {
     doTestSelectAll(tableName, 100, -1);
   }
 
+  @Test
   public final void testSelectAllFetchSize() throws Exception {
     doTestSelectAll(tableName, 100, 20);
   }
 
+  @Test
   public void testNullType() throws Exception {
     Statement stmt = con.createStatement();
     try {
@@ -466,6 +517,7 @@ public class TestJdbcDriver2 extends Tes
 
   // executeQuery should always throw a SQLException,
   // when it executes a non-ResultSet query (like create)
+  @Test
   public void testExecuteQueryException() throws Exception {
     Statement stmt = con.createStatement();
     try {
@@ -506,6 +558,7 @@ public class TestJdbcDriver2 extends Tes
     fail(e.toString());
   }
 
+  @Test
   public void testNullResultSet() throws Exception {
     List<String> setupQueries = new ArrayList<String>();
     String testQuery;
@@ -541,6 +594,7 @@ public class TestJdbcDriver2 extends Tes
     stmt.close();
   }
 
+  @Test
   public void testCloseResultSet() throws Exception {
     Statement stmt = con.createStatement();
 
@@ -574,6 +628,7 @@ public class TestJdbcDriver2 extends Tes
     assertTrue(stmt.isClosed());
   }
 
+  @Test
   public void testDataTypes() throws Exception {
     Statement stmt = con.createStatement();
 
@@ -590,7 +645,7 @@ public class TestJdbcDriver2 extends Tes
     // getXXX returns 0 for numeric types, false for boolean and null for other
     assertEquals(0, res.getInt(1));
     assertEquals(false, res.getBoolean(2));
-    assertEquals(0d, res.getDouble(3));
+    assertEquals(0d, res.getDouble(3), floatCompareDelta);
     assertEquals(null, res.getString(4));
     assertEquals(null, res.getString(5));
     assertEquals(null, res.getString(6));
@@ -598,7 +653,7 @@ public class TestJdbcDriver2 extends Tes
     assertEquals(null, res.getString(8));
     assertEquals(0, res.getByte(9));
     assertEquals(0, res.getShort(10));
-    assertEquals(0f, res.getFloat(11));
+    assertEquals(0f, res.getFloat(11), floatCompareDelta);
     assertEquals(0L, res.getLong(12));
     assertEquals(null, res.getString(13));
     assertEquals(null, res.getString(14));
@@ -615,7 +670,7 @@ public class TestJdbcDriver2 extends Tes
     assertTrue(res.next());
     assertEquals(-1, res.getInt(1));
     assertEquals(false, res.getBoolean(2));
-    assertEquals(-1.1d, res.getDouble(3));
+    assertEquals(-1.1d, res.getDouble(3), floatCompareDelta);
     assertEquals("", res.getString(4));
     assertEquals("[]", res.getString(5));
     assertEquals("{}", res.getString(6));
@@ -623,7 +678,7 @@ public class TestJdbcDriver2 extends Tes
     assertEquals("{\"r\":null,\"s\":null,\"t\":null}", res.getString(8));
     assertEquals(-1, res.getByte(9));
     assertEquals(-1, res.getShort(10));
-    assertEquals(-1.0f, res.getFloat(11));
+    assertEquals(-1.0f, res.getFloat(11), floatCompareDelta);
     assertEquals(-1, res.getLong(12));
     assertEquals("[]", res.getString(13));
     assertEquals("{}", res.getString(14));
@@ -641,7 +696,7 @@ public class TestJdbcDriver2 extends Tes
     assertTrue(res.next());
     assertEquals(1, res.getInt(1));
     assertEquals(true, res.getBoolean(2));
-    assertEquals(1.1d, res.getDouble(3));
+    assertEquals(1.1d, res.getDouble(3), floatCompareDelta);
     assertEquals("1", res.getString(4));
     assertEquals("[1,2]", res.getString(5));
     assertEquals("{1:\"x\",2:\"y\"}", res.getString(6));
@@ -649,7 +704,7 @@ public class TestJdbcDriver2 extends Tes
     assertEquals("{\"r\":\"a\",\"s\":9,\"t\":2.2}", res.getString(8));
     assertEquals(1, res.getByte(9));
     assertEquals(1, res.getShort(10));
-    assertEquals(1.0f, res.getFloat(11));
+    assertEquals(1.0f, res.getFloat(11), floatCompareDelta);
     assertEquals(1, res.getLong(12));
     assertEquals("[[\"a\",\"b\"],[\"c\",\"d\"]]", res.getString(13));
     assertEquals("{1:{11:12,13:14},2:{21:22}}", res.getString(14));
@@ -759,6 +814,7 @@ public class TestJdbcDriver2 extends Tes
     assertTrue("Statement should be closed", stmt.isClosed());
   }
 
+  @Test
   public void testErrorMessages() throws SQLException {
     String invalidSyntaxSQLState = "42000";
 
@@ -807,6 +863,7 @@ public class TestJdbcDriver2 extends Tes
         exceptionFound);
   }
 
+  @Test
   public void testShowTables() throws SQLException {
     Statement stmt = con.createStatement();
     assertNotNull("Statement is null", stmt);
@@ -825,10 +882,12 @@ public class TestJdbcDriver2 extends Tes
         + " not found in SHOW TABLES result set", testTableExists);
   }
 
+  @Test
   public void testMetaDataGetTables() throws SQLException {
     getTablesTest(ClassicTableTypes.TABLE.toString(), ClassicTableTypes.VIEW.toString());
   }
 
+  @Test
   public  void testMetaDataGetTablesHive() throws SQLException {
     Statement stmt = con.createStatement();
     stmt.execute("set " + HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING.varname +
@@ -836,6 +895,7 @@ public class TestJdbcDriver2 extends Tes
     getTablesTest(TableType.MANAGED_TABLE.toString(), TableType.VIRTUAL_VIEW.toString());
   }
 
+  @Test
   public  void testMetaDataGetTablesClassic() throws SQLException {
     Statement stmt = con.createStatement();
     stmt.execute("set " + HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING.varname +
@@ -907,6 +967,7 @@ public class TestJdbcDriver2 extends Tes
     assertEquals("Incorrect number of views found.", 1, cnt);
   }
 
+  @Test
   public void testMetaDataGetCatalogs() throws SQLException {
     ResultSet rs = (ResultSet)con.getMetaData().getCatalogs();
     ResultSetMetaData resMeta = rs.getMetaData();
@@ -916,6 +977,7 @@ public class TestJdbcDriver2 extends Tes
     assertFalse(rs.next());
   }
 
+  @Test
   public void testMetaDataGetSchemas() throws SQLException {
     ResultSet rs = (ResultSet)con.getMetaData().getSchemas();
     ResultSetMetaData resMeta = rs.getMetaData();
@@ -925,20 +987,21 @@ public class TestJdbcDriver2 extends Tes
 
     assertTrue(rs.next());
     assertEquals("default", rs.getString(1));
-    //    assertNull(rs.getString(2));
 
     assertFalse(rs.next());
     rs.close();
   }
 
-  //test default table types returned in
+  // test default table types returned in
   // Connection.getMetaData().getTableTypes()
+  @Test
   public void testMetaDataGetTableTypes() throws SQLException {
     metaDataGetTableTypeTest(new ClassicTableTypeMapping().getTableTypeNames());
   }
 
-  //test default table types returned in
+  // test default table types returned in
   // Connection.getMetaData().getTableTypes() when type config is set to "HIVE"
+  @Test
   public void testMetaDataGetHiveTableTypes() throws SQLException {
     Statement stmt = con.createStatement();
     stmt.execute("set " + HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING.varname +
@@ -947,8 +1010,9 @@ public class TestJdbcDriver2 extends Tes
     metaDataGetTableTypeTest(new HiveTableTypeMapping().getTableTypeNames());
   }
 
-  //test default table types returned in
+  // test default table types returned in
   // Connection.getMetaData().getTableTypes() when type config is set to "CLASSIC"
+  @Test
   public void testMetaDataGetClassicTableTypes() throws SQLException {
     Statement stmt = con.createStatement();
     stmt.execute("set " + HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING.varname +
@@ -979,6 +1043,7 @@ public class TestJdbcDriver2 extends Tes
     assertTrue("Found less tabletypes then we test for.", cnt >= tabletypes.size());
   }
 
+  @Test
   public void testMetaDataGetColumns() throws SQLException {
     Map<String[], Integer> tests = new HashMap<String[], Integer>();
     tests.put(new String[]{"testhivejdbcdriver\\_table", null}, 2);
@@ -1027,6 +1092,7 @@ public class TestJdbcDriver2 extends Tes
   /**
    * Validate the Metadata for the result set of a metadata getColumns call.
    */
+  @Test
   public void testMetaDataGetColumnsMetaData() throws SQLException {
     ResultSet rs = (ResultSet)con.getMetaData().getColumns(null, null
         , "testhivejdbcdriver\\_table", null);
@@ -1084,6 +1150,7 @@ public class TestJdbcDriver2 extends Tes
   }
    */
 
+  @Test
   public void testDescribeTable() throws SQLException {
     Statement stmt = con.createStatement();
     assertNotNull("Statement is null", stmt);
@@ -1102,6 +1169,7 @@ public class TestJdbcDriver2 extends Tes
     assertFalse("More results found than expected", res.next());
   }
 
+  @Test
   public void testDatabaseMetaData() throws SQLException {
     DatabaseMetaData meta = con.getMetaData();
 
@@ -1120,12 +1188,13 @@ public class TestJdbcDriver2 extends Tes
     assertTrue(meta.supportsAlterTableWithAddColumn());
   }
 
+  @Test
   public void testResultSetMetaData() throws SQLException {
     Statement stmt = con.createStatement();
 
     ResultSet res = stmt.executeQuery(
         "select c1, c2, c3, c4, c5 as a, c6, c7, c8, c9, c10, c11, c12, " +
-        "c1*2, sentences(null, null, null) as b, c17, c18, c20, c21 from " + dataTypeTableName +
+            "c1*2, sentences(null, null, null) as b, c17, c18, c20, c21 from " + dataTypeTableName +
         " limit 1");
     ResultSetMetaData meta = res.getMetaData();
 
@@ -1370,6 +1439,7 @@ public class TestJdbcDriver2 extends Tes
         "server", "10002", "db"},
   };
 
+  @Test
   public void testDriverProperties() throws SQLException {
     HiveDriver driver = new HiveDriver();
     for (String[] testValues : URL_PROPERTIES) {
@@ -1386,12 +1456,13 @@ public class TestJdbcDriver2 extends Tes
         "user=foo;password=bar?" +
         "hive.server2.transport.mode=http;" +
         "hive.server2.thrift.http.path=hs2", "server", "10002", "db", "http", "hs2"},
-    {"jdbc:hive2://server:10000/testdb;" +
-        "user=foo;password=bar?" +
-        "hive.server2.transport.mode=binary;" +
-        "hive.server2.thrift.http.path=", "server", "10000", "testdb", "binary", ""},
+        {"jdbc:hive2://server:10000/testdb;" +
+            "user=foo;password=bar?" +
+            "hive.server2.transport.mode=binary;" +
+            "hive.server2.thrift.http.path=", "server", "10000", "testdb", "binary", ""},
   };
 
+  @Test
   public void testParseUrlHttpMode() throws SQLException {
     HiveDriver driver = new HiveDriver();
     for (String[] testValues : HTTP_URL_PROPERTIES) {
@@ -1416,6 +1487,7 @@ public class TestJdbcDriver2 extends Tes
    * validate schema generated by "set" command
    * @throws SQLException
    */
+  @Test
   public void testSetCommand() throws SQLException {
     // execute set command
     String sql = "set -v";
@@ -1438,6 +1510,7 @@ public class TestJdbcDriver2 extends Tes
    * Validate error on closed resultset
    * @throws SQLException
    */
+  @Test
   public void testPostClose() throws SQLException {
     Statement stmt = con.createStatement();
     ResultSet res = stmt.executeQuery("select * from " + tableName);
@@ -1455,6 +1528,7 @@ public class TestJdbcDriver2 extends Tes
    * The JDBC spec says when you have duplicate column names,
    * the first one should be returned.
    */
+  @Test
   public void testDuplicateColumnNameOrder() throws SQLException {
     Statement stmt = con.createStatement();
     ResultSet rs = stmt.executeQuery("SELECT 1 AS a, 2 AS a from " + tableName);
@@ -1467,6 +1541,7 @@ public class TestJdbcDriver2 extends Tes
    * Test bad args to getXXX()
    * @throws SQLException
    */
+  @Test
   public void testOutOfBoundCols() throws SQLException {
     Statement stmt = con.createStatement();
 
@@ -1492,6 +1567,7 @@ public class TestJdbcDriver2 extends Tes
    * Verify selecting using builtin UDFs
    * @throws SQLException
    */
+  @Test
   public void testBuiltInUDFCol() throws SQLException {
     Statement stmt = con.createStatement();
     ResultSet res = stmt.executeQuery("select c12, bin(c12) from " + dataTypeTableName
@@ -1509,6 +1585,7 @@ public class TestJdbcDriver2 extends Tes
    * Verify selecting named expression columns
    * @throws SQLException
    */
+  @Test
   public void testExprCol() throws SQLException {
     Statement stmt = con.createStatement();
     ResultSet res = stmt.executeQuery("select c1+1 as col1, length(c4) as len from " + dataTypeTableName
@@ -1527,6 +1604,7 @@ public class TestJdbcDriver2 extends Tes
    * test getProcedureColumns()
    * @throws SQLException
    */
+  @Test
   public void testProcCols() throws SQLException {
     DatabaseMetaData dbmd = con.getMetaData();
     assertNotNull(dbmd);
@@ -1541,6 +1619,7 @@ public class TestJdbcDriver2 extends Tes
    * test testProccedures()
    * @throws SQLException
    */
+  @Test
   public void testProccedures() throws SQLException {
     DatabaseMetaData dbmd = con.getMetaData();
     assertNotNull(dbmd);
@@ -1555,6 +1634,7 @@ public class TestJdbcDriver2 extends Tes
    * test getPrimaryKeys()
    * @throws SQLException
    */
+  @Test
   public void testPrimaryKeys() throws SQLException {
     DatabaseMetaData dbmd = con.getMetaData();
     assertNotNull(dbmd);
@@ -1569,6 +1649,7 @@ public class TestJdbcDriver2 extends Tes
    * test getImportedKeys()
    * @throws SQLException
    */
+  @Test
   public void testImportedKeys() throws SQLException {
     DatabaseMetaData dbmd = con.getMetaData();
     assertNotNull(dbmd);
@@ -1583,12 +1664,10 @@ public class TestJdbcDriver2 extends Tes
    * If the Driver implementation understands the URL, it will return a Connection object;
    * otherwise it returns null
    */
+  @Test
   public void testInvalidURL() throws Exception {
     HiveDriver driver = new HiveDriver();
     Connection conn = driver.connect("jdbc:derby://localhost:10000/default", new Properties());
     assertNull(conn);
   }
-
-
-
 }

Modified: hive/branches/tez/metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql (original)
+++ hive/branches/tez/metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql Fri Oct  4 23:35:50 2013
@@ -296,4 +296,4 @@ ALTER TABLE "APP"."IDXS" ADD CONSTRAINT 
 
 ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED IN ('Y','N'));
 
-INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, COMMENT) VALUES (1, '0.13.0', 'Hive release version 0.13.0');
+INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.13.0', 'Hive release version 0.13.0');

Modified: hive/branches/tez/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql (original)
+++ hive/branches/tez/metastore/scripts/upgrade/oracle/014-HIVE-3764.oracle.sql Fri Oct  4 23:35:50 2013
@@ -1,10 +1,10 @@
 -- HIVE-3764 Support metastore version consistency check
 
-CREATE TABLE IF NOT EXISTS VERSION (
+CREATE TABLE VERSION (
   VER_ID NUMBER NOT NULL,
   SCHEMA_VERSION VARCHAR(127) NOT NULL,
   VERSION_COMMENT VARCHAR(255)
-)
+);
 ALTER TABLE VERSION ADD CONSTRAINT VERSION_PK PRIMARY KEY (VER_ID);
 
-INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '', 'Initial value');
+INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, ' ', 'Initial value');

Modified: hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.12.0.oracle.sql
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.12.0.oracle.sql?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.12.0.oracle.sql (original)
+++ hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.12.0.oracle.sql Fri Oct  4 23:35:50 2013
@@ -483,11 +483,11 @@ CREATE TABLE TAB_COL_STATS (
  LAST_ANALYZED NUMBER NOT NULL
 );
 
-CREATE TABLE IF NOT EXISTS VERSION (
+CREATE TABLE VERSION (
   VER_ID NUMBER NOT NULL,
   SCHEMA_VERSION VARCHAR(127) NOT NULL,
   VERSION_COMMENT VARCHAR(255)
-)
+);
 ALTER TABLE VERSION ADD CONSTRAINT VERSION_PK PRIMARY KEY (VER_ID);
 
 ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_PKEY PRIMARY KEY (CS_ID);

Modified: hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql?rev=1529347&r1=1529346&r2=1529347&view=diff
==============================================================================
--- hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql (original)
+++ hive/branches/tez/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql Fri Oct  4 23:35:50 2013
@@ -483,11 +483,11 @@ CREATE TABLE TAB_COL_STATS (
  LAST_ANALYZED NUMBER NOT NULL
 );
 
-CREATE TABLE IF NOT EXISTS VERSION (
+CREATE TABLE VERSION (
   VER_ID NUMBER NOT NULL,
   SCHEMA_VERSION VARCHAR(127) NOT NULL,
-  COMMENT VARCHAR(255)
-)
+  VERSION_COMMENT VARCHAR(255)
+);
 ALTER TABLE VERSION ADD CONSTRAINT VERSION_PK PRIMARY KEY (VER_ID);
 
 ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_PKEY PRIMARY KEY (CS_ID);