http://git-wip-us.apache.org/repos/asf/hive/blob/7584b327/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java 
b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index 8584250..cf3edbf 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -56,6 +56,7 @@ import org.apache.curator.framework.api.ACLProvider;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
+import org.apache.curator.framework.recipes.leader.LeaderLatch;
 import org.apache.curator.framework.recipes.leader.LeaderLatchListener;
 import org.apache.curator.framework.recipes.nodes.PersistentEphemeralNode;
 import org.apache.curator.retry.ExponentialBackoffRetry;
@@ -82,6 +83,8 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveMaterializedViewsRegistry;
 import org.apache.hadoop.hive.ql.metadata.events.NotificationEventPoll;
 import org.apache.hadoop.hive.ql.plan.mapper.StatsSources;
+import org.apache.hadoop.hive.ql.security.authorization.PrivilegeSynchonizer;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
 import org.apache.hadoop.hive.ql.session.ClearDanglingScratchDir;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper;
@@ -137,6 +140,7 @@ public class HiveServer2 extends CompositeService {
   private ThriftCLIService thriftCLIService;
   private PersistentEphemeralNode znode;
   private CuratorFramework zooKeeperClient;
+  private CuratorFramework zKClientForPrivSync = null;
   private boolean deregisteredWithZooKeeper = false; // Set to true only when 
deregistration happens
   private HttpServer webServer; // Web UI
   private TezSessionPoolManager tezSessionPoolManager;
@@ -455,17 +459,9 @@ public class HiveServer2 extends CompositeService {
     }
   };
 
-  /**
-   * Adds a server instance to ZooKeeper as a znode.
-   *
-   * @param hiveConf
-   * @throws Exception
-   */
-  private void addServerInstanceToZooKeeper(HiveConf hiveConf, Map<String, 
String> confsToPublish) throws Exception {
-    String zooKeeperEnsemble = ZooKeeperHiveHelper.getQuorumServers(hiveConf);
-    String rootNamespace = 
hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE);
-    String instanceURI = getServerInstanceURI();
+  private CuratorFramework startZookeeperClient(HiveConf hiveConf) throws 
Exception {
     setUpZooKeeperAuth(hiveConf);
+    String zooKeeperEnsemble = ZooKeeperHiveHelper.getQuorumServers(hiveConf);
     int sessionTimeout =
         (int) 
hiveConf.getTimeVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT,
             TimeUnit.MILLISECONDS);
@@ -475,14 +471,16 @@ public class HiveServer2 extends CompositeService {
     int maxRetries = 
hiveConf.getIntVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_CONNECTION_MAX_RETRIES);
     // Create a CuratorFramework instance to be used as the ZooKeeper client
     // Use the zooKeeperAclProvider to create appropriate ACLs
-    zooKeeperClient =
+    CuratorFramework zkClient =
         CuratorFrameworkFactory.builder().connectString(zooKeeperEnsemble)
             .sessionTimeoutMs(sessionTimeout).aclProvider(zooKeeperAclProvider)
             .retryPolicy(new ExponentialBackoffRetry(baseSleepTime, 
maxRetries)).build();
-    zooKeeperClient.start();
+    zkClient.start();
+
     // Create the parent znodes recursively; ignore if the parent already 
exists.
+    String rootNamespace = 
hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE);
     try {
-      
zooKeeperClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
+      
zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
           .forPath(ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + 
rootNamespace);
       LOG.info("Created the root name space: " + rootNamespace + " on 
ZooKeeper for HiveServer2");
     } catch (KeeperException e) {
@@ -491,6 +489,20 @@ public class HiveServer2 extends CompositeService {
         throw e;
       }
     }
+    return zkClient;
+  }
+
+  /**
+   * Adds a server instance to ZooKeeper as a znode.
+   *
+   * @param hiveConf
+   * @throws Exception
+   */
+  private void addServerInstanceToZooKeeper(HiveConf hiveConf, Map<String, 
String> confsToPublish) throws Exception {
+    zooKeeperClient = startZookeeperClient(hiveConf);
+    String rootNamespace = 
hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE);
+    String instanceURI = getServerInstanceURI();
+
     // Create a znode under the rootNamespace parent for this instance of the 
