You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sr...@apache.org on 2014/06/23 21:50:28 UTC

[3/3] git commit: SENTRY-162: Cleanup DB store privilege metadata on Hive DDL statements ( Prasad Mujumdar via Sravya Tirukkovalur)

SENTRY-162: Cleanup DB store privilege metadata on Hive DDL statements ( Prasad Mujumdar via Sravya Tirukkovalur)


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

Branch: refs/heads/master
Commit: b11f5aab9970b56886c8c9a04f7598ba3b50879c
Parents: 5988495
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Mon Jun 23 12:21:36 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Mon Jun 23 12:21:36 2014 -0700

----------------------------------------------------------------------
 .../sentry/binding/hive/conf/HiveAuthzConf.java |    3 +
 .../SentryMetastorePostEventListener.java       |  206 +++
 .../db/service/thrift/SentryPolicyService.java  | 1612 ++++++++++++++++++
 .../service/thrift/TDropPrivilegesRequest.java  |  592 +++++++
 .../service/thrift/TDropPrivilegesResponse.java |  390 +++++
 .../thrift/TRenamePrivilegesRequest.java        |  698 ++++++++
 .../thrift/TRenamePrivilegesResponse.java       |  390 +++++
 .../db/service/model/MSentryPrivilege.java      |    4 +
 .../db/service/persistent/SentryStore.java      |  316 +++-
 .../thrift/SentryPolicyServiceClient.java       |   87 +-
 .../thrift/SentryPolicyStoreProcessor.java      |   42 +
 .../main/resources/sentry_policy_service.thrift |   26 +
 .../db/service/persistent/TestSentryStore.java  |  189 ++
 .../TestDbPrivilegeCleanupOnDrop.java           |  292 ++++
 .../AbstractTestWithStaticConfiguration.java    |   15 +-
 .../e2e/hive/hiveserver/HiveServerFactory.java  |    1 +
 16 files changed, 4757 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b11f5aab/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
