You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2017/11/03 23:26:47 UTC

[06/12] hive git commit: HIVE-17980 Moved HiveMetaStoreClient plus a few remaining classes.

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index 5f2a34e..cd5581f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.ErrorMsg;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 549b38d..2ded4b3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -65,7 +65,6 @@ import org.apache.hadoop.hive.common.StatsSetupConst.StatDB;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.TransactionalValidationListener;
 import org.apache.hadoop.hive.metastore.Warehouse;
@@ -6814,6 +6813,24 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       checkImmutableTable(qb, dest_tab, dest_path, false);
 
+      // check for partition
+      List<FieldSchema> parts = dest_tab.getPartitionKeys();
+      if (parts != null && parts.size() > 0) { // table is partitioned
+        if (partSpec == null || partSpec.size() == 0) { // user did NOT specify partition
+          throw new SemanticException(generateErrorMessage(
+              qb.getParseInfo().getDestForClause(dest),
+              ErrorMsg.NEED_PARTITION_ERROR.getMsg()));
+        }
+        dpCtx = qbm.getDPCtx(dest);
+        if (dpCtx == null) {
+          dest_tab.validatePartColumnNames(partSpec, false);
+          dpCtx = new DynamicPartitionCtx(dest_tab, partSpec,
+              conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME),
+              conf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTSPERNODE));
+          qbm.setDPCtx(dest, dpCtx);
+        }
+      }
+
       // Check for dynamic partitions.
       dpCtx = checkDynPart(qb, qbm, dest_tab, partSpec, dest);
       if (dpCtx != null && dpCtx.getSPPath() != null) {
@@ -7395,7 +7412,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
     try {
       FileSystem fs = dest_path.getFileSystem(conf);
-      if (! MetaStoreUtils.isDirEmpty(fs,dest_path)){
+      if (! org.apache.hadoop.hive.metastore.utils.FileUtils.isDirEmpty(fs,dest_path)){
         LOG.warn("Attempted write into an immutable table : "
             + dest_tab.getTableName() + " : " + dest_path);
         throw new SemanticException(

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
index 9e130dc..f6fb42c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
@@ -28,7 +28,6 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
@@ -36,6 +35,7 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.DDLTask;
 import org.apache.hadoop.hive.ql.exec.Utilities;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
index 550e6f8..84476d8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Objects;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java
index 1fa7b40..4108b73 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/LoadTableDesc.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
index bf8ee75..47f20b5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
@@ -31,8 +31,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.CopyOnFirstWriteProperties;
 import org.apache.hadoop.hive.common.StringInternUtils;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
 import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
@@ -87,7 +87,7 @@ public class PartitionDesc implements Serializable, Cloneable {
     if (Utilities.isInputFileFormatSelfDescribing(this)) {
       // if IF is self describing no need to send column info per partition, since its not used anyway.
       Table tbl = part.getTable();
-      setProperties(MetaStoreUtils.getSchemaWithoutCols(part.getTPartition().getSd(), part.getTPartition().getSd(),
+      setProperties(MetaStoreUtils.getSchemaWithoutCols(part.getTPartition().getSd(),
           part.getParameters(), tbl.getDbName(), tbl.getTableName(), tbl.getPartitionKeys()));
     } else {
       setProperties(part.getMetadataFromPartitionSchema());

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index 3c1e92a..d9f0bd6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@ -34,9 +34,10 @@ import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.LlapOutputFormat;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
@@ -586,7 +587,7 @@ public final class PlanUtils {
     // last one for union column.
     List<FieldSchema> schemas = new ArrayList<FieldSchema>(length + 1);
     for (int i = 0; i < length; i++) {
-      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(
+      schemas.add(HiveMetaStoreUtils.getFieldSchemaFromTypeInfo(
           fieldPrefix + outputColumnNames.get(i), cols.get(i).getTypeInfo()));
     }
 
@@ -603,7 +604,7 @@ public final class PlanUtils {
       unionTypes.add(TypeInfoFactory.getStructTypeInfo(names, types));
     }
     if (outputColumnNames.size() - length > 0) {
-      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(
+      schemas.add(HiveMetaStoreUtils.getFieldSchemaFromTypeInfo(
           fieldPrefix + outputColumnNames.get(length),
           TypeInfoFactory.getUnionTypeInfo(unionTypes)));
     }
@@ -619,7 +620,7 @@ public final class PlanUtils {
       String fieldPrefix) {
     List<FieldSchema> schemas = new ArrayList<FieldSchema>(cols.size());
     for (int i = 0; i < cols.size(); i++) {
-      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(fieldPrefix
+      schemas.add(HiveMetaStoreUtils.getFieldSchemaFromTypeInfo(fieldPrefix
           + outputColumnNames.get(i + start), cols.get(i).getTypeInfo()));
     }
     return schemas;
@@ -632,7 +633,7 @@ public final class PlanUtils {
       List<ExprNodeDesc> cols, String fieldPrefix) {
     List<FieldSchema> schemas = new ArrayList<FieldSchema>(cols.size());
     for (int i = 0; i < cols.size(); i++) {
-      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(fieldPrefix + i,
+      schemas.add(HiveMetaStoreUtils.getFieldSchemaFromTypeInfo(fieldPrefix + i,
           cols.get(i).getTypeInfo()));
     }
     return schemas;
@@ -662,7 +663,7 @@ public final class PlanUtils {
       if (name.equals(String.valueOf(i))) {
         name = fieldPrefix + name;
       }
-      schemas.add(MetaStoreUtils.getFieldSchemaFromTypeInfo(name, cols.get(i)
+      schemas.add(HiveMetaStoreUtils.getFieldSchemaFromTypeInfo(name, cols.get(i)
           .getType()));
     }
     return schemas;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
index 5202571..8569ffb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterators;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -30,7 +31,6 @@ import org.apache.hadoop.hive.common.classification.InterfaceAudience.Private;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStorePreEventListener;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
index 7d4d379..b94e47b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hive.common.StringableMap;
 import org.apache.hadoop.hive.common.ValidCompactorTxnList;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index 8daa5c0..ab3cfc8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.CompactionType;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index 662462c..6d3e156 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -96,7 +96,7 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     Path bucket = AcidUtils.createBucketFile(new Path(new Path(getWarehouseDir(), table.toString().toLowerCase()), AcidUtils.deltaSubdir(txnId, txnId, stmtId)), bucketNum);
     FileOutputStream delta = new FileOutputStream(testName.getMethodName() + "_" + bucket.getParent().getName() + "_" +  bucket.getName());
 //    try {
-//      FileDump.printJsonData(hiveConf, bucket.toString(), delta);
+//      FileDump.printJsonData(conf, bucket.toString(), delta);
 //    }
 //    catch(FileNotFoundException ex) {
       ;//this happens if you change BUCKET_COUNT
@@ -351,7 +351,6 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 1, TimeUnit.SECONDS);
     // Have to reset the conf when we change it so that the change takes affect
     houseKeeperService.setConf(hiveConf);
-    //hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER, true);
     runStatementOnDriver("start transaction");
     runStatementOnDriver("select count(*) from " + Table.ACIDTBL + " where a = 17");
     pause(750);

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/service/pom.xml
----------------------------------------------------------------------
diff --git a/service/pom.xml b/service/pom.xml
index 412bde5..484230f 100644
--- a/service/pom.xml
+++ b/service/pom.xml
@@ -252,7 +252,7 @@
       </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
-      <artifactId>hive-metastore</artifactId>
+      <artifactId>hive-standalone-metastore</artifactId>
       <version>${project.version}</version>
       <type>test-jar</type>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java
index e9a5830..fbfd57a 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java
@@ -38,7 +38,7 @@ public class EmbeddedThriftBinaryCLIService extends ThriftBinaryCLIService {
   @Override
   public synchronized void init(HiveConf hiveConf) {
 	// Null HiveConf is passed in jdbc driver side code since driver side is supposed to be
-	// independent of hiveConf object. Create new HiveConf object here in this case.
+	// independent of conf object. Create new HiveConf object here in this case.
 	if (hiveConf == null) {
 	  hiveConf = new HiveConf();
 	}

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index 2c4fe7f..285ab02 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -63,11 +63,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.coordinator.LlapCoordinator;
 import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.RawStore;
-import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.cache.CachedStore;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl;
 import org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager;
 import org.apache.hadoop.hive.ql.exec.tez.WorkloadManager;
@@ -162,7 +158,7 @@ public class HiveServer2 extends CompositeService {
     }
     addService(thriftCLIService);
     super.init(hiveConf);
-    // Set host name in hiveConf
+    // Set host name in conf
     try {
       hiveConf.set(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, getServerHost());
     } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
index 6660097..3aa475f 100644
--- a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
+++ b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hive.service.Service;
 import org.apache.hive.service.auth.HiveAuthConstants;
-import org.apache.hive.service.auth.HiveAuthConstants.AuthTypes;
 import org.apache.hive.service.cli.OperationHandle;
 import org.apache.hive.service.cli.OperationState;
 import org.apache.hive.service.cli.OperationStatus;

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/standalone-metastore/pom.xml
----------------------------------------------------------------------
diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml
index eee6528..fd834fa 100644
--- a/standalone-metastore/pom.xml
+++ b/standalone-metastore/pom.xml
@@ -190,6 +190,22 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commmons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
     <!-- This is our one and only Hive dependency.-->
     <dependency>
       <groupId>org.apache.hive</groupId>
@@ -472,7 +488,7 @@
             <javax.jdo.option.ConnectionURL>jdbc:derby:memory:${test.tmp.dir}/junit_metastore_db;create=true</javax.jdo.option.ConnectionURL>
             <metastore.schema.verification>false</metastore.schema.verification>
             <test.tmp.dir>${test.tmp.dir}</test.tmp.dir>
-            <test.warehouse.dir>${test.warehouse.scheme}${test.warehouse.dir}</test.warehouse.dir>
+            <metastore.warehouse.dir>${test.warehouse.scheme}${test.warehouse.dir}</metastore.warehouse.dir>
           </systemPropertyVariables>
           <additionalClasspathElements>
             <additionalClasspathElement>${log4j.conf.dir}</additionalClasspathElement>
@@ -480,6 +496,17 @@
         </configuration>
       </plugin>
       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>build-helper-maven-plugin</artifactId>
         <version>3.0.0</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
index a41b5ee..d5dea4d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ColumnType.java
@@ -86,8 +86,6 @@ public class ColumnType {
 
   public static final String COLUMN_NAME_DELIMITER = "column.name.delimiter";
 
-  public static final String SERIALIZATION_FORMAT = "serialization.format";
-
   public static final Set<String> PrimitiveTypes = StringUtils.asSet(
     VOID_TYPE_NAME,
     BOOLEAN_TYPE_NAME,
@@ -239,4 +237,65 @@ public class ColumnType {
     }
     return false;
   }
+
+  // These aren't column types, they are info for how things are stored in thrift.
+  // It didn't seem useful to create another Constants class just for these though.
+  public static final String SERIALIZATION_FORMAT = "serialization.format";
+
+  public static final String SERIALIZATION_LIB = "serialization.lib";
+
+  public static final String SERIALIZATION_DDL = "serialization.ddl";
+
+  public static final char COLUMN_COMMENTS_DELIMITER = '\0';
+
+  private static HashMap<String, String> typeToThriftTypeMap;
+  static {
+    typeToThriftTypeMap = new HashMap<>();
+    typeToThriftTypeMap.put(BOOLEAN_TYPE_NAME, "bool");
+    typeToThriftTypeMap.put(TINYINT_TYPE_NAME, "byte");
+    typeToThriftTypeMap.put(SMALLINT_TYPE_NAME, "i16");
+    typeToThriftTypeMap.put(INT_TYPE_NAME, "i32");
+    typeToThriftTypeMap.put(BIGINT_TYPE_NAME, "i64");
+    typeToThriftTypeMap.put(DOUBLE_TYPE_NAME, "double");
+    typeToThriftTypeMap.put(FLOAT_TYPE_NAME, "float");
+    typeToThriftTypeMap.put(LIST_TYPE_NAME, "list");
+    typeToThriftTypeMap.put(MAP_TYPE_NAME, "map");
+    typeToThriftTypeMap.put(STRING_TYPE_NAME, "string");
+    typeToThriftTypeMap.put(BINARY_TYPE_NAME, "binary");
+    // These 4 types are not supported yet.
+    // We should define a complex type date in thrift that contains a single int
+    // member, and DynamicSerDe
+    // should convert it to date type at runtime.
+    typeToThriftTypeMap.put(DATE_TYPE_NAME, "date");
+    typeToThriftTypeMap.put(DATETIME_TYPE_NAME, "datetime");
+    typeToThriftTypeMap.put(TIMESTAMP_TYPE_NAME, "timestamp");
+    typeToThriftTypeMap.put(DECIMAL_TYPE_NAME, "decimal");
+    typeToThriftTypeMap.put(INTERVAL_YEAR_MONTH_TYPE_NAME, INTERVAL_YEAR_MONTH_TYPE_NAME);
+    typeToThriftTypeMap.put(INTERVAL_DAY_TIME_TYPE_NAME, INTERVAL_DAY_TIME_TYPE_NAME);
+  }
+
+  /**
+   * Convert type to ThriftType. We do that by tokenizing the type and convert
+   * each token.
+   */
+  public static String typeToThriftType(String type) {
+    StringBuilder thriftType = new StringBuilder();
+    int last = 0;
+    boolean lastAlphaDigit = Character.isLetterOrDigit(type.charAt(last));
+    for (int i = 1; i <= type.length(); i++) {
+      if (i == type.length()
+          || Character.isLetterOrDigit(type.charAt(i)) != lastAlphaDigit) {
+        String token = type.substring(last, i);
+        last = i;
+        String thriftToken = typeToThriftTypeMap.get(token);
+        thriftType.append(thriftToken == null ? token : thriftToken);
+        lastAlphaDigit = !lastAlphaDigit;
+      }
+    }
+    return thriftType.toString();
+  }
+
+  public static String getListType(String t) {
+    return "array<" + t + ">";
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/20c86d1f/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index ccadac1..03c082c 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -66,19 +66,19 @@ import java.util.Map.Entry;
  */
 public class HiveAlterHandler implements AlterHandler {
 
-  protected Configuration hiveConf;
+  protected Configuration conf;
   private static final Logger LOG = LoggerFactory.getLogger(HiveAlterHandler.class
       .getName());
 
   @Override
   public Configuration getConf() {
-    return hiveConf;
+    return conf;
   }
 
   @Override
   @SuppressWarnings("nls")
   public void setConf(Configuration conf) {
-    hiveConf = conf;
+    this.conf = conf;
   }
 
   @Override
@@ -106,7 +106,7 @@ public class HiveAlterHandler implements AlterHandler {
     String newTblName = newt.getTableName().toLowerCase();
     String newDbName = newt.getDbName().toLowerCase();
 
-    if (!MetaStoreUtils.validateName(newTblName, hiveConf)) {
+    if (!MetaStoreUtils.validateName(newTblName, conf)) {
       throw new InvalidOperationException(newTblName + " is not a valid object name");
     }
     String validate = MetaStoreUtils.validateTblColumns(newt.getSd().getCols());
@@ -155,7 +155,7 @@ public class HiveAlterHandler implements AlterHandler {
       // Views derive the column type from the base table definition.  So the view definition
       // can be altered to change the column types.  The column type compatibility checks should
       // be done only for non-views.
-      if (MetastoreConf.getBoolVar(hiveConf,
+      if (MetastoreConf.getBoolVar(conf,
             MetastoreConf.ConfVars.DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES) &&
           !oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())) {
         // Throws InvalidOperationException if the new column types are not
@@ -276,7 +276,7 @@ public class HiveAlterHandler implements AlterHandler {
         }
       } else {
         // operations other than table rename
-        if (MetaStoreUtils.requireCalStats(hiveConf, null, null, newt, environmentContext) &&
+        if (MetaStoreUtils.requireCalStats(null, null, newt, environmentContext) &&
             !isPartitionedTable) {
           Database db = msdb.getDatabase(newDbName);
           // Update table stats. For partitioned table, we update stats in alterPartition()
@@ -422,7 +422,7 @@ public class HiveAlterHandler implements AlterHandler {
       try {
         msdb.openTransaction();
         oldPart = msdb.getPartition(dbname, name, new_part.getValues());
-        if (MetaStoreUtils.requireCalStats(hiveConf, oldPart, new_part, tbl, environmentContext)) {
+        if (MetaStoreUtils.requireCalStats(oldPart, new_part, tbl, environmentContext)) {
           // if stats are same, no need to update
           if (MetaStoreUtils.isFastStatsSame(oldPart, new_part)) {
             MetaStoreUtils.updateBasicState(environmentContext, new_part.getParameters());
@@ -555,7 +555,7 @@ public class HiveAlterHandler implements AlterHandler {
         new_part.getSd().setLocation(oldPart.getSd().getLocation());
       }
 
-      if (MetaStoreUtils.requireCalStats(hiveConf, oldPart, new_part, tbl, environmentContext)) {
+      if (MetaStoreUtils.requireCalStats(oldPart, new_part, tbl, environmentContext)) {
         MetaStoreUtils.updatePartitionStatsFast(new_part, wh, false, true, environmentContext);
       }
 
@@ -647,7 +647,7 @@ public class HiveAlterHandler implements AlterHandler {
         oldParts.add(oldTmpPart);
         partValsList.add(tmpPart.getValues());
 
-        if (MetaStoreUtils.requireCalStats(hiveConf, oldTmpPart, tmpPart, tbl, environmentContext)) {
+        if (MetaStoreUtils.requireCalStats(oldTmpPart, tmpPart, tbl, environmentContext)) {
           // Check if stats are same, no need to update
           if (MetaStoreUtils.isFastStatsSame(oldTmpPart, tmpPart)) {
             MetaStoreUtils.updateBasicState(environmentContext, tmpPart.getParameters());