server
     // Znode name: 
serverUri=host:port;version=versionInfo;sequence=sequenceNumber
     try {
@@ -703,6 +715,14 @@ public class HiveServer2 extends CompositeService {
         throw new ServiceException(e);
       }
     }
+
+    try {
+      startPrivilegeSynchonizer(hiveConf);
+    } catch (Exception e) {
+      LOG.error("Error starting priviledge synchonizer: ", e);
+      throw new ServiceException(e);
+    }
+
     if (webServer != null) {
       try {
         webServer.start();
@@ -913,6 +933,10 @@ public class HiveServer2 extends CompositeService {
         LOG.error("Spark session pool manager failed to stop during 
HiveServer2 shutdown.", ex);
       }
     }
+
+    if (zKClientForPrivSync != null) {
+      zKClientForPrivSync.close();
+    }
   }
 
   private void shutdownExecutor(final ExecutorService 
leaderActionsExecutorService) {
@@ -946,6 +970,27 @@ public class HiveServer2 extends CompositeService {
     }
   }
 
+  public void startPrivilegeSynchonizer(HiveConf hiveConf) throws Exception {
+    if (hiveConf.getBoolVar(ConfVars.HIVE_PRIVILEGE_SYNCHRONIZER)) {
+      zKClientForPrivSync = startZookeeperClient(hiveConf);
+      String rootNamespace = 
hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE);
+      String path = ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + 
rootNamespace
+          + ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + "leader";
+      LeaderLatch privilegeSynchonizerLatch = new 
LeaderLatch(zKClientForPrivSync, path);
+      privilegeSynchonizerLatch.start();
+      HiveAuthorizer authorizer = SessionState.get().getAuthorizerV2();
+      if (authorizer.getHivePolicyProvider() == null) {
+        LOG.warn(
+            "Cannot start PrivilegeSynchonizer, policyProvider of " + 
authorizer.getClass().getName() + " is null");
+        privilegeSynchonizerLatch.close();
+        return;
+      }
+      Thread privilegeSynchonizerThread = new Thread(
+          new PrivilegeSynchonizer(privilegeSynchonizerLatch, authorizer, 
hiveConf), "PrivilegeSynchonizer");
+      privilegeSynchonizerThread.start();
+    }
+  }
+
   private static void startHiveServer2() throws Throwable {
     long attempts = 0, maxAttempts = 1;
     while (true) {

http://git-wip-us.apache.org/repos/asf/hive/blob/7584b327/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp 
b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 3cbabf9..a25ebe5 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -33225,6 +33225,229 @@ uint32_t 
ThriftHiveMetastore_grant_revoke_privileges_presult::read(::apache::thr
 }
 
 
+ThriftHiveMetastore_refresh_privileges_args::~ThriftHiveMetastore_refresh_privileges_args()
 throw() {
+}
+
+
+uint32_t 
ThriftHiveMetastore_refresh_privileges_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;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->objToRefresh.read(iprot);
+          this->__isset.objToRefresh = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->grantRequest.read(iprot);
+          this->__isset.grantRequest = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t 
ThriftHiveMetastore_refresh_privileges_args::write(::apache::thrift::protocol::TProtocol*
 oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += 
oprot->writeStructBegin("ThriftHiveMetastore_refresh_privileges_args");
+
+  xfer += oprot->writeFieldBegin("objToRefresh", 
::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->objToRefresh.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantRequest", 
::apache::thrift::protocol::T_STRUCT, 2);
+  xfer += this->grantRequest.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_refresh_privileges_pargs::~ThriftHiveMetastore_refresh_privileges_pargs()
 throw() {
+}
+
+
+uint32_t 
ThriftHiveMetastore_refresh_privileges_pargs::write(::apache::thrift::protocol::TProtocol*
 oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += 
oprot->writeStructBegin("ThriftHiveMetastore_refresh_privileges_pargs");
+
+  xfer += oprot->writeFieldBegin("objToRefresh", 
::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->objToRefresh)).write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("grantRequest", 
::apache::thrift::protocol::T_STRUCT, 2);
+  xfer += (*(this->grantRequest)).write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_refresh_privileges_result::~ThriftHiveMetastore_refresh_privileges_result()
 throw() {
+}
+
+
+uint32_t 
ThriftHiveMetastore_refresh_privileges_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_STRUCT) {
+          xfer += this->success.read(iprot);
+          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_refresh_privileges_result::write(::apache::thrift::protocol::TProtocol*
 oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += 
oprot->writeStructBegin("ThriftHiveMetastore_refresh_privileges_result");
+
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", 
::apache::thrift::protocol::T_STRUCT, 0);
+    xfer += this->success.write(oprot);
+    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_refresh_privileges_presult::~ThriftHiveMetastore_refresh_privileges_presult()
 throw() {
+}
+
+
+uint32_t 
ThriftHiveMetastore_refresh_privileges_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_STRUCT) {
+          xfer += (*(this->success)).read(iprot);
+          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;
+}
+
+
 ThriftHiveMetastore_set_ugi_args::~ThriftHiveMetastore_set_ugi_args() throw() {
 }
 
@@ -57981,6 +58204,68 @@ void 
ThriftHiveMetastoreClient::recv_grant_revoke_privileges(GrantRevokePrivileg
   throw 
::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT,
 "grant_revoke_privileges failed: unknown result");
 }
 
+void 
ThriftHiveMetastoreClient::refresh_privileges(GrantRevokePrivilegeResponse& 
_return, const HiveObjectRef& objToRefresh, const GrantRevokePrivilegeRequest& 
grantRequest)
+{
+  send_refresh_privileges(objToRefresh, grantRequest);
+  recv_refresh_privileges(_return);
+}
+
+void ThriftHiveMetastoreClient::send_refresh_privileges(const HiveObjectRef& 
objToRefresh, const GrantRevokePrivilegeRequest& grantRequest)
+{
+  int32_t cseqid = 0;
+  oprot_->writeMessageBegin("refresh_privileges", 
::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_refresh_privileges_pargs args;
+  args.objToRefresh = &objToRefresh;
+  args.grantRequest = &grantRequest;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+}
+
+void 
ThriftHiveMetastoreClient::recv_refresh_privileges(GrantRevokePrivilegeResponse&
 _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("refresh_privileges") != 0) {
+    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+  }
+  ThriftHiveMetastore_refresh_privileges_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,
 "refresh_privileges failed: unknown result");
+}
+
 void ThriftHiveMetastoreClient::set_ugi(std::vector<std::string> & _return, 
const std::string& user_name, const std::vector<std::string> & group_names)
 {
   send_set_ugi(user_name, group_names);
@@ -70406,6 +70691,63 @@ void 
ThriftHiveMetastoreProcessor::process_grant_revoke_privileges(int32_t seqid
   }
 }
 
