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:49 UTC

[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)

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) {