You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ng...@apache.org on 2017/05/12 17:27:47 UTC

[1/3] hive git commit: HIVE-16555 Add a new thrift API call for get_metastore_uuid (Vihang Karajgaonkar, reviewed by Naveen Gangam, Sahil Takiar and Alan Gates)

Repository: hive
Updated Branches:
  refs/heads/master 603c5341e -> c9cfb6019


http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 4fb7183..2bb4755 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1195,6 +1195,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @return \metastore\CacheFileMetadataResult
    */
   public function cache_file_metadata(\metastore\CacheFileMetadataRequest $req);
+  /**
+   * @return string
+   * @throws \metastore\MetaException
+   */
+  public function get_metastore_db_uuid();
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@ -9941,6 +9946,59 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("cache_file_metadata failed: unknown result");
   }
 
+  public function get_metastore_db_uuid()
+  {
+    $this->send_get_metastore_db_uuid();
+    return $this->recv_get_metastore_db_uuid();
+  }
+
+  public function send_get_metastore_db_uuid()
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_metastore_db_uuid_args();
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_metastore_db_uuid', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_metastore_db_uuid', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_metastore_db_uuid()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_metastore_db_uuid_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_metastore_db_uuid_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    throw new \Exception("get_metastore_db_uuid failed: unknown result");
+  }
+
 }
 
 // HELPER FUNCTIONS AND STRUCTURES
@@ -45318,4 +45376,154 @@ class ThriftHiveMetastore_cache_file_metadata_result {
 
 }
 