index eb9ef00..1bb4a99 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
@@ -64,6 +64,9 @@ public class HiveAuthzConf extends Configuration {
     AUTHZ_ALLOW_HIVE_IMPERSONATION("sentry.hive.allow.hive.impersonation", "false"),
     AUTHZ_ONFAILURE_HOOKS("sentry.hive.failure.hooks", ""),
     AUTHZ_METASTORE_SERVICE_USERS("sentry.metastore.service.users", ""),
+    AUTHZ_SYNC_ALTER_WITH_POLICY_STORE("sentry.hive.sync.alter", "true"),
+    AUTHZ_SYNC_CREATE_WITH_POLICY_STORE("sentry.hive.sync.create", "false"),
+    AUTHZ_SYNC_DROP_WITH_POLICY_STORE("sentry.hive.sync.drop", "true"),
 
     AUTHZ_PROVIDER_DEPRECATED("hive.sentry.provider",
       "org.apache.sentry.provider.file.ResourceAuthorizationProvider"),

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b11f5aab/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java
new file mode 100644
index 0000000..5634879
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListener.java
@@ -0,0 +1,206 @@
+/**
+ * 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.sentry.binding.metastore;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropTableEvent;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.sentry.SentryUserException;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
+import org.apache.sentry.core.common.Authorizable;
+import org.apache.sentry.core.model.db.Database;
+import org.apache.sentry.core.model.db.Server;
+import org.apache.sentry.core.model.db.Table;
+import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
+import org.apache.sentry.service.thrift.SentryServiceClientFactory;
+
+public class SentryMetastorePostEventListener extends MetaStoreEventListener {
+  private final SentryServiceClientFactory sentryClientFactory;
+  private final HiveAuthzConf authzConf;
+  private final Server server;
+
+  public SentryMetastorePostEventListener(Configuration config) {
+    super(config);
+    sentryClientFactory = new SentryServiceClientFactory();
+    authzConf = HiveAuthzConf.getAuthzConf(new HiveConf());
+    server = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()));
+  }
+
+  @Override
+  public void onCreateTable (CreateTableEvent tableEvent) throws MetaException {
+    // drop the privileges on the given table, in case if anything was left
+    // behind during the drop
+    if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE)) {
+      return;
+    }
+    dropSentryTablePrivilege(tableEvent.getTable().getDbName(),
+        tableEvent.getTable().getTableName());
+  }
+
+  @Override
+  public void onDropTable(DropTableEvent tableEvent) throws MetaException {
+    // drop the privileges on the given table
+    if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE)) {
+      return;
+    }
+    dropSentryTablePrivilege(tableEvent.getTable().getDbName(),
+        tableEvent.getTable().getTableName());
+  }
+
+  @Override
+  public void onCreateDatabase(CreateDatabaseEvent dbEvent)
+      throws MetaException {
+    // drop the privileges on the database, incase anything left behind during
+    // last drop db
+    if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE)) {
+      return;
+    }
+    dropSentryDbPrivileges(dbEvent.getDatabase().getName());
+  }
+
+  /**
+   * Drop the privileges on the database // note that child tables will be
+   * dropped individually by client, so we // just need to handle the removing
+   * the db privileges. The table drop // should cleanup the table privileges
+   */
+  @Override
+  public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException {
+    if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE)) {
+      return;
+    }
+    dropSentryDbPrivileges(dbEvent.getDatabase().getName());
+  }
+
+  /**
+   * Adjust the privileges when table is renamed
+   */
+  @Override
+  public void onAlterTable (AlterTableEvent tableEvent) throws MetaException {
+    String oldTableName = null, newTableName = null;
+    if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_ALTER_WITH_POLICY_STORE)) {
+      return;
+    }
+    if (tableEvent.getOldTable() != null) {
+      oldTableName = tableEvent.getOldTable().getTableName();
+    }
+    if (tableEvent.getNewTable() != null) {
+      newTableName = tableEvent.getNewTable().getTableName();
+    }
+    if (!oldTableName.equalsIgnoreCase(newTableName)) {
+      renameSentryTablePrivilege(tableEvent.getOldTable().getDbName(),
+          oldTableName, tableEvent.getNewTable().getDbName(), newTableName);
+    }
+  }
+
+  private SentryPolicyServiceClient getSentryServiceClient()
+      throws MetaException {
+    try {
+      return sentryClientFactory.create(getConf());
+    } catch (Exception e) {
+      throw new MetaException("Failed to connect to Sentry service "
+          + e.getMessage());
+    }
+  }
+
+  private void dropSentryDbPrivileges(String dbName) throws MetaException {
+    List<Authorizable> authorizableTable = new ArrayList<Authorizable>();
+    authorizableTable.add(server);
+    authorizableTable.add(new Database(dbName));
+    try {
+      dropSentryPrivileges(authorizableTable);
+    } catch (SentryUserException e) {
+      throw new MetaException("Failed to remove Sentry policies for drop DB "
+          + dbName + " Error: " + e.getMessage());
+    } catch (IOException e) {
+      throw new MetaException("Failed to find local user " + e.getMessage());
+    }
+
+  }
+
+  private void dropSentryTablePrivilege(String dbName, String tabName)
+      throws MetaException {
+    List<Authorizable> authorizableTable = new ArrayList<Authorizable>();
+    authorizableTable.add(server);
+    authorizableTable.add(new Database(dbName));
+    authorizableTable.add(new Table(tabName));
+
+    try {
+      dropSentryPrivileges(authorizableTable);
+    } catch (SentryUserException e) {
+      throw new MetaException(
+          "Failed to remove Sentry policies for drop table " + dbName + "."
+              + tabName + " Error: " + e.getMessage());
+    } catch (IOException e) {
+      throw new MetaException("Failed to find local user " + e.getMessage());
+    }
+
+  }
+  private void dropSentryPrivileges(
+      List<? extends Authorizable> authorizableTable)
+      throws SentryUserException, IOException, MetaException {
+    String requestorUserName = UserGroupInformation.getCurrentUser()
+        .getShortUserName();
+    SentryPolicyServiceClient sentryClient = getSentryServiceClient();
+    sentryClient.dropPrivileges(requestorUserName, authorizableTable);
+  }
+
+  private void renameSentryTablePrivilege(String oldDbName, String oldTabName,
+      String newDbName, String newTabName)
+      throws MetaException {
+    List<Authorizable> oldAuthorizableTable = new ArrayList<Authorizable>();
+    oldAuthorizableTable.add(server);
+    oldAuthorizableTable.add(new Database(oldDbName));
+    oldAuthorizableTable.add(new Table(oldTabName));
+
+    List<Authorizable> newAuthorizableTable = new ArrayList<Authorizable>();
+    newAuthorizableTable.add(server);
+    newAuthorizableTable.add(new Database(newDbName));
+    newAuthorizableTable.add(new Table(newTabName));
+
+    try {
+      String requestorUserName = UserGroupInformation.getCurrentUser()
+          .getShortUserName();
+      SentryPolicyServiceClient sentryClient = getSentryServiceClient();
+      sentryClient.renamePrivileges(requestorUserName, oldAuthorizableTable, newAuthorizableTable);
+    } catch (SentryUserException e) {
+      throw new MetaException(
+          "Failed to remove Sentry policies for rename table " + oldDbName
+              + "." + oldTabName + "to " + newDbName + "." + newTabName
+              + " Error: " + e.getMessage());
+    } catch (IOException e) {
+      throw new MetaException("Failed to find local user " + e.getMessage());
+    }
+  }
+
+  private boolean syncWithPolicyStore(AuthzConfVars syncConfVar) {
+    return "true"
+        .equalsIgnoreCase((authzConf.get(syncConfVar.getVar(), "true")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b11f5aab/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
index ff09fdf..8a006aa 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
@@ -53,6 +53,10 @@ public class SentryPolicyService {
 
     public TListSentryPrivilegesForProviderResponse list_sentry_privileges_for_provider(TListSentryPrivilegesForProviderRequest request) throws org.apache.thrift.TException;
 
+    public TDropPrivilegesResponse drop_sentry_privilege(TDropPrivilegesRequest request) throws org.apache.thrift.TException;
+
+    public TRenamePrivilegesResponse rename_sentry_privilege(TRenamePrivilegesRequest request) throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -75,6 +79,10 @@ public class SentryPolicyService {
 
     public void list_sentry_privileges_for_provider(TListSentryPrivilegesForProviderRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.list_sentry_privileges_for_provider_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void drop_sentry_privilege(TDropPrivilegesRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_sentry_privilege_call> resultHandler) throws org.apache.thrift.TException;
+
+    public void rename_sentry_privilege(TRenamePrivilegesRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.rename_sentry_privilege_call> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -304,6 +312,52 @@ public class SentryPolicyService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "list_sentry_privileges_for_provider failed: unknown result");
     }
 
+    public TDropPrivilegesResponse drop_sentry_privilege(TDropPrivilegesRequest request) throws org.apache.thrift.TException
+    {
+      send_drop_sentry_privilege(request);
+      return recv_drop_sentry_privilege();
+    }
+
+    public void send_drop_sentry_privilege(TDropPrivilegesRequest request) throws org.apache.thrift.TException
+    {
+      drop_sentry_privilege_args args = new drop_sentry_privilege_args();
+      args.setRequest(request);
+      sendBase("drop_sentry_privilege", args);
+    }
+
+    public TDropPrivilegesResponse recv_drop_sentry_privilege() throws org.apache.thrift.TException
+    {
+      drop_sentry_privilege_result result = new drop_sentry_privilege_result();
+      receiveBase(result, "drop_sentry_privilege");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "drop_sentry_privilege failed: unknown result");
+    }
+
+    public TRenamePrivilegesResponse rename_sentry_privilege(TRenamePrivilegesRequest request) throws org.apache.thrift.TException
+    {
+      send_rename_sentry_privilege(request);
+      return recv_rename_sentry_privilege();
+    }
+
+    public void send_rename_sentry_privilege(TRenamePrivilegesRequest request) throws org.apache.thrift.TException
+    {
+      rename_sentry_privilege_args args = new rename_sentry_privilege_args();
+      args.setRequest(request);
+      sendBase("rename_sentry_privilege", args);
+    }
+
+    public TRenamePrivilegesResponse recv_rename_sentry_privilege() throws org.apache.thrift.TException
+    {
+      rename_sentry_privilege_result result = new rename_sentry_privilege_result();
+      receiveBase(result, "rename_sentry_privilege");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "rename_sentry_privilege failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -610,6 +664,70 @@ public class SentryPolicyService {
       }
     }
 
+    public void drop_sentry_privilege(TDropPrivilegesRequest request, org.apache.thrift.async.AsyncMethodCallback<drop_sentry_privilege_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      drop_sentry_privilege_call method_call = new drop_sentry_privilege_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class drop_sentry_privilege_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TDropPrivilegesRequest request;
+      public drop_sentry_privilege_call(TDropPrivilegesRequest request, org.apache.thrift.async.AsyncMethodCallback<drop_sentry_privilege_call> 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);
+        this.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("drop_sentry_privilege", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        drop_sentry_privilege_args args = new drop_sentry_privilege_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TDropPrivilegesResponse getResult() throws 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_drop_sentry_privilege();
+      }
+    }
+
+    public void rename_sentry_privilege(TRenamePrivilegesRequest request, org.apache.thrift.async.AsyncMethodCallback<rename_sentry_privilege_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      rename_sentry_privilege_call method_call = new rename_sentry_privilege_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class rename_sentry_privilege_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TRenamePrivilegesRequest request;
+      public rename_sentry_privilege_call(TRenamePrivilegesRequest request, org.apache.thrift.async.AsyncMethodCallback<rename_sentry_privilege_call> 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);
+        this.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("rename_sentry_privilege", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        rename_sentry_privilege_args args = new rename_sentry_privilege_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TRenamePrivilegesResponse getResult() throws 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_rename_sentry_privilege();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -632,6 +750,8 @@ public class SentryPolicyService {
       processMap.put("list_sentry_roles_by_group", new list_sentry_roles_by_group());
       processMap.put("list_sentry_privileges_by_role", new list_sentry_privileges_by_role());
       processMap.put("list_sentry_privileges_for_provider", new list_sentry_privileges_for_provider());
+      processMap.put("drop_sentry_privilege", new drop_sentry_privilege());
+      processMap.put("rename_sentry_privilege", new rename_sentry_privilege());
       return processMap;
     }
 
@@ -815,6 +935,46 @@ public class SentryPolicyService {
       }
     }
 
+    public static class drop_sentry_privilege<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_sentry_privilege_args> {
+      public drop_sentry_privilege() {
+        super("drop_sentry_privilege");
+      }
+
+      public drop_sentry_privilege_args getEmptyArgsInstance() {
+        return new drop_sentry_privilege_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public drop_sentry_privilege_result getResult(I iface, drop_sentry_privilege_args args) throws org.apache.thrift.TException {
+        drop_sentry_privilege_result result = new drop_sentry_privilege_result();
+        result.success = iface.drop_sentry_privilege(args.request);
+        return result;
+      }
+    }
+
+    public static class rename_sentry_privilege<I extends Iface> extends org.apache.thrift.ProcessFunction<I, rename_sentry_privilege_args> {
+      public rename_sentry_privilege() {
+        super("rename_sentry_privilege");
+      }
+
+      public rename_sentry_privilege_args getEmptyArgsInstance() {
+        return new rename_sentry_privilege_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public rename_sentry_privilege_result getResult(I iface, rename_sentry_privilege_args args) throws org.apache.thrift.TException {
+        rename_sentry_privilege_result result = new rename_sentry_privilege_result();
+        result.success = iface.rename_sentry_privilege(args.request);
+        return result;
+      }
+    }
+
   }
 
   public static class create_sentry_role_args implements org.apache.thrift.TBase<create_sentry_role_args, create_sentry_role_args._Fields>, java.io.Serializable, Cloneable   {
@@ -7351,4 +7511,1456 @@ public class SentryPolicyService {
 
   }
 
+  public static class drop_sentry_privilege_args implements org.apache.thrift.TBase<drop_sentry_privilege_args, drop_sentry_privilege_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drop_sentry_privilege_args");
+
+    private static final org.apache.thrift.protocol.TField REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", 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 drop_sentry_privilege_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new drop_sentry_privilege_argsTupleSchemeFactory());
+    }
+
+    private TDropPrivilegesRequest request; // 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 {
+      REQUEST((short)1, "request");
+
+      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: // REQUEST
+            return REQUEST;
+          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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDropPrivilegesRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(drop_sentry_privilege_args.class, metaDataMap);
+    }
+
+    public drop_sentry_privilege_args() {
+    }
+
+    public drop_sentry_privilege_args(
+      TDropPrivilegesRequest request)
+    {
+      this();
+      this.request = request;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public drop_sentry_privilege_args(drop_sentry_privilege_args other) {
+      if (other.isSetRequest()) {
+        this.request = new TDropPrivilegesRequest(other.request);
+      }
+    }
+
+    public drop_sentry_privilege_args deepCopy() {
+      return new drop_sentry_privilege_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.request = null;
+    }
+
+    public TDropPrivilegesRequest getRequest() {
+      return this.request;
+    }
+
+    public void setRequest(TDropPrivilegesRequest request) {
+      this.request = request;
+    }
+
+    public void unsetRequest() {
+      this.request = null;
+    }
+
+    /** Returns true if field request is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequest() {
+      return this.request != null;
+    }
+
+    public void setRequestIsSet(boolean value) {
+      if (!value) {
+        this.request = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case REQUEST:
+        if (value == null) {
+          unsetRequest();
+        } else {
+          setRequest((TDropPrivilegesRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case REQUEST:
+        return getRequest();
+
+      }
+      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 REQUEST:
+        return isSetRequest();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof drop_sentry_privilege_args)
+        return this.equals((drop_sentry_privilege_args)that);
+      return false;
+    }
+
+    public boolean equals(drop_sentry_privilege_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_request = true && this.isSetRequest();
+      boolean that_present_request = true && that.isSetRequest();
+      if (this_present_request || that_present_request) {
+        if (!(this_present_request && that_present_request))
+          return false;
+        if (!this.request.equals(that.request))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_request = true && (isSetRequest());
+      builder.append(present_request);
+      if (present_request)
+        builder.append(request);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(drop_sentry_privilege_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      drop_sentry_privilege_args typedOther = (drop_sentry_privilege_args)other;
+
+      lastComparison = Boolean.valueOf(isSetRequest()).compareTo(typedOther.isSetRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, typedOther.request);
+        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("drop_sentry_privilege_args(");
+      boolean first = true;
+
+      sb.append("request:");
+      if (this.request == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.request);
+      }
+      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 (request != null) {
+        request.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 drop_sentry_privilege_argsStandardSchemeFactory implements SchemeFactory {
+      public drop_sentry_privilege_argsStandardScheme getScheme() {
+        return new drop_sentry_privilege_argsStandardScheme();
+      }
+    }
+
+    private static class drop_sentry_privilege_argsStandardScheme extends StandardScheme<drop_sentry_privilege_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, drop_sentry_privilege_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: // REQUEST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.request = new TDropPrivilegesRequest();
+                struct.request.read(iprot);
+                struct.setRequestIsSet(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, drop_sentry_privilege_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.request != null) {
+          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
+          struct.request.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class drop_sentry_privilege_argsTupleSchemeFactory implements SchemeFactory {
+      public drop_sentry_privilege_argsTupleScheme getScheme() {
+        return new drop_sentry_privilege_argsTupleScheme();
+      }
+    }
+
+    private static class drop_sentry_privilege_argsTupleScheme extends TupleScheme<drop_sentry_privilege_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, drop_sentry_privilege_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetRequest()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetRequest()) {
+          struct.request.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, drop_sentry_privilege_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.request = new TDropPrivilegesRequest();
+          struct.request.read(iprot);
+          struct.setRequestIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class drop_sentry_privilege_result implements org.apache.thrift.TBase<drop_sentry_privilege_result, drop_sentry_privilege_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drop_sentry_privilege_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 drop_sentry_privilege_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new drop_sentry_privilege_resultTupleSchemeFactory());
+    }
+
+    private TDropPrivilegesResponse 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, TDropPrivilegesResponse.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(drop_sentry_privilege_result.class, metaDataMap);
+    }
+
+    public drop_sentry_privilege_result() {
+    }
+
+    public drop_sentry_privilege_result(
+      TDropPrivilegesResponse success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public drop_sentry_privilege_result(drop_sentry_privilege_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TDropPrivilegesResponse(other.success);
+      }
+    }
+
+    public drop_sentry_privilege_result deepCopy() {
+      return new drop_sentry_privilege_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public TDropPrivilegesResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(TDropPrivilegesResponse 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((TDropPrivilegesResponse)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 drop_sentry_privilege_result)
+        return this.equals((drop_sentry_privilege_result)that);
+      return false;
+    }
+
+    public boolean equals(drop_sentry_privilege_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() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (isSetSuccess());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(drop_sentry_privilege_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      drop_sentry_privilege_result typedOther = (drop_sentry_privilege_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.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("drop_sentry_privilege_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 drop_sentry_privilege_resultStandardSchemeFactory implements SchemeFactory {
+      public drop_sentry_privilege_resultStandardScheme getScheme() {
+        return new drop_sentry_privilege_resultStandardScheme();
+      }
+    }
+
+    private static class drop_sentry_privilege_resultStandardScheme extends StandardScheme<drop_sentry_privilege_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, drop_sentry_privilege_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 TDropPrivilegesResponse();
+                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, drop_sentry_privilege_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 drop_sentry_privilege_resultTupleSchemeFactory implements SchemeFactory {
+      public drop_sentry_privilege_resultTupleScheme getScheme() {
+        return new drop_sentry_privilege_resultTupleScheme();
+      }
+    }
+
+    private static class drop_sentry_privilege_resultTupleScheme extends TupleScheme<drop_sentry_privilege_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, drop_sentry_privilege_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, drop_sentry_privilege_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new TDropPrivilegesResponse();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class rename_sentry_privilege_args implements org.apache.thrift.TBase<rename_sentry_privilege_args, rename_sentry_privilege_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("rename_sentry_privilege_args");
+
+    private static final org.apache.thrift.protocol.TField REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", 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 rename_sentry_privilege_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new rename_sentry_privilege_argsTupleSchemeFactory());
+    }
+
+    private TRenamePrivilegesRequest request; // 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 {
+      REQUEST((short)1, "request");
+
+      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: // REQUEST
+            return REQUEST;
+          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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenamePrivilegesRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(rename_sentry_privilege_args.class, metaDataMap);
+    }
+
+    public rename_sentry_privilege_args() {
+    }
+
+    public rename_sentry_privilege_args(
+      TRenamePrivilegesRequest request)
+    {
+      this();
+      this.request = request;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public rename_sentry_privilege_args(rename_sentry_privilege_args other) {
+      if (other.isSetRequest()) {
+        this.request = new TRenamePrivilegesRequest(other.request);
+      }
+    }
+
+    public rename_sentry_privilege_args deepCopy() {
+      return new rename_sentry_privilege_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.request = null;
+    }
+
+    public TRenamePrivilegesRequest getRequest() {
+      return this.request;
+    }
+
+    public void setRequest(TRenamePrivilegesRequest request) {
+      this.request = request;
+    }
+
+    public void unsetRequest() {
+      this.request = null;
+    }
+
+    /** Returns true if field request is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequest() {
+      return this.request != null;
+    }
+
+    public void setRequestIsSet(boolean value) {
+      if (!value) {
+        this.request = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case REQUEST:
+        if (value == null) {
+          unsetRequest();
+        } else {
+          setRequest((TRenamePrivilegesRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case REQUEST:
+        return getRequest();
+
+      }
+      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 REQUEST:
+        return isSetRequest();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof rename_sentry_privilege_args)
+        return this.equals((rename_sentry_privilege_args)that);
+      return false;
+    }
+
+    public boolean equals(rename_sentry_privilege_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_request = true && this.isSetRequest();
+      boolean that_present_request = true && that.isSetRequest();
+      if (this_present_request || that_present_request) {
+        if (!(this_present_request && that_present_request))
+          return false;
+        if (!this.request.equals(that.request))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_request = true && (isSetRequest());
+      builder.append(present_request);
+      if (present_request)
+        builder.append(request);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(rename_sentry_privilege_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      rename_sentry_privilege_args typedOther = (rename_sentry_privilege_args)other;
+
+      lastComparison = Boolean.valueOf(isSetRequest()).compareTo(typedOther.isSetRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, typedOther.request);
+        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("rename_sentry_privilege_args(");
+      boolean first = true;
+
+      sb.append("request:");
+      if (this.request == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.request);
+      }
+      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 (request != null) {
+        request.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 rename_sentry_privilege_argsStandardSchemeFactory implements SchemeFactory {
+      public rename_sentry_privilege_argsStandardScheme getScheme() {
+        return new rename_sentry_privilege_argsStandardScheme();
+      }
+    }
+
+    private static class rename_sentry_privilege_argsStandardScheme extends StandardScheme<rename_sentry_privilege_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, rename_sentry_privilege_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: // REQUEST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.request = new TRenamePrivilegesRequest();
+                struct.request.read(iprot);
+                struct.setRequestIsSet(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, rename_sentry_privilege_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.request != null) {
+          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
+          struct.request.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class rename_sentry_privilege_argsTupleSchemeFactory implements SchemeFactory {
+      public rename_sentry_privilege_argsTupleScheme getScheme() {
+        return new rename_sentry_privilege_argsTupleScheme();
+      }
+    }
+
+    private static class rename_sentry_privilege_argsTupleScheme extends TupleScheme<rename_sentry_privilege_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, rename_sentry_privilege_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetRequest()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetRequest()) {
+          struct.request.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, rename_sentry_privilege_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.request = new TRenamePrivilegesRequest();
+          struct.request.read(iprot);
+          struct.setRequestIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class rename_sentry_privilege_result implements org.apache.thrift.TBase<rename_sentry_privilege_result, rename_sentry_privilege_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("rename_sentry_privilege_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 rename_sentry_privilege_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new rename_sentry_privilege_resultTupleSchemeFactory());
+    }
+
+    private TRenamePrivilegesResponse 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, TRenamePrivilegesResponse.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(rename_sentry_privilege_result.class, metaDataMap);
+    }
+
+    public rename_sentry_privilege_result() {
+    }
+
+    public rename_sentry_privilege_result(
+      TRenamePrivilegesResponse success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public rename_sentry_privilege_result(rename_sentry_privilege_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TRenamePrivilegesResponse(other.success);
+      }
+    }
+
+    public rename_sentry_privilege_result deepCopy() {
+      return new rename_sentry_privilege_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public TRenamePrivilegesResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(TRenamePrivilegesResponse 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((TRenamePrivilegesResponse)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 rename_sentry_privilege_result)
+        return this.equals((rename_sentry_privilege_result)that);
+      return false;
+    }
+
+    public boolean equals(rename_sentry_privilege_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() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (isSetSuccess());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(rename_sentry_privilege_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      rename_sentry_privilege_result typedOther = (rename_sentry_privilege_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.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("rename_sentry_privilege_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 rename_sentry_privilege_resultStandardSchemeFactory implements SchemeFactory {
+      public rename_sentry_privilege_resultStandardScheme getScheme() {
+        return new rename_sentry_privilege_resultStandardScheme();
+      }
+    }
+
+    private static class rename_sentry_privilege_resultStandardScheme extends StandardScheme<rename_sentry_privilege_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, rename_sentry_privilege_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 TRenamePrivilegesResponse();
+                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, rename_sentry_privilege_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 rename_sentry_privilege_resultTupleSchemeFactory implements SchemeFactory {
+      public rename_sentry_privilege_resultTupleScheme getScheme() {
+        return new rename_sentry_privilege_resultTupleScheme();
+      }
+    }
+
+    private static class rename_sentry_privilege_resultTupleScheme extends TupleScheme<rename_sentry_privilege_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, rename_sentry_privilege_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, rename_sentry_privilege_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new TRenamePrivilegesResponse();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
 }