You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2018/07/19 21:44:57 UTC

[19/54] [abbrv] hive git commit: HIVE-19532 : fix tests for master-txnstats branch - more tests (Sergey Shelukhin)

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index b7fe6ba..923fe2c 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -22793,6 +22793,212 @@ class AlterPartitionsResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class RenamePartitionRequest:
+  """
+  Attributes:
+   - catName
+   - dbName
+   - tableName
+   - partVals
+   - newPart
+   - txnId
+   - validWriteIdList
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'catName', None, None, ), # 1
+    (2, TType.STRING, 'dbName', None, None, ), # 2
+    (3, TType.STRING, 'tableName', None, None, ), # 3
+    (4, TType.LIST, 'partVals', (TType.STRING,None), None, ), # 4
+    (5, TType.STRUCT, 'newPart', (Partition, Partition.thrift_spec), None, ), # 5
+    (6, TType.I64, 'txnId', None, -1, ), # 6
+    (7, TType.STRING, 'validWriteIdList', None, None, ), # 7
+  )
+
+  def __init__(self, catName=None, dbName=None, tableName=None, partVals=None, newPart=None, txnId=thrift_spec[6][4], validWriteIdList=None,):
+    self.catName = catName
+    self.dbName = dbName
+    self.tableName = tableName
+    self.partVals = partVals
+    self.newPart = newPart
+    self.txnId = txnId
+    self.validWriteIdList = validWriteIdList
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.tableName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.partVals = []
+          (_etype847, _size844) = iprot.readListBegin()
+          for _i848 in xrange(_size844):
+            _elem849 = iprot.readString()
+            self.partVals.append(_elem849)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRUCT:
+          self.newPart = Partition()
+          self.newPart.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.I64:
+          self.txnId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRING:
+          self.validWriteIdList = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('RenamePartitionRequest')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 1)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 2)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.tableName is not None:
+      oprot.writeFieldBegin('tableName', TType.STRING, 3)
+      oprot.writeString(self.tableName)
+      oprot.writeFieldEnd()
+    if self.partVals is not None:
+      oprot.writeFieldBegin('partVals', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRING, len(self.partVals))
+      for iter850 in self.partVals:
+        oprot.writeString(iter850)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.newPart is not None:
+      oprot.writeFieldBegin('newPart', TType.STRUCT, 5)
+      self.newPart.write(oprot)
+      oprot.writeFieldEnd()
+    if self.txnId is not None:
+      oprot.writeFieldBegin('txnId', TType.I64, 6)
+      oprot.writeI64(self.txnId)
+      oprot.writeFieldEnd()
+    if self.validWriteIdList is not None:
+      oprot.writeFieldBegin('validWriteIdList', TType.STRING, 7)
+      oprot.writeString(self.validWriteIdList)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.dbName is None:
+      raise TProtocol.TProtocolException(message='Required field dbName is unset!')
+    if self.tableName is None:
+      raise TProtocol.TProtocolException(message='Required field tableName is unset!')
+    if self.partVals is None:
+      raise TProtocol.TProtocolException(message='Required field partVals is unset!')
+    if self.newPart is None:
+      raise TProtocol.TProtocolException(message='Required field newPart is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.catName)
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.tableName)
+    value = (value * 31) ^ hash(self.partVals)
+    value = (value * 31) ^ hash(self.newPart)
+    value = (value * 31) ^ hash(self.txnId)
+    value = (value * 31) ^ hash(self.validWriteIdList)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class RenamePartitionResponse:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('RenamePartitionResponse')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class AlterTableRequest:
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 345cf31..2e2392d 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -5155,6 +5155,53 @@ class AlterPartitionsResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class RenamePartitionRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  CATNAME = 1
+  DBNAME = 2
+  TABLENAME = 3
+  PARTVALS = 4
+  NEWPART = 5
+  TXNID = 6
+  VALIDWRITEIDLIST = 7
+
+  FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true},
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
+    PARTVALS => {:type => ::Thrift::Types::LIST, :name => 'partVals', :element => {:type => ::Thrift::Types::STRING}},
+    NEWPART => {:type => ::Thrift::Types::STRUCT, :name => 'newPart', :class => ::Partition},
+    TXNID => {:type => ::Thrift::Types::I64, :name => 'txnId', :default => -1, :optional => true},
+    VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbName is unset!') unless @dbName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tableName is unset!') unless @tableName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field partVals is unset!') unless @partVals
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field newPart is unset!') unless @newPart
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class RenamePartitionResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+
+  FIELDS = {
+
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class AlterTableRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   CATNAME = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index e7cfc62..92424a4 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -1514,6 +1514,23 @@ module ThriftHiveMetastore
       return
     end
 
+    def rename_partition_req(req)
+      send_rename_partition_req(req)
+      return recv_rename_partition_req()
+    end
+
+    def send_rename_partition_req(req)
+      send_message('rename_partition_req', Rename_partition_req_args, :req => req)
+    end
+
+    def recv_rename_partition_req()
+      result = receive_message(Rename_partition_req_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'rename_partition_req failed: unknown result')
+    end
+
     def partition_name_has_valid_characters(part_vals, throw_exception)
       send_partition_name_has_valid_characters(part_vals, throw_exception)
       return recv_partition_name_has_valid_characters()
@@ -4755,6 +4772,19 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'rename_partition', seqid)
     end
 
+    def process_rename_partition_req(seqid, iprot, oprot)
+      args = read_args(iprot, Rename_partition_req_args)
+      result = Rename_partition_req_result.new()
+      begin
+        result.success = @handler.rename_partition_req(args.req)
+      rescue ::InvalidOperationException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'rename_partition_req', seqid)
+    end
+
     def process_partition_name_has_valid_characters(seqid, iprot, oprot)
       args = read_args(iprot, Partition_name_has_valid_characters_args)
       result = Partition_name_has_valid_characters_result.new()
@@ -9677,6 +9707,42 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Rename_partition_req_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::RenamePartitionRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Rename_partition_req_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::RenamePartitionResponse},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::InvalidOperationException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Partition_name_has_valid_characters_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     PART_VALS = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index e8226f8..719f001 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -547,6 +547,7 @@ public class HiveAlterHandler implements AlterHandler {
       // 2) partition column stats if there are any because of part_name field in HMS table PART_COL_STATS
       // 3) rename the partition directory if it is not an external table
       if (!tbl.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
+        // TODO: refactor this into a separate method after master merge, this one is too big.
         try {
           db = msdb.getDatabase(catName, dbname);
 
@@ -620,8 +621,6 @@ public class HiveAlterHandler implements AlterHandler {
       if (cs != null) {
         cs.getStatsDesc().setPartName(newPartName);
         try {
-          // Verifying ACID state again is not strictly needed here (alterPartition above does it),
-          // but we are going to use the uniform approach for simplicity.
           msdb.updatePartitionColumnStatistics(cs, new_part.getValues(),
               txnId, validWriteIds, new_part.getWriteId());
         } catch (InvalidInputException iie) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index ab25593..285f7fb 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -4809,6 +4809,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           request.getPartitionOrder(), request.getMaxParts());
     }
 
+    @Deprecated
     @Override
     public void alter_partition(final String db_name, final String tbl_name,
         final Partition new_part)
@@ -4816,6 +4817,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       rename_partition(db_name, tbl_name, null, new_part);
     }
 
+    @Deprecated
     @Override
     public void alter_partition_with_environment_context(final String dbName,
         final String tableName, final Partition newPartition,
@@ -4824,9 +4826,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       String[] parsedDbName = parseDbName(dbName, conf);
       // TODO: this method name is confusing, it actually does full alter (sortof)
       rename_partition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, null, newPartition,
-          envContext);
+          envContext, -1, null);
     }
 
+    @Deprecated
     @Override
     public void rename_partition(final String db_name, final String tbl_name,
         final List<String> part_vals, final Partition new_part)
@@ -4834,13 +4837,19 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       // Call rename_partition without an environment context.
       String[] parsedDbName = parseDbName(db_name, conf);
       rename_partition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, part_vals, new_part,
-          null);
+          null, -1, null);
     }
 
-    private void rename_partition(final String catName, final String db_name, final String tbl_name,
-        final List<String> part_vals, final Partition new_part,
-        final EnvironmentContext envContext)
-        throws TException {
+    public RenamePartitionResponse rename_partition_req(
+        RenamePartitionRequest req) throws InvalidOperationException ,MetaException ,TException {
+      rename_partition(req.getCatName(), req.getDbName(), req.getTableName(), req.getPartVals(),
+          req.getNewPart(), null, req.getTxnId(), req.getValidWriteIdList());
+      return new RenamePartitionResponse();
+    };
+
+    private void rename_partition(String catName, String db_name, String tbl_name,
+        List<String> part_vals, Partition new_part, EnvironmentContext envContext, long txnId,
+        String validWriteIds) throws TException {
       startTableFunction("alter_partition", catName, db_name, tbl_name);
 
       if (LOG.isInfoEnabled()) {
@@ -4874,7 +4883,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
 
         oldPart = alterHandler.alterPartition(getMS(), wh, catName, db_name, tbl_name,
-            part_vals, new_part, envContext, this, -1, null);
+            part_vals, new_part, envContext, this, txnId, validWriteIds);
 
         // Only fetch the table if we actually have a listener
         Table table = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 1fa719d..70edb96 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -434,17 +434,21 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     client.alter_table_req(req);
   }
 
+  @Deprecated
   @Override
   public void renamePartition(final String dbname, final String tableName, final List<String> part_vals,
                               final Partition newPart) throws TException {
-    renamePartition(getDefaultCatalog(conf), dbname, tableName, part_vals, newPart);
+    renamePartition(getDefaultCatalog(conf), dbname, tableName, part_vals, newPart, -1, null);
   }
 
   @Override
   public void renamePartition(String catName, String dbname, String tableName, List<String> part_vals,
-                              Partition newPart) throws TException {
-    client.rename_partition(prependCatalogToDbName(catName, dbname, conf), tableName, part_vals, newPart);
-
+                              Partition newPart, long txnId, String validWriteIds) throws TException {
+    RenamePartitionRequest req = new RenamePartitionRequest(dbname, tableName, part_vals, newPart);
+    req.setCatName(catName);
+    req.setTxnId(txnId);
+    req.setValidWriteIdList(validWriteIds);
+    client.rename_partition_req(req);
   }
 
   private void open() throws MetaException {

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 41fd91e..70be8d8 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -2217,6 +2217,7 @@ public interface IMetaStoreClient {
    * @throws TException
    *          if error in communicating with metastore server
    */
+  @Deprecated
   void renamePartition(final String dbname, final String tableName, final List<String> part_vals,
                        final Partition newPart)
       throws InvalidOperationException, MetaException, TException;
@@ -2240,7 +2241,7 @@ public interface IMetaStoreClient {
    *          if error in communicating with metastore server
    */
   void renamePartition(String catName, String dbname, String tableName, List<String> part_vals,
-                       Partition newPart)
+                       Partition newPart, long txnId, String validWriteIds)
       throws InvalidOperationException, MetaException, TException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index 92205ae..6fcfbce 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@ -224,6 +224,19 @@ public final class TxnDbUtil {
       }
 
       try {
+        stmt.execute("CREATE TABLE \"APP\".\"DBS\" (\"DB_ID\" BIGINT NOT NULL, \"DESC\" " +
+            "VARCHAR(4000), \"DB_LOCATION_URI\" VARCHAR(4000) NOT NULL, \"NAME\" VARCHAR(128), " +
+            "\"OWNER_NAME\" VARCHAR(128), \"OWNER_TYPE\" VARCHAR(10), " +
+            "\"CTLG_NAME\" VARCHAR(256) NOT NULL, PRIMARY KEY (DB_ID))");
+      } catch (SQLException e) {
+        if (e.getMessage() != null && e.getMessage().contains("already exists")) {
+          LOG.info("TBLS table already exist, ignoring");
+        } else {
+          throw e;
+        }
+      }
+
+      try {
         stmt.execute("CREATE TABLE \"APP\".\"PARTITIONS\" (" +
             " \"PART_ID\" BIGINT NOT NULL, \"CREATE_TIME\" INTEGER NOT NULL, " +
             " \"LAST_ACCESS_TIME\" INTEGER NOT NULL, \"PART_NAME\" VARCHAR(767), " +

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
index 633fddf..4cfa6a1 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -1623,6 +1623,19 @@ struct AlterPartitionsRequest {
 struct AlterPartitionsResponse {
 }
 
+struct RenamePartitionRequest {
+  1: optional string catName,
+  2: required string dbName,
+  3: required string tableName,
+  4: required list<string> partVals,
+  5: required Partition newPart,
+  6: optional i64 txnId=-1,
+  7: optional string validWriteIdList
+}
+
+struct RenamePartitionResponse {
+}
+
 struct AlterTableRequest {
   1: optional string catName,
   2: required string dbName,
@@ -1990,6 +2003,9 @@ service ThriftHiveMetastore extends fb303.FacebookService
   // partition keys in new_part should be the same as those in old partition.
   void rename_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals, 4:Partition new_part)
                        throws (1:InvalidOperationException o1, 2:MetaException o2)
+  
+  RenamePartitionResponse rename_partition_req(1:RenamePartitionRequest req)
+                       throws (1:InvalidOperationException o1, 2:MetaException o2)
 
   // returns whether or not the partition name is valid based on the value of the config
   // hive.metastore.partition.name.whitelist.pattern

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index 3709db5..bc04e06 100644
--- a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -3311,8 +3311,8 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
 
   @Override
   public void renamePartition(String catName, String dbname, String tableName,
-                              List<String> part_vals, Partition newPart) throws
-      InvalidOperationException, MetaException, TException {
+      List<String> part_vals, Partition newPart, long txnId, String validWriteIds)
+          throws InvalidOperationException, MetaException, TException {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0a2fffa/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
index 2ec20c2..0aab253 100644
--- a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
+++ b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
@@ -1080,7 +1080,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
 
     Partition partToRename = oldParts.get(3);
     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
-    client.renamePartition("nosuch", DB_NAME, TABLE_NAME, oldValues.get(3), partToRename);
+    client.renamePartition("nosuch", DB_NAME, TABLE_NAME, oldValues.get(3), partToRename, -1, null);
   }
 
   @Test(expected = InvalidOperationException.class)