+class ThriftHiveMetastore_get_metastore_db_uuid_args {
+  static $_TSPEC;
+
+
+  public function __construct() {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        );
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_metastore_db_uuid_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_metastore_db_uuid_args');
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_metastore_db_uuid_result {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRING,
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_metastore_db_uuid_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->success);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_metastore_db_uuid_result');
+    if ($this->success !== null) {
+      $xfer += $output->writeFieldBegin('success', TType::STRING, 0);
+      $xfer += $output->writeString($this->success);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index f2a9799..9faf830 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -178,6 +178,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req)')
   print('  ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req)')
   print('  CacheFileMetadataResult cache_file_metadata(CacheFileMetadataRequest req)')
+  print('  string get_metastore_db_uuid()')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -1171,6 +1172,12 @@ elif cmd == 'cache_file_metadata':
     sys.exit(1)
   pp.pprint(client.cache_file_metadata(eval(args[0]),))
 
+elif cmd == 'get_metastore_db_uuid':
+  if len(args) != 0:
+    print('get_metastore_db_uuid requires 0 args')
+    sys.exit(1)
+  pp.pprint(client.get_metastore_db_uuid())
+
 elif cmd == 'getName':
   if len(args) != 0:
     print('getName requires 0 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 8ee84af..b6306f0 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1239,6 +1239,9 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_metastore_db_uuid(self):
+    pass
+
 
 class Client(fb303.FacebookService.Client, Iface):
   """
@@ -6804,6 +6807,34 @@ class Client(fb303.FacebookService.Client, Iface):
       return result.success
     raise TApplicationException(TApplicationException.MISSING_RESULT, "cache_file_metadata failed: unknown result")
 
+  def get_metastore_db_uuid(self):
+    self.send_get_metastore_db_uuid()
+    return self.recv_get_metastore_db_uuid()
+
+  def send_get_metastore_db_uuid(self):
+    self._oprot.writeMessageBegin('get_metastore_db_uuid', TMessageType.CALL, self._seqid)
+    args = get_metastore_db_uuid_args()
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_metastore_db_uuid(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_metastore_db_uuid_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_metastore_db_uuid failed: unknown result")
+
 
 class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
   def __init__(self, handler):
@@ -6962,6 +6993,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["put_file_metadata"] = Processor.process_put_file_metadata
     self._processMap["clear_file_metadata"] = Processor.process_clear_file_metadata
     self._processMap["cache_file_metadata"] = Processor.process_cache_file_metadata
+    self._processMap["get_metastore_db_uuid"] = Processor.process_get_metastore_db_uuid
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -10732,6 +10764,28 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_metastore_db_uuid(self, seqid, iprot, oprot):
+    args = get_metastore_db_uuid_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_metastore_db_uuid_result()
+    try:
+      result.success = self._handler.get_metastore_db_uuid()
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_metastore_db_uuid", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
 
 # HELPER FUNCTIONS AND STRUCTURES
 
@@ -36796,3 +36850,127 @@ class cache_file_metadata_result:
 
   def __ne__(self, other):
     return not (self == other)
+
+class get_metastore_db_uuid_args:
+
+  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('get_metastore_db_uuid_args')
+    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 get_metastore_db_uuid_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.STRING, 'success', None, None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, o1=None,):
+    self.success = success
+    self.o1 = o1
+
+  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 == 0:
+        if ftype == TType.STRING:
+          self.success = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        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('get_metastore_db_uuid_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRING, 0)
+      oprot.writeString(self.success)
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    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)

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 04e63f3..b9d0fa2 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2562,6 +2562,22 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'cache_file_metadata failed: unknown result')
     end
 
+    def get_metastore_db_uuid()
+      send_get_metastore_db_uuid()
+      return recv_get_metastore_db_uuid()
+    end
+
+    def send_get_metastore_db_uuid()
+      send_message('get_metastore_db_uuid', Get_metastore_db_uuid_args)
+    end
+
+    def recv_get_metastore_db_uuid()
+      result = receive_message(Get_metastore_db_uuid_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_metastore_db_uuid failed: unknown result')
+    end
+
   end
 
   class Processor < ::FacebookService::Processor 
@@ -4457,6 +4473,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'cache_file_metadata', seqid)
     end
 
+    def process_get_metastore_db_uuid(seqid, iprot, oprot)
+      args = read_args(iprot, Get_metastore_db_uuid_args)
+      result = Get_metastore_db_uuid_result.new()
+      begin
+        result.success = @handler.get_metastore_db_uuid()
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'get_metastore_db_uuid', seqid)
+    end
+
   end
 
   # HELPER FUNCTIONS AND STRUCTURES
@@ -10231,5 +10258,38 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_metastore_db_uuid_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+
+    FIELDS = {
+
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_metastore_db_uuid_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRING, :name => 'success'},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
 end
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index cbcfc72..d9d50ab 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -6970,6 +6970,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
       return new ForeignKeysResponse(ret);
     }
+
+    @Override
+    public String get_metastore_db_uuid() throws MetaException, TException {
+      try {
+        return getMS().getMetastoreDbUuid();
+      } catch (MetaException e) {
+        LOG.error("Exception thrown while querying metastore db uuid", e);
+        throw e;
+      }
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 53f8118..3e6add2 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -2536,4 +2536,9 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     CacheFileMetadataResult result = client.cache_file_metadata(req);
     return result.isIsSupported();
   }
+
+  @Override
+  public String getMetastoreDbUuid() throws TException {
+    return client.get_metastore_db_uuid();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 023a289..9c24c23 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -1664,4 +1664,12 @@ public interface IMetaStoreClient {
 
   void addForeignKey(List<SQLForeignKey> foreignKeyCols) throws
   MetaException, NoSuchObjectException, TException;
+
+  /**
+   * Gets the unique id of the backing database instance used for storing metadata
+   * @return unique id of the backing database instance
+   * @throws MetaException if HMS is not able to fetch the UUID or if there are multiple UUIDs found in the database
+   * @throws TException in case of Thrift errors
+   */
+  String getMetastoreDbUuid() throws MetaException, TException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index a83e12e..ed19f42 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -25,6 +25,8 @@ import java.lang.reflect.Field;
 import java.net.InetAddress;
 import java.net.URI;
 import java.nio.ByteBuffer;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -38,6 +40,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Lock;
@@ -145,6 +148,7 @@ import org.apache.hadoop.hive.metastore.model.MTableColumnStatistics;
 import org.apache.hadoop.hive.metastore.model.MTablePrivilege;
 import org.apache.hadoop.hive.metastore.model.MType;
 import org.apache.hadoop.hive.metastore.model.MVersionTable;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.FilterBuilder;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
@@ -170,6 +174,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
+
 /**
  * This class is the interface between the application logic and the database
  * store that contains the objects. Refrain putting any logic in mode.M* objects
@@ -3505,6 +3510,77 @@ public class ObjectStore implements RawStore, Configurable {
    addForeignKeys(fks, true);
   }
 
+  @Override
+  public String getMetastoreDbUuid() throws MetaException {
+    String ret = getGuidFromDB();
+    if(ret != null) {
+      return ret;
+    }
+    return createDbGuidAndPersist();
+  }
+
+  private String createDbGuidAndPersist() throws MetaException {
+    boolean success = false;
+    Query query = null;
+    try {
+      openTransaction();
+      MMetastoreDBProperties prop = new MMetastoreDBProperties();
+      prop.setPropertykey("guid");
+      final String guid = UUID.randomUUID().toString();
+      LOG.debug("Attempting to add a guid " + guid + " for the metastore db");
+      prop.setPropertyValue(guid);
+      prop.setDescription("Metastore DB GUID generated on "
+          + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")));
+      pm.makePersistent(prop);
+      success = commitTransaction();
+      if (success) {
+        LOG.info("Metastore db guid " + guid + " created successfully");
+        return guid;
+      }
+    } catch (Exception e) {
+      LOG.warn(e.getMessage(), e);
+    } finally {
+      rollbackAndCleanup(success, query);
+    }
+    // it possible that some other HMS instance could have created the guid
+    // at the same time due which this instance could not create a guid above
+    // in such case return the guid already generated
+    final String guid = getGuidFromDB();
+    if (guid == null) {
+      throw new MetaException("Unable to create or fetch the metastore database uuid");
+    }
+    return guid;
+  }
+
+  private String getGuidFromDB() throws MetaException {
+    boolean success = false;
+    Query query = null;
+    try {
+      openTransaction();
+      query = pm.newQuery(MMetastoreDBProperties.class, "this.propertyKey == key");
+      query.declareParameters("java.lang.String key");
+      Collection<MMetastoreDBProperties> names = (Collection<MMetastoreDBProperties>) query.execute("guid");
+      List<String> uuids = new ArrayList<String>();
+      for (Iterator<MMetastoreDBProperties> i = names.iterator(); i.hasNext();) {
+        String uuid = i.next().getPropertyValue();
+        LOG.debug("Found guid " + uuid);
+        uuids.add(uuid);
+      }
+      success = commitTransaction();
+      if(uuids.size() > 1) {
+        throw new MetaException("Multiple uuids found");
+      }
+      if(!uuids.isEmpty()) {
+        LOG.debug("Returning guid of metastore db : " + uuids.get(0));
+        return uuids.get(0);
+      }
+    } finally {
+      rollbackAndCleanup(success, query);
+    }
+    LOG.warn("Guid for metastore db not found");
+    return null;
+  }
+
   private void addForeignKeys(
     List<SQLForeignKey> fks, boolean retrieveCD) throws InvalidObjectException,
     MetaException {

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
index c22a1db..c1af690 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -708,4 +708,11 @@ public interface RawStore extends Configurable {
   void addPrimaryKeys(List<SQLPrimaryKey> pks) throws InvalidObjectException, MetaException;
 
   void addForeignKeys(List<SQLForeignKey> fks) throws InvalidObjectException, MetaException;
+
+  /**
+   * Gets the unique id of the backing datastore for the metadata
+   * @return
+   * @throws MetaException
+   */
+  String getMetastoreDbUuid() throws MetaException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 39b1676..a04fd5b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -1576,4 +1576,9 @@ public class CachedStore implements RawStore, Configurable {
   public void setRawStore(RawStore rawStore) {
     this.rawStore = rawStore;
   }
+
+  @Override
+  public String getMetastoreDbUuid() throws MetaException {
+    return rawStore.getMetastoreDbUuid();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index f6420f5..0c7d8bb 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -2859,4 +2859,9 @@ public class HBaseStore implements RawStore {
     // TODO: see if it makes sense to implement this here
     return null;
   }
+
+  @Override
+  public String getMetastoreDbUuid() throws MetaException {
+    throw new MetaException("Get metastore DB uuid is not implemented");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/model/org/apache/hadoop/hive/metastore/model/MMetastoreDBProperties.java
----------------------------------------------------------------------
diff --git a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MMetastoreDBProperties.java b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MMetastoreDBProperties.java
new file mode 100644
index 0000000..c0a2485
--- /dev/null
+++ b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MMetastoreDBProperties.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.model;
+
+public class MMetastoreDBProperties {
+  private String propertyKey;
+  private String propertyValue;
+  private String description;
+
+  public MMetastoreDBProperties() {}
+
+  public MMetastoreDBProperties(String propertykey, String propertyValue, String description) {
+    this.propertyKey = propertykey;
+    this.propertyValue = propertyValue;
+    this.description = description;
+  }
+
+  public String getPropertykey() {
+    return propertyKey;
+  }
+
+  public void setPropertykey(String propertykey) {
+    this.propertyKey = propertykey;
+  }
+
+  public String getPropertyValue() {
+    return propertyValue;
+  }
+
+  public void setPropertyValue(String propertyValue) {
+    this.propertyValue = propertyValue;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/model/package.jdo
----------------------------------------------------------------------
diff --git a/metastore/src/model/package.jdo b/metastore/src/model/package.jdo
index 969e199..9c7c691 100644
--- a/metastore/src/model/package.jdo
+++ b/metastore/src/model/package.jdo
@@ -971,6 +971,24 @@
       </field>
     </class>
 
+    <class name="MMetastoreDBProperties" table="METASTORE_DB_PROPERTIES" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="PROPERTY_ID"/>
+      </datastore-identity>
+      <field name ="propertyKey">
+        <column name="PROPERTY_KEY" length="255" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name ="propertyValue">
+        <column name="PROPERTY_VALUE" length="1000" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name ="description">
+        <column name="DESCRIPTION" length="1000" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <index name="PropertyKeyIndex" unique="true">
+        <column name="PROPERTY_KEY"/>
+      </index>
+    </class>
+
     <class name="MResourceUri" embedded-only="true" table="RESOURCE_URI" detachable="true">
       <field name="resourceType">
         <column name="RESOURCE_TYPE" jdbc-type="INTEGER"/>

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 3e3fd20..f613c30 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -877,4 +877,9 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
     // TODO Auto-generated method stub
     return null;
   }
+
+  @Override
+  public String getMetastoreDbUuid() throws MetaException {
+    throw new MetaException("Get metastore uuid is not implemented");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 91d8c2a..1720e37 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -893,6 +893,9 @@ public class DummyRawStoreForJdoConnection implements RawStore {
     // TODO Auto-generated method stub
     return null;
   }
-}
-
 
+  @Override
+  public String getMetastoreDbUuid() throws MetaException {
+    throw new MetaException("Get metastore uuid is not implemented");
+  }
+}


[3/3] hive git commit: HIVE-16555 Add a new thrift API call for get_metastore_uuid (Vihang Karajgaonkar, reviewed by Naveen Gangam, Sahil Takiar and Alan Gates)

Posted by ng...@apache.org.
HIVE-16555 Add a new thrift API call for get_metastore_uuid (Vihang Karajgaonkar, reviewed by Naveen Gangam, Sahil Takiar and Alan Gates)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c9cfb601
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c9cfb601
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c9cfb601

Branch: refs/heads/master
Commit: c9cfb60199df5163c22015a536ead22971dd0d19
Parents: 603c534
Author: Naveen Gangam <ng...@apache.org>
Authored: Fri May 12 13:27:30 2017 -0400
Committer: Naveen Gangam <ng...@apache.org>
Committed: Fri May 12 13:27:30 2017 -0400

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |    7 +-
 .../metastore/TestEmbeddedHiveMetaStore.java    |   18 +-
 .../hive/metastore/TestHiveMetaStore.java       |   49 +
 .../hive/metastore/TestRemoteHiveMetaStore.java |   11 +-
 .../hive/metastore/TestSetUGIOnOnlyClient.java  |    9 +-
 .../hive/metastore/TestSetUGIOnOnlyServer.java  |    9 +-
 metastore/if/hive_metastore.thrift              |    2 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  |  390 ++++
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  122 ++
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../hive/metastore/api/ThriftHiveMetastore.java | 1767 +++++++++++++-----
 .../gen-php/metastore/ThriftHiveMetastore.php   |  208 +++
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  178 ++
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   60 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   10 +
 .../hive/metastore/HiveMetaStoreClient.java     |    5 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    8 +
 .../hadoop/hive/metastore/ObjectStore.java      |   76 +
 .../apache/hadoop/hive/metastore/RawStore.java  |    7 +
 .../hive/metastore/cache/CachedStore.java       |    5 +
 .../hadoop/hive/metastore/hbase/HBaseStore.java |    5 +
 .../metastore/model/MMetastoreDBProperties.java |   56 +
 metastore/src/model/package.jdo                 |   18 +
 .../DummyRawStoreControlledCommit.java          |    5 +
 .../DummyRawStoreForJdoConnection.java          |    7 +-
 26 files changed, 2565 insertions(+), 479 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 88b9faf..91a3a38 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -918,4 +918,9 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
       String tableName) throws MetaException, NoSuchObjectException {
     return objectStore.getAggrColStatsForTablePartitions(dbName, tableName);
   }
-}
\ No newline at end of file
+
+  @Override
+  public String getMetastoreDbUuid() throws MetaException {
+    throw new MetaException("getMetastoreDbUuid is not implemented");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
index bc00d11..462768d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.metastore;
 
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.util.StringUtils;
 
 public class TestEmbeddedHiveMetaStore extends TestHiveMetaStore {
@@ -27,22 +26,27 @@ public class TestEmbeddedHiveMetaStore extends TestHiveMetaStore {
   protected void setUp() throws Exception {
     super.setUp();
     warehouse = new Warehouse(hiveConf);
+    client = createClient();
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
     try {
-      client = new HiveMetaStoreClient(hiveConf);
+      super.tearDown();
+      client.close();
     } catch (Throwable e) {
-      System.err.println("Unable to open the metastore");
+      System.err.println("Unable to close metastore");
       System.err.println(StringUtils.stringifyException(e));
       throw new Exception(e);
     }
   }
 
   @Override
-  protected void tearDown() throws Exception {
+  protected HiveMetaStoreClient createClient() throws Exception {
     try {
-      super.tearDown();
-      client.close();
+      return new HiveMetaStoreClient(hiveConf);
     } catch (Throwable e) {
-      System.err.println("Unable to close metastore");
+      System.err.println("Unable to open the metastore");
       System.err.println(StringUtils.stringifyException(e));
       throw new Exception(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index b95c25c..4f7d56b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -31,6 +31,10 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 import junit.framework.TestCase;
 
@@ -99,6 +103,8 @@ public abstract class TestHiveMetaStore extends TestCase {
 
   private static final int DEFAULT_LIMIT_PARTITION_REQUEST = 100;
 
+  protected abstract HiveMetaStoreClient createClient() throws Exception;
+
   @Override
   protected void setUp() throws Exception {
     hiveConf = new HiveConf(this.getClass());
@@ -110,6 +116,7 @@ public abstract class TestHiveMetaStore extends TestCase {
     hiveConf.set("hive.key2", "http://www.example.com");
     hiveConf.set("hive.key3", "");
     hiveConf.set("hive.key4", "0");
+    hiveConf.set("datanucleus.autoCreateTables", "false");
 
     hiveConf.setIntVar(ConfVars.METASTORE_BATCH_RETRIEVE_MAX, 2);
     hiveConf.setIntVar(ConfVars.METASTORE_LIMIT_PARTITION_REQUEST, DEFAULT_LIMIT_PARTITION_REQUEST);
@@ -3324,4 +3331,46 @@ public abstract class TestHiveMetaStore extends TestCase {
       throw e;
     }
   }
+
+  public void testGetMetastoreUuid() throws Throwable {
+    String uuid = client.getMetastoreDbUuid();
+    assertNotNull(uuid);
+  }
+
+  public void testGetUUIDInParallel() throws Exception {
+    int numThreads = 5;
+    int parallelCalls = 10;
+    int numAPICallsPerThread = 10;
+    ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+    List<Future<List<String>>> futures = new ArrayList<>();
+    for (int n = 0; n < parallelCalls; n++) {
+      futures.add(executorService.submit(new Callable<List<String>>() {
+        @Override
+        public List<String> call() throws Exception {
+          HiveMetaStoreClient testClient = new HiveMetaStoreClient(hiveConf);
+          List<String> uuids = new ArrayList<>(10);
+          for (int i = 0; i < numAPICallsPerThread; i++) {
+            String uuid = testClient.getMetastoreDbUuid();
+            uuids.add(uuid);
+          }
+          return uuids;
+        }
+      }));
+    }
+
+    String firstUUID = null;
+    List<String> allUuids = new ArrayList<String>();
+    for (Future<List<String>> future : futures) {
+      for (String uuid : future.get()) {
+        if (firstUUID == null) {
+          firstUUID = uuid;
+        } else {
+          assertEquals(firstUUID.toLowerCase(), uuid.toLowerCase());
+        }
+        allUuids.add(uuid);
+      }
+    }
+    int size = allUuids.size();
+    assertEquals(numAPICallsPerThread * parallelCalls, size);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
index ef02968..878f913 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.shims.ShimLoader;
 
 public class TestRemoteHiveMetaStore extends TestHiveMetaStore {
   private static boolean isServerStarted = false;
-  private static int port;
+  protected static int port;
 
   public TestRemoteHiveMetaStore() {
     super();
@@ -48,12 +48,13 @@ public class TestRemoteHiveMetaStore extends TestHiveMetaStore {
     isServerStarted = true;
 
     // This is default case with setugi off for both client and server
-    createClient(false);
+    client = createClient();
   }
 
-  protected void createClient(boolean setugi) throws Exception {
+  @Override
+  protected HiveMetaStoreClient createClient() throws Exception {
     hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    hiveConf.setBoolVar(ConfVars.METASTORE_EXECUTE_SET_UGI,setugi);
-    client = new HiveMetaStoreClient(hiveConf);
+    hiveConf.setBoolVar(ConfVars.METASTORE_EXECUTE_SET_UGI, false);
+    return new HiveMetaStoreClient(hiveConf);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
index 29768c1..1a9abc9 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
@@ -18,11 +18,14 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.conf.HiveConf;
+
 public class TestSetUGIOnOnlyClient extends TestRemoteHiveMetaStore{
 
   @Override
-  protected void createClient(boolean setugi) throws Exception {
-    // turn it on for client.
-    super.createClient(true);
+  protected HiveMetaStoreClient createClient() throws Exception {
+    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
+    hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true);
+    return new HiveMetaStoreClient(hiveConf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
index 4a46f75..b45fd01 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
@@ -18,11 +18,14 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.conf.HiveConf;
+
 public class TestSetUGIOnOnlyServer extends TestSetUGIOnBothClientServer {
 
   @Override
-  protected void createClient(boolean setugi) throws Exception {
-    // It is turned on for both client and server because of super class. Turn it off for client.
-    super.createClient(false);
+  protected HiveMetaStoreClient createClient() throws Exception {
+    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
+    hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, false);
+    return new HiveMetaStoreClient(hiveConf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index ca6a007..53e5f29 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -1491,6 +1491,8 @@ service ThriftHiveMetastore extends fb303.FacebookService
   ClearFileMetadataResult clear_file_metadata(1:ClearFileMetadataRequest req)
   CacheFileMetadataResult cache_file_metadata(1:CacheFileMetadataRequest req)
 
+  // Metastore DB properties
+  string get_metastore_db_uuid() throws (1:MetaException o1)
 }
 
 // * Note about the DDL_TIME: When creating or altering a table or a partition,

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 9042cdb..273d37e 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -38187,6 +38187,192 @@ uint32_t ThriftHiveMetastore_cache_file_metadata_presult::read(::apache::thrift:
   return xfer;
 }
 
+
+ThriftHiveMetastore_get_metastore_db_uuid_args::~ThriftHiveMetastore_get_metastore_db_uuid_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    xfer += iprot->skip(ftype);
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_args");
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_metastore_db_uuid_pargs::~ThriftHiveMetastore_get_metastore_db_uuid_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_pargs");
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_metastore_db_uuid_result::~ThriftHiveMetastore_get_metastore_db_uuid_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->success);
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_result");
+
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0);
+    xfer += oprot->writeString(this->success);
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_metastore_db_uuid_presult::~ThriftHiveMetastore_get_metastore_db_uuid_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString((*(this->success)));
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
 void ThriftHiveMetastoreClient::getMetaConf(std::string& _return, const std::string& key)
 {
   send_getMetaConf(key);
@@ -47932,6 +48118,66 @@ void ThriftHiveMetastoreClient::recv_cache_file_metadata(CacheFileMetadataResult
   throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cache_file_metadata failed: unknown result");
 }
 
+void ThriftHiveMetastoreClient::get_metastore_db_uuid(std::string& _return)
+{
+  send_get_metastore_db_uuid();
+  recv_get_metastore_db_uuid(_return);
+}
+
+void ThriftHiveMetastoreClient::send_get_metastore_db_uuid()
+{
+  int32_t cseqid = 0;
+  oprot_->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_get_metastore_db_uuid_pargs args;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+}
+
+void ThriftHiveMetastoreClient::recv_get_metastore_db_uuid(std::string& _return)
+{
+
+  int32_t rseqid = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TMessageType mtype;
+
+  iprot_->readMessageBegin(fname, mtype, rseqid);
+  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
+    ::apache::thrift::TApplicationException x;
+    x.read(iprot_);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+    throw x;
+  }
+  if (mtype != ::apache::thrift::protocol::T_REPLY) {
+    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+  }
+  if (fname.compare("get_metastore_db_uuid") != 0) {
+    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+  }
+  ThriftHiveMetastore_get_metastore_db_uuid_presult result;
+  result.success = &_return;
+  result.read(iprot_);
+  iprot_->readMessageEnd();
+  iprot_->getTransport()->readEnd();
+
+  if (result.__isset.success) {
+    // _return pointer has now been filled
+    return;
+  }
+  if (result.__isset.o1) {
+    throw result.o1;
+  }
+  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_metastore_db_uuid failed: unknown result");
+}
+
 bool ThriftHiveMetastoreProcessor::dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext) {
   ProcessMap::iterator pfn;
   pfn = processMap_.find(fname);
@@ -57050,6 +57296,63 @@ void ThriftHiveMetastoreProcessor::process_cache_file_metadata(int32_t seqid, ::
   }
 }
 
+void ThriftHiveMetastoreProcessor::process_get_metastore_db_uuid(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
+{
+  void* ctx = NULL;
+  if (this->eventHandler_.get() != NULL) {
+    ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_metastore_db_uuid", callContext);
+  }
+  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_metastore_db_uuid");
+
+  if (this->eventHandler_.get() != NULL) {
+    this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_metastore_db_uuid");
+  }
+
+  ThriftHiveMetastore_get_metastore_db_uuid_args args;
+  args.read(iprot);
+  iprot->readMessageEnd();
+  uint32_t bytes = iprot->getTransport()->readEnd();
+
+  if (this->eventHandler_.get() != NULL) {
+    this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_metastore_db_uuid", bytes);
+  }
+
+  ThriftHiveMetastore_get_metastore_db_uuid_result result;
+  try {
+    iface_->get_metastore_db_uuid(result.success);
+    result.__isset.success = true;
+  } catch (MetaException &o1) {
+    result.o1 = o1;
+    result.__isset.o1 = true;
+  } catch (const std::exception& e) {
+    if (this->eventHandler_.get() != NULL) {
+      this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_metastore_db_uuid");
+    }
+
+    ::apache::thrift::TApplicationException x(e.what());
+    oprot->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_EXCEPTION, seqid);
+    x.write(oprot);
+    oprot->writeMessageEnd();
+    oprot->getTransport()->writeEnd();
+    oprot->getTransport()->flush();
+    return;
+  }
+
+  if (this->eventHandler_.get() != NULL) {
+    this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_metastore_db_uuid");
+  }
+
+  oprot->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_REPLY, seqid);
+  result.write(oprot);
+  oprot->writeMessageEnd();
+  bytes = oprot->getTransport()->writeEnd();
+  oprot->getTransport()->flush();
+
+  if (this->eventHandler_.get() != NULL) {
+    this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_metastore_db_uuid", bytes);
+  }
+}
+
 ::boost::shared_ptr< ::apache::thrift::TProcessor > ThriftHiveMetastoreProcessorFactory::getProcessor(const ::apache::thrift::TConnectionInfo& connInfo) {
   ::apache::thrift::ReleaseHandler< ThriftHiveMetastoreIfFactory > cleanup(handlerFactory_);
   ::boost::shared_ptr< ThriftHiveMetastoreIf > handler(handlerFactory_->getHandler(connInfo), cleanup);
@@ -71046,5 +71349,92 @@ void ThriftHiveMetastoreConcurrentClient::recv_cache_file_metadata(CacheFileMeta
   } // end while(true)
 }
 
+void ThriftHiveMetastoreConcurrentClient::get_metastore_db_uuid(std::string& _return)
+{
+  int32_t seqid = send_get_metastore_db_uuid();
+  recv_get_metastore_db_uuid(_return, seqid);
+}
+
+int32_t ThriftHiveMetastoreConcurrentClient::send_get_metastore_db_uuid()
+{
+  int32_t cseqid = this->sync_.generateSeqId();
+  ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_);
+  oprot_->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_get_metastore_db_uuid_pargs args;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+
+  sentry.commit();
+  return cseqid;
+}
+
+void ThriftHiveMetastoreConcurrentClient::recv_get_metastore_db_uuid(std::string& _return, const int32_t seqid)
+{
+
+  int32_t rseqid = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TMessageType mtype;
+
+  // the read mutex gets dropped and reacquired as part of waitForWork()
+  // The destructor of this sentry wakes up other clients
+  ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid);
+
+  while(true) {
+    if(!this->sync_.getPending(fname, mtype, rseqid)) {
+      iprot_->readMessageBegin(fname, mtype, rseqid);
+    }
+    if(seqid == rseqid) {
+      if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
+        ::apache::thrift::TApplicationException x;
+        x.read(iprot_);
+        iprot_->readMessageEnd();
+        iprot_->getTransport()->readEnd();
+        sentry.commit();
+        throw x;
+      }
+      if (mtype != ::apache::thrift::protocol::T_REPLY) {
+        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+        iprot_->readMessageEnd();
+        iprot_->getTransport()->readEnd();
+      }
+      if (fname.compare("get_metastore_db_uuid") != 0) {
+        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+        iprot_->readMessageEnd();
+        iprot_->getTransport()->readEnd();
+
+        // in a bad state, don't commit
+        using ::apache::thrift::protocol::TProtocolException;
+        throw TProtocolException(TProtocolException::INVALID_DATA);
+      }
+      ThriftHiveMetastore_get_metastore_db_uuid_presult result;
+      result.success = &_return;
+      result.read(iprot_);
+      iprot_->readMessageEnd();
+      iprot_->getTransport()->readEnd();
+
+      if (result.__isset.success) {
+        // _return pointer has now been filled
+        sentry.commit();
+        return;
+      }
+      if (result.__isset.o1) {
+        sentry.commit();
+        throw result.o1;
+      }
+      // in a bad state, don't commit
+      throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_metastore_db_uuid failed: unknown result");
+    }
+    // seqid != rseqid
+    this->sync_.updatePending(fname, mtype, rseqid);
+
+    // this will temporarily unlock the readMutex, and let other clients get work done
+    this->sync_.waitForWork(seqid);
+  } // end while(true)
+}
+
 }}} // namespace
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index ca71711..ac08ce1 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -176,6 +176,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) = 0;
   virtual void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) = 0;
   virtual void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) = 0;
+  virtual void get_metastore_db_uuid(std::string& _return) = 0;
 };
 
 class ThriftHiveMetastoreIfFactory : virtual public  ::facebook::fb303::FacebookServiceIfFactory {
@@ -695,6 +696,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void cache_file_metadata(CacheFileMetadataResult& /* _return */, const CacheFileMetadataRequest& /* req */) {
     return;
   }
+  void get_metastore_db_uuid(std::string& /* _return */) {
+    return;
+  }
 };
 
 typedef struct _ThriftHiveMetastore_getMetaConf_args__isset {
@@ -19723,6 +19727,106 @@ class ThriftHiveMetastore_cache_file_metadata_presult {
 
 };
 
+
+class ThriftHiveMetastore_get_metastore_db_uuid_args {
+ public:
+
+  ThriftHiveMetastore_get_metastore_db_uuid_args(const ThriftHiveMetastore_get_metastore_db_uuid_args&);
+  ThriftHiveMetastore_get_metastore_db_uuid_args& operator=(const ThriftHiveMetastore_get_metastore_db_uuid_args&);
+  ThriftHiveMetastore_get_metastore_db_uuid_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_metastore_db_uuid_args() throw();
+
+  bool operator == (const ThriftHiveMetastore_get_metastore_db_uuid_args & /* rhs */) const
+  {
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_metastore_db_uuid_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_metastore_db_uuid_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_metastore_db_uuid_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_metastore_db_uuid_pargs() throw();
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_metastore_db_uuid_result__isset {
+  _ThriftHiveMetastore_get_metastore_db_uuid_result__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_metastore_db_uuid_result__isset;
+
+class ThriftHiveMetastore_get_metastore_db_uuid_result {
+ public:
+
+  ThriftHiveMetastore_get_metastore_db_uuid_result(const ThriftHiveMetastore_get_metastore_db_uuid_result&);
+  ThriftHiveMetastore_get_metastore_db_uuid_result& operator=(const ThriftHiveMetastore_get_metastore_db_uuid_result&);
+  ThriftHiveMetastore_get_metastore_db_uuid_result() : success() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_metastore_db_uuid_result() throw();
+  std::string success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_metastore_db_uuid_result__isset __isset;
+
+  void __set_success(const std::string& val);
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_metastore_db_uuid_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_metastore_db_uuid_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_metastore_db_uuid_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset {
+  _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset;
+
+class ThriftHiveMetastore_get_metastore_db_uuid_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_metastore_db_uuid_presult() throw();
+  std::string* success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public  ::facebook::fb303::FacebookServiceClient {
  public:
   ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :
@@ -20196,6 +20300,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req);
   void send_cache_file_metadata(const CacheFileMetadataRequest& req);
   void recv_cache_file_metadata(CacheFileMetadataResult& _return);
+  void get_metastore_db_uuid(std::string& _return);
+  void send_get_metastore_db_uuid();
+  void recv_get_metastore_db_uuid(std::string& _return);
 };
 
 class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceProcessor {
@@ -20360,6 +20467,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_put_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_clear_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_cache_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_metastore_db_uuid(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
  public:
   ThriftHiveMetastoreProcessor(boost::shared_ptr<ThriftHiveMetastoreIf> iface) :
      ::facebook::fb303::FacebookServiceProcessor(iface),
@@ -20518,6 +20626,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["put_file_metadata"] = &ThriftHiveMetastoreProcessor::process_put_file_metadata;
     processMap_["clear_file_metadata"] = &ThriftHiveMetastoreProcessor::process_clear_file_metadata;
     processMap_["cache_file_metadata"] = &ThriftHiveMetastoreProcessor::process_cache_file_metadata;
+    processMap_["get_metastore_db_uuid"] = &ThriftHiveMetastoreProcessor::process_get_metastore_db_uuid;
   }
 
   virtual ~ThriftHiveMetastoreProcessor() {}
@@ -22027,6 +22136,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void get_metastore_db_uuid(std::string& _return) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_metastore_db_uuid(_return);
+    }
+    ifaces_[i]->get_metastore_db_uuid(_return);
+    return;
+  }
+
 };
 
 // The 'concurrent' client is a thread safe client that correctly handles
@@ -22505,6 +22624,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req);
   int32_t send_cache_file_metadata(const CacheFileMetadataRequest& req);
   void recv_cache_file_metadata(CacheFileMetadataResult& _return, const int32_t seqid);
+  void get_metastore_db_uuid(std::string& _return);
+  int32_t send_get_metastore_db_uuid();
+  void recv_get_metastore_db_uuid(std::string& _return, const int32_t seqid);
 };
 
 #ifdef _WIN32

http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index b4a2a92..80786e1 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -792,6 +792,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("cache_file_metadata\n");
   }
 
+  void get_metastore_db_uuid(std::string& _return) {
+    // Your implementation goes here
+    printf("get_metastore_db_uuid\n");
+  }
+
 };
 
 int main(int argc, char **argv) {


[2/3] hive git commit: HIVE-16555 Add a new thrift API call for get_metastore_uuid (Vihang Karajgaonkar, reviewed by Naveen Gangam, Sahil Takiar and Alan Gates)

Posted by ng...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c9cfb601/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 1915150..19c88cb 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -350,6 +350,8 @@ public class ThriftHiveMetastore {
 
     public CacheFileMetadataResult cache_file_metadata(CacheFileMetadataRequest req) throws org.apache.thrift.TException;
 
+    public String get_metastore_db_uuid() throws MetaException, org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
@@ -662,6 +664,8 @@ public class ThriftHiveMetastore {
 
     public void cache_file_metadata(CacheFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_metastore_db_uuid(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface {
@@ -5111,6 +5115,31 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "cache_file_metadata failed: unknown result");
     }
 
+    public String get_metastore_db_uuid() throws MetaException, org.apache.thrift.TException
+    {
+      send_get_metastore_db_uuid();
+      return recv_get_metastore_db_uuid();
+    }
+
+    public void send_get_metastore_db_uuid() throws org.apache.thrift.TException
+    {
+      get_metastore_db_uuid_args args = new get_metastore_db_uuid_args();
+      sendBase("get_metastore_db_uuid", args);
+    }
+
+    public String recv_get_metastore_db_uuid() throws MetaException, org.apache.thrift.TException
+    {
+      get_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
+      receiveBase(result, "get_metastore_db_uuid");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_metastore_db_uuid failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends com.facebook.fb303.FacebookService.AsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -10552,6 +10581,35 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void get_metastore_db_uuid(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_metastore_db_uuid_call method_call = new get_metastore_db_uuid_call(resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class get_metastore_db_uuid_call extends org.apache.thrift.async.TAsyncMethodCall {
+      public get_metastore_db_uuid_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_metastore_db_uuid", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_metastore_db_uuid_args args = new get_metastore_db_uuid_args();
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public String getResult() throws MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_metastore_db_uuid();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends com.facebook.fb303.FacebookService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -10719,6 +10777,7 @@ public class ThriftHiveMetastore {
       processMap.put("put_file_metadata", new put_file_metadata());
       processMap.put("clear_file_metadata", new clear_file_metadata());
       processMap.put("cache_file_metadata", new cache_file_metadata());
+      processMap.put("get_metastore_db_uuid", new get_metastore_db_uuid());
       return processMap;
     }
 
@@ -14642,6 +14701,30 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_metastore_db_uuid<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_metastore_db_uuid_args> {
+      public get_metastore_db_uuid() {
+        super("get_metastore_db_uuid");
+      }
+
+      public get_metastore_db_uuid_args getEmptyArgsInstance() {
+        return new get_metastore_db_uuid_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_metastore_db_uuid_result getResult(I iface, get_metastore_db_uuid_args args) throws org.apache.thrift.TException {
+        get_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
+        try {
+          result.success = iface.get_metastore_db_uuid();
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends com.facebook.fb303.FacebookService.AsyncProcessor<I> {
@@ -14809,6 +14892,7 @@ public class ThriftHiveMetastore {
       processMap.put("put_file_metadata", new put_file_metadata());
       processMap.put("clear_file_metadata", new clear_file_metadata());
       processMap.put("cache_file_metadata", new cache_file_metadata());
+      processMap.put("get_metastore_db_uuid", new get_metastore_db_uuid());
       return processMap;
     }
 
@@ -24168,6 +24252,63 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class get_metastore_db_uuid<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_metastore_db_uuid_args, String> {
+      public get_metastore_db_uuid() {
+        super("get_metastore_db_uuid");
+      }
+
+      public get_metastore_db_uuid_args getEmptyArgsInstance() {
+        return new get_metastore_db_uuid_args();
+      }
+
+      public AsyncMethodCallback<String> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<String>() { 
+          public void onComplete(String o) {
+            get_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_metastore_db_uuid_args args, org.apache.thrift.async.AsyncMethodCallback<String> resultHandler) throws TException {
+        iface.get_metastore_db_uuid(resultHandler);
+      }
+    }
+
   }
 
   public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
@@ -177693,15 +177834,741 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_file_metadata_by_expr_argsStandardSchemeFactory implements SchemeFactory {
-      public get_file_metadata_by_expr_argsStandardScheme getScheme() {
-        return new get_file_metadata_by_expr_argsStandardScheme();
+    private static class get_file_metadata_by_expr_argsStandardSchemeFactory implements SchemeFactory {
+      public get_file_metadata_by_expr_argsStandardScheme getScheme() {
+        return new get_file_metadata_by_expr_argsStandardScheme();
+      }
+    }
+
+    private static class get_file_metadata_by_expr_argsStandardScheme extends StandardScheme<get_file_metadata_by_expr_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // REQ
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.req = new GetFileMetadataByExprRequest();
+                struct.req.read(iprot);
+                struct.setReqIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.req != null) {
+          oprot.writeFieldBegin(REQ_FIELD_DESC);
+          struct.req.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_file_metadata_by_expr_argsTupleSchemeFactory implements SchemeFactory {
+      public get_file_metadata_by_expr_argsTupleScheme getScheme() {
+        return new get_file_metadata_by_expr_argsTupleScheme();
+      }
+    }
+
+    private static class get_file_metadata_by_expr_argsTupleScheme extends TupleScheme<get_file_metadata_by_expr_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetReq()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetReq()) {
+          struct.req.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.req = new GetFileMetadataByExprRequest();
+          struct.req.read(iprot);
+          struct.setReqIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_file_metadata_by_expr_result implements org.apache.thrift.TBase<get_file_metadata_by_expr_result, get_file_metadata_by_expr_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_by_expr_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_by_expr_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_file_metadata_by_expr_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_file_metadata_by_expr_resultTupleSchemeFactory());
+    }
+
+    private GetFileMetadataByExprResult success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataByExprResult.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_by_expr_result.class, metaDataMap);
+    }
+
+    public get_file_metadata_by_expr_result() {
+    }
+
+    public get_file_metadata_by_expr_result(
+      GetFileMetadataByExprResult success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_file_metadata_by_expr_result(get_file_metadata_by_expr_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new GetFileMetadataByExprResult(other.success);
+      }
+    }
+
+    public get_file_metadata_by_expr_result deepCopy() {
+      return new get_file_metadata_by_expr_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public GetFileMetadataByExprResult getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(GetFileMetadataByExprResult success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((GetFileMetadataByExprResult)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_file_metadata_by_expr_result)
+        return this.equals((get_file_metadata_by_expr_result)that);
+      return false;
+    }
+
+    public boolean equals(get_file_metadata_by_expr_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_file_metadata_by_expr_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("get_file_metadata_by_expr_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class get_file_metadata_by_expr_resultStandardSchemeFactory implements SchemeFactory {
+      public get_file_metadata_by_expr_resultStandardScheme getScheme() {
+        return new get_file_metadata_by_expr_resultStandardScheme();
+      }
+    }
+
+    private static class get_file_metadata_by_expr_resultStandardScheme extends StandardScheme<get_file_metadata_by_expr_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new GetFileMetadataByExprResult();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_file_metadata_by_expr_resultTupleSchemeFactory implements SchemeFactory {
+      public get_file_metadata_by_expr_resultTupleScheme getScheme() {
+        return new get_file_metadata_by_expr_resultTupleScheme();
+      }
+    }
+
+    private static class get_file_metadata_by_expr_resultTupleScheme extends TupleScheme<get_file_metadata_by_expr_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new GetFileMetadataByExprResult();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_file_metadata_args implements org.apache.thrift.TBase<get_file_metadata_args, get_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_args");
+
+    private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_file_metadata_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_file_metadata_argsTupleSchemeFactory());
+    }
+
+    private GetFileMetadataRequest req; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      REQ((short)1, "req");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // REQ
+            return REQ;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_args.class, metaDataMap);
+    }
+
+    public get_file_metadata_args() {
+    }
+
+    public get_file_metadata_args(
+      GetFileMetadataRequest req)
+    {
+      this();
+      this.req = req;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_file_metadata_args(get_file_metadata_args other) {
+      if (other.isSetReq()) {
+        this.req = new GetFileMetadataRequest(other.req);
+      }
+    }
+
+    public get_file_metadata_args deepCopy() {
+      return new get_file_metadata_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.req = null;
+    }
+
+    public GetFileMetadataRequest getReq() {
+      return this.req;
+    }
+
+    public void setReq(GetFileMetadataRequest req) {
+      this.req = req;
+    }
+
+    public void unsetReq() {
+      this.req = null;
+    }
+
+    /** Returns true if field req is set (has been assigned a value) and false otherwise */
+    public boolean isSetReq() {
+      return this.req != null;
+    }
+
+    public void setReqIsSet(boolean value) {
+      if (!value) {
+        this.req = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case REQ:
+        if (value == null) {
+          unsetReq();
+        } else {
+          setReq((GetFileMetadataRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case REQ:
+        return getReq();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case REQ:
+        return isSetReq();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_file_metadata_args)
+        return this.equals((get_file_metadata_args)that);
+      return false;
+    }
+
+    public boolean equals(get_file_metadata_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_req = true && this.isSetReq();
+      boolean that_present_req = true && that.isSetReq();
+      if (this_present_req || that_present_req) {
+        if (!(this_present_req && that_present_req))
+          return false;
+        if (!this.req.equals(that.req))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_req = true && (isSetReq());
+      list.add(present_req);
+      if (present_req)
+        list.add(req);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_file_metadata_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetReq()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("get_file_metadata_args(");
+      boolean first = true;
+
+      sb.append("req:");
+      if (this.req == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.req);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (req != null) {
+        req.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class get_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
+      public get_file_metadata_argsStandardScheme getScheme() {
+        return new get_file_metadata_argsStandardScheme();
       }
     }
 
-    private static class get_file_metadata_by_expr_argsStandardScheme extends StandardScheme<get_file_metadata_by_expr_args> {
+    private static class get_file_metadata_argsStandardScheme extends StandardScheme<get_file_metadata_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -177713,7 +178580,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 1: // REQ
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.req = new GetFileMetadataByExprRequest();
+                struct.req = new GetFileMetadataRequest();
                 struct.req.read(iprot);
                 struct.setReqIsSet(true);
               } else { 
@@ -177729,7 +178596,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -177744,16 +178611,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_file_metadata_by_expr_argsTupleSchemeFactory implements SchemeFactory {
-      public get_file_metadata_by_expr_argsTupleScheme getScheme() {
-        return new get_file_metadata_by_expr_argsTupleScheme();
+    private static class get_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
+      public get_file_metadata_argsTupleScheme getScheme() {
+        return new get_file_metadata_argsTupleScheme();
       }
     }
 
-    private static class get_file_metadata_by_expr_argsTupleScheme extends TupleScheme<get_file_metadata_by_expr_args> {
+    private static class get_file_metadata_argsTupleScheme extends TupleScheme<get_file_metadata_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetReq()) {
@@ -177766,11 +178633,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.req = new GetFileMetadataByExprRequest();
+          struct.req = new GetFileMetadataRequest();
           struct.req.read(iprot);
           struct.setReqIsSet(true);
         }
@@ -177779,18 +178646,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class get_file_metadata_by_expr_result implements org.apache.thrift.TBase<get_file_metadata_by_expr_result, get_file_metadata_by_expr_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_by_expr_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_by_expr_result");
+  public static class get_file_metadata_result implements org.apache.thrift.TBase<get_file_metadata_result, get_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_file_metadata_by_expr_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_file_metadata_by_expr_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_file_metadata_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_file_metadata_resultTupleSchemeFactory());
     }
 
-    private GetFileMetadataByExprResult success; // required
+    private GetFileMetadataResult success; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -177855,16 +178722,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataByExprResult.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataResult.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_by_expr_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_result.class, metaDataMap);
     }
 
-    public get_file_metadata_by_expr_result() {
+    public get_file_metadata_result() {
     }
 
-    public get_file_metadata_by_expr_result(
-      GetFileMetadataByExprResult success)
+    public get_file_metadata_result(
+      GetFileMetadataResult success)
     {
       this();
       this.success = success;
@@ -177873,14 +178740,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_file_metadata_by_expr_result(get_file_metadata_by_expr_result other) {
+    public get_file_metadata_result(get_file_metadata_result other) {
       if (other.isSetSuccess()) {
-        this.success = new GetFileMetadataByExprResult(other.success);
+        this.success = new GetFileMetadataResult(other.success);
       }
     }
 
-    public get_file_metadata_by_expr_result deepCopy() {
-      return new get_file_metadata_by_expr_result(this);
+    public get_file_metadata_result deepCopy() {
+      return new get_file_metadata_result(this);
     }
 
     @Override
@@ -177888,11 +178755,11 @@ public class ThriftHiveMetastore {
       this.success = null;
     }
 
-    public GetFileMetadataByExprResult getSuccess() {
+    public GetFileMetadataResult getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(GetFileMetadataByExprResult success) {
+    public void setSuccess(GetFileMetadataResult success) {
       this.success = success;
     }
 
@@ -177917,7 +178784,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((GetFileMetadataByExprResult)value);
+          setSuccess((GetFileMetadataResult)value);
         }
         break;
 
@@ -177950,12 +178817,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_file_metadata_by_expr_result)
-        return this.equals((get_file_metadata_by_expr_result)that);
+      if (that instanceof get_file_metadata_result)
+        return this.equals((get_file_metadata_result)that);
       return false;
     }
 
-    public boolean equals(get_file_metadata_by_expr_result that) {
+    public boolean equals(get_file_metadata_result that) {
       if (that == null)
         return false;
 
@@ -177984,7 +178851,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_file_metadata_by_expr_result other) {
+    public int compareTo(get_file_metadata_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -178018,7 +178885,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_file_metadata_by_expr_result(");
+      StringBuilder sb = new StringBuilder("get_file_metadata_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -178056,15 +178923,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_file_metadata_by_expr_resultStandardSchemeFactory implements SchemeFactory {
-      public get_file_metadata_by_expr_resultStandardScheme getScheme() {
-        return new get_file_metadata_by_expr_resultStandardScheme();
+    private static class get_file_metadata_resultStandardSchemeFactory implements SchemeFactory {
+      public get_file_metadata_resultStandardScheme getScheme() {
+        return new get_file_metadata_resultStandardScheme();
       }
     }
 
-    private static class get_file_metadata_by_expr_resultStandardScheme extends StandardScheme<get_file_metadata_by_expr_result> {
+    private static class get_file_metadata_resultStandardScheme extends StandardScheme<get_file_metadata_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -178076,7 +178943,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new GetFileMetadataByExprResult();
+                struct.success = new GetFileMetadataResult();
                 struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
@@ -178092,7 +178959,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -178107,16 +178974,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_file_metadata_by_expr_resultTupleSchemeFactory implements SchemeFactory {
-      public get_file_metadata_by_expr_resultTupleScheme getScheme() {
-        return new get_file_metadata_by_expr_resultTupleScheme();
+    private static class get_file_metadata_resultTupleSchemeFactory implements SchemeFactory {
+      public get_file_metadata_resultTupleScheme getScheme() {
+        return new get_file_metadata_resultTupleScheme();
       }
     }
 
-    private static class get_file_metadata_by_expr_resultTupleScheme extends TupleScheme<get_file_metadata_by_expr_result> {
+    private static class get_file_metadata_resultTupleScheme extends TupleScheme<get_file_metadata_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -178129,11 +178996,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new GetFileMetadataByExprResult();
+          struct.success = new GetFileMetadataResult();
           struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
@@ -178142,18 +179009,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class get_file_metadata_args implements org.apache.thrift.TBase<get_file_metadata_args, get_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_args");
+  public static class put_file_metadata_args implements org.apache.thrift.TBase<put_file_metadata_args, put_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<put_file_metadata_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_file_metadata_args");
 
     private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_file_metadata_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_file_metadata_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new put_file_metadata_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new put_file_metadata_argsTupleSchemeFactory());
     }
 
-    private GetFileMetadataRequest req; // required
+    private PutFileMetadataRequest req; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -178218,16 +179085,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PutFileMetadataRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(put_file_metadata_args.class, metaDataMap);
     }
 
-    public get_file_metadata_args() {
+    public put_file_metadata_args() {
     }
 
-    public get_file_metadata_args(
-      GetFileMetadataRequest req)
+    public put_file_metadata_args(
+      PutFileMetadataRequest req)
     {
       this();
       this.req = req;
@@ -178236,14 +179103,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_file_metadata_args(get_file_metadata_args other) {
+    public put_file_metadata_args(put_file_metadata_args other) {
       if (other.isSetReq()) {
-        this.req = new GetFileMetadataRequest(other.req);
+        this.req = new PutFileMetadataRequest(other.req);
       }
     }
 
-    public get_file_metadata_args deepCopy() {
-      return new get_file_metadata_args(this);
+    public put_file_metadata_args deepCopy() {
+      return new put_file_metadata_args(this);
     }
 
     @Override
@@ -178251,11 +179118,11 @@ public class ThriftHiveMetastore {
       this.req = null;
     }
 
-    public GetFileMetadataRequest getReq() {
+    public PutFileMetadataRequest getReq() {
       return this.req;
     }
 
-    public void setReq(GetFileMetadataRequest req) {
+    public void setReq(PutFileMetadataRequest req) {
       this.req = req;
     }
 
@@ -178280,7 +179147,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetReq();
         } else {
-          setReq((GetFileMetadataRequest)value);
+          setReq((PutFileMetadataRequest)value);
         }
         break;
 
@@ -178313,12 +179180,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_file_metadata_args)
-        return this.equals((get_file_metadata_args)that);
+      if (that instanceof put_file_metadata_args)
+        return this.equals((put_file_metadata_args)that);
       return false;
     }
 
-    public boolean equals(get_file_metadata_args that) {
+    public boolean equals(put_file_metadata_args that) {
       if (that == null)
         return false;
 
@@ -178347,7 +179214,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_file_metadata_args other) {
+    public int compareTo(put_file_metadata_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -178381,7 +179248,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_file_metadata_args(");
+      StringBuilder sb = new StringBuilder("put_file_metadata_args(");
       boolean first = true;
 
       sb.append("req:");
@@ -178419,15 +179286,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
-      public get_file_metadata_argsStandardScheme getScheme() {
-        return new get_file_metadata_argsStandardScheme();
+    private static class put_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
+      public put_file_metadata_argsStandardScheme getScheme() {
+        return new put_file_metadata_argsStandardScheme();
       }
     }
 
-    private static class get_file_metadata_argsStandardScheme extends StandardScheme<get_file_metadata_args> {
+    private static class put_file_metadata_argsStandardScheme extends StandardScheme<put_file_metadata_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, put_file_metadata_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -178439,7 +179306,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 1: // REQ
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.req = new GetFileMetadataRequest();
+                struct.req = new PutFileMetadataRequest();
                 struct.req.read(iprot);
                 struct.setReqIsSet(true);
               } else { 
@@ -178455,7 +179322,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, put_file_metadata_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -178470,16 +179337,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
-      public get_file_metadata_argsTupleScheme getScheme() {
-        return new get_file_metadata_argsTupleScheme();
+    private static class put_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
+      public put_file_metadata_argsTupleScheme getScheme() {
+        return new put_file_metadata_argsTupleScheme();
       }
     }
 
-    private static class get_file_metadata_argsTupleScheme extends TupleScheme<get_file_metadata_args> {
+    private static class put_file_metadata_argsTupleScheme extends TupleScheme<put_file_metadata_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetReq()) {
@@ -178492,11 +179359,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.req = new GetFileMetadataRequest();
+          struct.req = new PutFileMetadataRequest();
           struct.req.read(iprot);
           struct.setReqIsSet(true);
         }
@@ -178505,18 +179372,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class get_file_metadata_result implements org.apache.thrift.TBase<get_file_metadata_result, get_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_result");
+  public static class put_file_metadata_result implements org.apache.thrift.TBase<put_file_metadata_result, put_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<put_file_metadata_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_file_metadata_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_file_metadata_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_file_metadata_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new put_file_metadata_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new put_file_metadata_resultTupleSchemeFactory());
     }
 
-    private GetFileMetadataResult success; // required
+    private PutFileMetadataResult success; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -178581,16 +179448,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataResult.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PutFileMetadataResult.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(put_file_metadata_result.class, metaDataMap);
     }
 
-    public get_file_metadata_result() {
+    public put_file_metadata_result() {
     }
 
-    public get_file_metadata_result(
-      GetFileMetadataResult success)
+    public put_file_metadata_result(
+      PutFileMetadataResult success)
     {
       this();
       this.success = success;
@@ -178599,14 +179466,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_file_metadata_result(get_file_metadata_result other) {
+    public put_file_metadata_result(put_file_metadata_result other) {
       if (other.isSetSuccess()) {
-        this.success = new GetFileMetadataResult(other.success);
+        this.success = new PutFileMetadataResult(other.success);
       }
     }
 
-    public get_file_metadata_result deepCopy() {
-      return new get_file_metadata_result(this);
+    public put_file_metadata_result deepCopy() {
+      return new put_file_metadata_result(this);
     }
 
     @Override
@@ -178614,11 +179481,11 @@ public class ThriftHiveMetastore {
       this.success = null;
     }
 
-    public GetFileMetadataResult getSuccess() {
+    public PutFileMetadataResult getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(GetFileMetadataResult success) {
+    public void setSuccess(PutFileMetadataResult success) {
       this.success = success;
     }
 
@@ -178643,7 +179510,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((GetFileMetadataResult)value);
+          setSuccess((PutFileMetadataResult)value);
         }
         break;
 
@@ -178676,12 +179543,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_file_metadata_result)
-        return this.equals((get_file_metadata_result)that);
+      if (that instanceof put_file_metadata_result)
+        return this.equals((put_file_metadata_result)that);
       return false;
     }
 
-    public boolean equals(get_file_metadata_result that) {
+    public boolean equals(put_file_metadata_result that) {
       if (that == null)
         return false;
 
@@ -178710,7 +179577,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_file_metadata_result other) {
+    public int compareTo(put_file_metadata_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -178744,7 +179611,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_file_metadata_result(");
+      StringBuilder sb = new StringBuilder("put_file_metadata_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -178782,15 +179649,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_file_metadata_resultStandardSchemeFactory implements SchemeFactory {
-      public get_file_metadata_resultStandardScheme getScheme() {
-        return new get_file_metadata_resultStandardScheme();
+    private static class put_file_metadata_resultStandardSchemeFactory implements SchemeFactory {
+      public put_file_metadata_resultStandardScheme getScheme() {
+        return new put_file_metadata_resultStandardScheme();
       }
     }
 
-    private static class get_file_metadata_resultStandardScheme extends StandardScheme<get_file_metadata_result> {
+    private static class put_file_metadata_resultStandardScheme extends StandardScheme<put_file_metadata_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, put_file_metadata_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -178802,7 +179669,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new GetFileMetadataResult();
+                struct.success = new PutFileMetadataResult();
                 struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
@@ -178818,7 +179685,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, put_file_metadata_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -178833,16 +179700,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_file_metadata_resultTupleSchemeFactory implements SchemeFactory {
-      public get_file_metadata_resultTupleScheme getScheme() {
-        return new get_file_metadata_resultTupleScheme();
+    private static class put_file_metadata_resultTupleSchemeFactory implements SchemeFactory {
+      public put_file_metadata_resultTupleScheme getScheme() {
+        return new put_file_metadata_resultTupleScheme();
       }
     }
 
-    private static class get_file_metadata_resultTupleScheme extends TupleScheme<get_file_metadata_result> {
+    private static class put_file_metadata_resultTupleScheme extends TupleScheme<put_file_metadata_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -178855,11 +179722,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new GetFileMetadataResult();
+          struct.success = new PutFileMetadataResult();
           struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
@@ -178868,18 +179735,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class put_file_metadata_args implements org.apache.thrift.TBase<put_file_metadata_args, put_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<put_file_metadata_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_file_metadata_args");
+  public static class clear_file_metadata_args implements org.apache.thrift.TBase<clear_file_metadata_args, clear_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<clear_file_metadata_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("clear_file_metadata_args");
 
     private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new put_file_metadata_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new put_file_metadata_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new clear_file_metadata_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new clear_file_metadata_argsTupleSchemeFactory());
     }
 
-    private PutFileMetadataRequest req; // required
+    private ClearFileMetadataRequest req; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -178944,16 +179811,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PutFileMetadataRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClearFileMetadataRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(put_file_metadata_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(clear_file_metadata_args.class, metaDataMap);
     }
 
-    public put_file_metadata_args() {
+    public clear_file_metadata_args() {
     }
 
-    public put_file_metadata_args(
-      PutFileMetadataRequest req)
+    public clear_file_metadata_args(
+      ClearFileMetadataRequest req)
     {
       this();
       this.req = req;
@@ -178962,14 +179829,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public put_file_metadata_args(put_file_metadata_args other) {
+    public clear_file_metadata_args(clear_file_metadata_args other) {
       if (other.isSetReq()) {
-        this.req = new PutFileMetadataRequest(other.req);
+        this.req = new ClearFileMetadataRequest(other.req);
       }
     }
 
-    public put_file_metadata_args deepCopy() {
-      return new put_file_metadata_args(this);
+    public clear_file_metadata_args deepCopy() {
+      return new clear_file_metadata_args(this);
     }
 
     @Override
@@ -178977,11 +179844,11 @@ public class ThriftHiveMetastore {
       this.req = null;
     }
 
-    public PutFileMetadataRequest getReq() {
+    public ClearFileMetadataRequest getReq() {
       return this.req;
     }
 
-    public void setReq(PutFileMetadataRequest req) {
+    public void setReq(ClearFileMetadataRequest req) {
       this.req = req;
     }
 
@@ -179006,7 +179873,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetReq();
         } else {
-          setReq((PutFileMetadataRequest)value);
+          setReq((ClearFileMetadataRequest)value);
         }
         break;
 
@@ -179039,12 +179906,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof put_file_metadata_args)
-        return this.equals((put_file_metadata_args)that);
+      if (that instanceof clear_file_metadata_args)
+        return this.equals((clear_file_metadata_args)that);
       return false;
     }
 
-    public boolean equals(put_file_metadata_args that) {
+    public boolean equals(clear_file_metadata_args that) {
       if (that == null)
         return false;
 
@@ -179073,7 +179940,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(put_file_metadata_args other) {
+    public int compareTo(clear_file_metadata_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -179107,7 +179974,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("put_file_metadata_args(");
+      StringBuilder sb = new StringBuilder("clear_file_metadata_args(");
       boolean first = true;
 
       sb.append("req:");
@@ -179145,15 +180012,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class put_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
-      public put_file_metadata_argsStandardScheme getScheme() {
-        return new put_file_metadata_argsStandardScheme();
+    private static class clear_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
+      public clear_file_metadata_argsStandardScheme getScheme() {
+        return new clear_file_metadata_argsStandardScheme();
       }
     }
 
-    private static class put_file_metadata_argsStandardScheme extends StandardScheme<put_file_metadata_args> {
+    private static class clear_file_metadata_argsStandardScheme extends StandardScheme<clear_file_metadata_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, put_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, clear_file_metadata_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -179165,7 +180032,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 1: // REQ
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.req = new PutFileMetadataRequest();
+                struct.req = new ClearFileMetadataRequest();
                 struct.req.read(iprot);
                 struct.setReqIsSet(true);
               } else { 
@@ -179181,7 +180048,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, put_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, clear_file_metadata_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -179196,16 +180063,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class put_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
-      public put_file_metadata_argsTupleScheme getScheme() {
-        return new put_file_metadata_argsTupleScheme();
+    private static class clear_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
+      public clear_file_metadata_argsTupleScheme getScheme() {
+        return new clear_file_metadata_argsTupleScheme();
       }
     }
 
-    private static class put_file_metadata_argsTupleScheme extends TupleScheme<put_file_metadata_args> {
+    private static class clear_file_metadata_argsTupleScheme extends TupleScheme<clear_file_metadata_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, clear_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetReq()) {
@@ -179218,11 +180085,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, clear_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.req = new PutFileMetadataRequest();
+          struct.req = new ClearFileMetadataRequest();
           struct.req.read(iprot);
           struct.setReqIsSet(true);
         }
@@ -179231,18 +180098,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class put_file_metadata_result implements org.apache.thrift.TBase<put_file_metadata_result, put_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<put_file_metadata_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_file_metadata_result");
+  public static class clear_file_metadata_result implements org.apache.thrift.TBase<clear_file_metadata_result, clear_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<clear_file_metadata_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("clear_file_metadata_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new put_file_metadata_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new put_file_metadata_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new clear_file_metadata_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new clear_file_metadata_resultTupleSchemeFactory());
     }
 
-    private PutFileMetadataResult success; // required
+    private ClearFileMetadataResult success; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -179307,16 +180174,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PutFileMetadataResult.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClearFileMetadataResult.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(put_file_metadata_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(clear_file_metadata_result.class, metaDataMap);
     }
 
-    public put_file_metadata_result() {
+    public clear_file_metadata_result() {
     }
 
-    public put_file_metadata_result(
-      PutFileMetadataResult success)
+    public clear_file_metadata_result(
+      ClearFileMetadataResult success)
     {
       this();
       this.success = success;
@@ -179325,14 +180192,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public put_file_metadata_result(put_file_metadata_result other) {
+    public clear_file_metadata_result(clear_file_metadata_result other) {
       if (other.isSetSuccess()) {
-        this.success = new PutFileMetadataResult(other.success);
+        this.success = new ClearFileMetadataResult(other.success);
       }
     }
 
-    public put_file_metadata_result deepCopy() {
-      return new put_file_metadata_result(this);
+    public clear_file_metadata_result deepCopy() {
+      return new clear_file_metadata_result(this);
     }
 
     @Override
@@ -179340,11 +180207,11 @@ public class ThriftHiveMetastore {
       this.success = null;
     }
 
-    public PutFileMetadataResult getSuccess() {
+    public ClearFileMetadataResult getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(PutFileMetadataResult success) {
+    public void setSuccess(ClearFileMetadataResult success) {
       this.success = success;
     }
 
@@ -179369,7 +180236,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((PutFileMetadataResult)value);
+          setSuccess((ClearFileMetadataResult)value);
         }
         break;
 
@@ -179402,12 +180269,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof put_file_metadata_result)
-        return this.equals((put_file_metadata_result)that);
+      if (that instanceof clear_file_metadata_result)
+        return this.equals((clear_file_metadata_result)that);
       return false;
     }
 
-    public boolean equals(put_file_metadata_result that) {
+    public boolean equals(clear_file_metadata_result that) {
       if (that == null)
         return false;
 
@@ -179436,7 +180303,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(put_file_metadata_result other) {
+    public int compareTo(clear_file_metadata_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -179470,7 +180337,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("put_file_metadata_result(");
+      StringBuilder sb = new StringBuilder("clear_file_metadata_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -179508,15 +180375,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class put_file_metadata_resultStandardSchemeFactory implements SchemeFactory {
-      public put_file_metadata_resultStandardScheme getScheme() {
-        return new put_file_metadata_resultStandardScheme();
+    private static class clear_file_metadata_resultStandardSchemeFactory implements SchemeFactory {
+      public clear_file_metadata_resultStandardScheme getScheme() {
+        return new clear_file_metadata_resultStandardScheme();
       }
     }
 
-    private static class put_file_metadata_resultStandardScheme extends StandardScheme<put_file_metadata_result> {
+    private static class clear_file_metadata_resultStandardScheme extends StandardScheme<clear_file_metadata_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, put_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, clear_file_metadata_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -179528,7 +180395,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new PutFileMetadataResult();
+                struct.success = new ClearFileMetadataResult();
                 struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
@@ -179544,7 +180411,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, put_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, clear_file_metadata_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -179559,16 +180426,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class put_file_metadata_resultTupleSchemeFactory implements SchemeFactory {
-      public put_file_metadata_resultTupleScheme getScheme() {
-        return new put_file_metadata_resultTupleScheme();
+    private static class clear_file_metadata_resultTupleSchemeFactory implements SchemeFactory {
+      public clear_file_metadata_resultTupleScheme getScheme() {
+        return new clear_file_metadata_resultTupleScheme();
       }
     }
 
-    private static class put_file_metadata_resultTupleScheme extends TupleScheme<put_file_metadata_result> {
+    private static class clear_file_metadata_resultTupleScheme extends TupleScheme<clear_file_metadata_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, clear_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -179581,11 +180448,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, clear_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new PutFileMetadataResult();
+          struct.success = new ClearFileMetadataResult();
           struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
@@ -179594,18 +180461,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class clear_file_metadata_args implements org.apache.thrift.TBase<clear_file_metadata_args, clear_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<clear_file_metadata_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("clear_file_metadata_args");
+  public static class cache_file_metadata_args implements org.apache.thrift.TBase<cache_file_metadata_args, cache_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<cache_file_metadata_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cache_file_metadata_args");
 
     private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new clear_file_metadata_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new clear_file_metadata_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new cache_file_metadata_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new cache_file_metadata_argsTupleSchemeFactory());
     }
 
-    private ClearFileMetadataRequest req; // required
+    private CacheFileMetadataRequest req; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -179670,16 +180537,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClearFileMetadataRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CacheFileMetadataRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(clear_file_metadata_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cache_file_metadata_args.class, metaDataMap);
     }
 
-    public clear_file_metadata_args() {
+    public cache_file_metadata_args() {
     }
 
-    public clear_file_metadata_args(
-      ClearFileMetadataRequest req)
+    public cache_file_metadata_args(
+      CacheFileMetadataRequest req)
     {
       this();
       this.req = req;
@@ -179688,14 +180555,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public clear_file_metadata_args(clear_file_metadata_args other) {
+    public cache_file_metadata_args(cache_file_metadata_args other) {
       if (other.isSetReq()) {
-        this.req = new ClearFileMetadataRequest(other.req);
+        this.req = new CacheFileMetadataRequest(other.req);
       }
     }
 
-    public clear_fil

<TRUNCATED>