+void ThriftHiveMetastoreProcessor::process_refresh_privileges(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.refresh_privileges", 
callContext);
+  }
+  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), 
ctx, "ThriftHiveMetastore.refresh_privileges");
+
+  if (this->eventHandler_.get() != NULL) {
+    this->eventHandler_->preRead(ctx, 
"ThriftHiveMetastore.refresh_privileges");
+  }
+
+  ThriftHiveMetastore_refresh_privileges_args args;
+  args.read(iprot);
+  iprot->readMessageEnd();
+  uint32_t bytes = iprot->getTransport()->readEnd();
+
+  if (this->eventHandler_.get() != NULL) {
+    this->eventHandler_->postRead(ctx, 
"ThriftHiveMetastore.refresh_privileges", bytes);
+  }
+
+  ThriftHiveMetastore_refresh_privileges_result result;
+  try {
+    iface_->refresh_privileges(result.success, args.objToRefresh, 
args.grantRequest);
+    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.refresh_privileges");
+    }
+
+    ::apache::thrift::TApplicationException x(e.what());
+    oprot->writeMessageBegin("refresh_privileges", 
::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.refresh_privileges");
+  }
+
+  oprot->writeMessageBegin("refresh_privileges", 
::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.refresh_privileges", bytes);
+  }
+}
+
 void ThriftHiveMetastoreProcessor::process_set_ugi(int32_t seqid, 
::apache::thrift::protocol::TProtocol* iprot, 
::apache::thrift::protocol::TProtocol* oprot, void* callContext)
 {
   void* ctx = NULL;
@@ -86682,6 +87024,95 @@ void 
ThriftHiveMetastoreConcurrentClient::recv_grant_revoke_privileges(GrantRevo
   } // end while(true)
 }
 
+void 
ThriftHiveMetastoreConcurrentClient::refresh_privileges(GrantRevokePrivilegeResponse&
 _return, const HiveObjectRef& objToRefresh, const GrantRevokePrivilegeRequest& 
grantRequest)
+{
+  int32_t seqid = send_refresh_privileges(objToRefresh, grantRequest);
+  recv_refresh_privileges(_return, seqid);
+}
+
+int32_t ThriftHiveMetastoreConcurrentClient::send_refresh_privileges(const 
HiveObjectRef& objToRefresh, const GrantRevokePrivilegeRequest& grantRequest)
+{
+  int32_t cseqid = this->sync_.generateSeqId();
+  ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_);
+  oprot_->writeMessageBegin("refresh_privileges", 
::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_refresh_privileges_pargs args;
+  args.objToRefresh = &objToRefresh;
+  args.grantRequest = &grantRequest;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+
+  sentry.commit();
+  return cseqid;
+}
+
+void 
ThriftHiveMetastoreConcurrentClient::recv_refresh_privileges(GrantRevokePrivilegeResponse&
 _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("refresh_privileges") != 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_refresh_privileges_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,
 "refresh_privileges 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)
+}
+
 void ThriftHiveMetastoreConcurrentClient::set_ugi(std::vector<std::string> & 
_return, const std::string& user_name, const std::vector<std::string> & 
group_names)
 {
   int32_t seqid = send_set_ugi(user_name, group_names);

http://git-wip-us.apache.org/repos/asf/hive/blob/7584b327/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h 
b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 2056730..dac6983 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -149,6 +149,7 @@ class ThriftHiveMetastoreIf : virtual public  
::facebook::fb303::FacebookService
   virtual bool grant_privileges(const PrivilegeBag& privileges) = 0;
   virtual bool revoke_privileges(const PrivilegeBag& privileges) = 0;
   virtual void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, 
const GrantRevokePrivilegeRequest& request) = 0;
+  virtual void refresh_privileges(GrantRevokePrivilegeResponse& _return, const 
HiveObjectRef& objToRefresh, const GrantRevokePrivilegeRequest& grantRequest) = 
0;
   virtual void set_ugi(std::vector<std::string> & _return, const std::string& 
user_name, const std::vector<std::string> & group_names) = 0;
   virtual void get_delegation_token(std::string& _return, const std::string& 
token_owner, const std::string& renewer_kerberos_principal_name) = 0;
   virtual int64_t renew_delegation_token(const std::string& token_str_form) = 
0;
@@ -659,6 +660,9 @@ class ThriftHiveMetastoreNull : virtual public 
ThriftHiveMetastoreIf , virtual p
   void grant_revoke_privileges(GrantRevokePrivilegeResponse& /* _return */, 
const GrantRevokePrivilegeRequest& /* request */) {
     return;
   }
+  void refresh_privileges(GrantRevokePrivilegeResponse& /* _return */, const 
HiveObjectRef& /* objToRefresh */, const GrantRevokePrivilegeRequest& /* 
grantRequest */) {
+    return;
+  }
   void set_ugi(std::vector<std::string> & /* _return */, const std::string& /* 
user_name */, const std::vector<std::string> & /* group_names */) {
     return;
   }
@@ -17168,6 +17172,125 @@ class 
ThriftHiveMetastore_grant_revoke_privileges_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_refresh_privileges_args__isset {
+  _ThriftHiveMetastore_refresh_privileges_args__isset() : objToRefresh(false), 
grantRequest(false) {}
+  bool objToRefresh :1;
+  bool grantRequest :1;
+} _ThriftHiveMetastore_refresh_privileges_args__isset;
+
+class ThriftHiveMetastore_refresh_privileges_args {
+ public:
+
+  ThriftHiveMetastore_refresh_privileges_args(const 
ThriftHiveMetastore_refresh_privileges_args&);
+  ThriftHiveMetastore_refresh_privileges_args& operator=(const 
ThriftHiveMetastore_refresh_privileges_args&);
+  ThriftHiveMetastore_refresh_privileges_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_refresh_privileges_args() throw();
+  HiveObjectRef objToRefresh;
+  GrantRevokePrivilegeRequest grantRequest;
+
+  _ThriftHiveMetastore_refresh_privileges_args__isset __isset;
+
+  void __set_objToRefresh(const HiveObjectRef& val);
+
+  void __set_grantRequest(const GrantRevokePrivilegeRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_refresh_privileges_args & rhs) 
const
+  {
+    if (!(objToRefresh == rhs.objToRefresh))
+      return false;
+    if (!(grantRequest == rhs.grantRequest))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_refresh_privileges_args &rhs) 
const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_refresh_privileges_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_refresh_privileges_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_refresh_privileges_pargs() throw();
+  const HiveObjectRef* objToRefresh;
+  const GrantRevokePrivilegeRequest* grantRequest;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_refresh_privileges_result__isset {
+  _ThriftHiveMetastore_refresh_privileges_result__isset() : success(false), 
o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_refresh_privileges_result__isset;
+
+class ThriftHiveMetastore_refresh_privileges_result {
+ public:
+
+  ThriftHiveMetastore_refresh_privileges_result(const 
ThriftHiveMetastore_refresh_privileges_result&);
+  ThriftHiveMetastore_refresh_privileges_result& operator=(const 
ThriftHiveMetastore_refresh_privileges_result&);
+  ThriftHiveMetastore_refresh_privileges_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_refresh_privileges_result() throw();
+  GrantRevokePrivilegeResponse success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_refresh_privileges_result__isset __isset;
+
+  void __set_success(const GrantRevokePrivilegeResponse& val);
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_refresh_privileges_result & rhs) 
const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_refresh_privileges_result &rhs) 
const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_refresh_privileges_result & ) 
const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_refresh_privileges_presult__isset {
+  _ThriftHiveMetastore_refresh_privileges_presult__isset() : success(false), 
o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_refresh_privileges_presult__isset;
+
+class ThriftHiveMetastore_refresh_privileges_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_refresh_privileges_presult() throw();
+  GrantRevokePrivilegeResponse* success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_refresh_privileges_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_set_ugi_args__isset {
   _ThriftHiveMetastore_set_ugi_args__isset() : user_name(false), 
group_names(false) {}
   bool user_name :1;
@@ -26350,6 +26473,9 @@ class ThriftHiveMetastoreClient : virtual public 
ThriftHiveMetastoreIf, public
   void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const 
GrantRevokePrivilegeRequest& request);
   void send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& 
request);
   void recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return);
+  void refresh_privileges(GrantRevokePrivilegeResponse& _return, const 
HiveObjectRef& objToRefresh, const GrantRevokePrivilegeRequest& grantRequest);
+  void send_refresh_privileges(const HiveObjectRef& objToRefresh, const 
GrantRevokePrivilegeRequest& grantRequest);
+  void recv_refresh_privileges(GrantRevokePrivilegeResponse& _return);
   void set_ugi(std::vector<std::string> & _return, const std::string& 
user_name, const std::vector<std::string> & group_names);
   void send_set_ugi(const std::string& user_name, const 
std::vector<std::string> & group_names);
   void recv_set_ugi(std::vector<std::string> & _return);
@@ -26721,6 +26847,7 @@ class ThriftHiveMetastoreProcessor : public  
::facebook::fb303::FacebookServiceP
   void process_grant_privileges(int32_t seqid, 
::apache::thrift::protocol::TProtocol* iprot, 
::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_revoke_privileges(int32_t seqid, 
::apache::thrift::protocol::TProtocol* iprot, 
::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_grant_revoke_privileges(int32_t seqid, 
::apache::thrift::protocol::TProtocol* iprot, 
::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_refresh_privileges(int32_t seqid, 
::apache::thrift::protocol::TProtocol* iprot, 
::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_set_ugi(int32_t seqid, ::apache::thrift::protocol::TProtocol* 
iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_delegation_token(int32_t seqid, 
::apache::thrift::protocol::TProtocol* iprot, 
::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_renew_delegation_token(int32_t seqid, 
::apache::thrift::protocol::TProtocol* iprot, 
::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -26930,6 +27057,7 @@ class ThriftHiveMetastoreProcessor : public  
::facebook::fb303::FacebookServiceP
     processMap_["grant_privileges"] = 
&ThriftHiveMetastoreProcessor::process_grant_privileges;
     processMap_["revoke_privileges"] = 
&ThriftHiveMetastoreProcessor::process_revoke_privileges;
     processMap_["grant_revoke_privileges"] = 
&ThriftHiveMetastoreProcessor::process_grant_revoke_privileges;
+    processMap_["refresh_privileges"] = 
&ThriftHiveMetastoreProcessor::process_refresh_privileges;
     processMap_["set_ugi"] = &ThriftHiveMetastoreProcessor::process_set_ugi;
     processMap_["get_delegation_token"] = 
&ThriftHiveMetastoreProcessor::process_get_delegation_token;
     processMap_["renew_delegation_token"] = 
&ThriftHiveMetastoreProcessor::process_renew_delegation_token;
@@ -28257,6 +28385,16 @@ class ThriftHiveMetastoreMultiface : virtual public 
ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void refresh_privileges(GrantRevokePrivilegeResponse& _return, const 
HiveObjectRef& objToRefresh, const GrantRevokePrivilegeRequest& grantRequest) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->refresh_privileges(_return, objToRefresh, grantRequest);
+    }
+    ifaces_[i]->refresh_privileges(_return, objToRefresh, grantRequest);
+    return;
+  }
+
   void set_ugi(std::vector<std::string> & _return, const std::string& 
user_name, const std::vector<std::string> & group_names) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -29408,6 +29546,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual 
public ThriftHiveMetastoreIf
   void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const 
GrantRevokePrivilegeRequest& request);
   int32_t send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& 
request);
   void recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, 
const int32_t seqid);
+  void refresh_privileges(GrantRevokePrivilegeResponse& _return, const 
HiveObjectRef& objToRefresh, const GrantRevokePrivilegeRequest& grantRequest);
+  int32_t send_refresh_privileges(const HiveObjectRef& objToRefresh, const 
GrantRevokePrivilegeRequest& grantRequest);
+  void recv_refresh_privileges(GrantRevokePrivilegeResponse& _return, const 
int32_t seqid);
   void set_ugi(std::vector<std::string> & _return, const std::string& 
user_name, const std::vector<std::string> & group_names);
   int32_t send_set_ugi(const std::string& user_name, const 
std::vector<std::string> & group_names);
   void recv_set_ugi(std::vector<std::string> & _return, const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/7584b327/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git 
a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
 
b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index c1c0b77..c4a8baf 100644
--- 
a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ 
b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -657,6 +657,11 @@ class ThriftHiveMetastoreHandler : virtual public 
ThriftHiveMetastoreIf {
     printf("grant_revoke_privileges\n");
   }
 
+  void refresh_privileges(GrantRevokePrivilegeResponse& _return, const 
HiveObjectRef& objToRefresh, const GrantRevokePrivilegeRequest& grantRequest) {
+    // Your implementation goes here
+    printf("refresh_privileges\n");
+  }
+
   void set_ugi(std::vector<std::string> & _return, const std::string& 
user_name, const std::vector<std::string> & group_names) {
     // Your implementation goes here
     printf("set_ugi\n");

Reply via email to