You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by pr...@apache.org on 2014/11/18 09:47:56 UTC

[5/5] incubator-sentry git commit: SENTRY-74: Add column-level privileges for Hive/Impala (Dapeng Sun and Xiaomeng Huang via Prasad Mujumdar)

SENTRY-74: Add column-level privileges for Hive/Impala (Dapeng Sun and Xiaomeng Huang via Prasad Mujumdar)


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

Branch: refs/heads/master
Commit: 13e83d6e524e696ef973f251577e93f236e39cd9
Parents: aab7873
Author: Prasad Mujumdar <pr...@cloudera.com>
Authored: Tue Nov 18 00:47:09 2014 -0800
Committer: Prasad Mujumdar <pr...@cloudera.com>
Committed: Tue Nov 18 00:47:09 2014 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/SentryHiveConstants.java |   1 -
 .../hive/ql/exec/SentryGrantRevokeTask.java     |  52 +-
 .../ql/exec/SentryHivePrivilegeObjectDesc.java  |  12 +
 .../binding/hive/HiveAuthzBindingHook.java      | 109 +-
 .../SentryHiveAuthorizationTaskFactoryImpl.java |  35 +-
 .../binding/hive/authz/HiveAuthzBinding.java    |   4 +-
 .../hive/authz/HiveAuthzPrivilegesMap.java      |   6 +-
 .../binding/hive/authz/SentryConfigTool.java    |  15 +
 .../binding/hive/TestHiveAuthzBindings.java     |  18 +-
 .../org/apache/sentry/core/model/db/Column.java |  51 +
 .../core/model/db/DBModelAuthorizable.java      |   1 +
 .../sentry/policy/db/DBModelAuthorizables.java  |   3 +
 .../common/ResourceAuthorizationProvider.java   |   5 +-
 .../TAlterSentryRoleGrantPrivilegeRequest.java  | 217 +++-
 .../TAlterSentryRoleGrantPrivilegeResponse.java | 168 +++-
 .../TAlterSentryRoleRevokePrivilegeRequest.java | 217 +++-
 .../TListSentryPrivilegesByAuthRequest.java     |  68 +-
 .../TListSentryPrivilegesByAuthResponse.java    |  52 +-
 ...TListSentryPrivilegesForProviderRequest.java |  32 +-
 ...ListSentryPrivilegesForProviderResponse.java |  32 +-
 .../thrift/TListSentryPrivilegesResponse.java   |  36 +-
 .../thrift/TListSentryRolesResponse.java        |  36 +-
 .../db/service/thrift/TSentryActiveRoleSet.java |  32 +-
 .../db/service/thrift/TSentryAuthorizable.java  | 114 ++-
 .../db/service/thrift/TSentryPrivilege.java     | 117 ++-
 .../db/service/thrift/TSentryPrivilegeMap.java  |  76 +-
 .../provider/db/service/thrift/TSentryRole.java |  36 +-
 .../db/log/entity/AuditMetadataLogEntity.java   |  15 +-
 .../db/log/entity/JsonLogEntityFactory.java     |  37 +-
 .../provider/db/log/util/CommandUtil.java       |  20 +-
 .../sentry/provider/db/log/util/Constants.java  |   1 +
 .../db/service/model/MSentryPrivilege.java      | 151 +--
 .../provider/db/service/model/package.jdo       |   4 +
 .../db/service/persistent/SentryStore.java      | 148 ++-
 .../thrift/SentryPolicyServiceClient.java       | 230 ++++-
 .../thrift/SentryPolicyStoreProcessor.java      |  72 +-
 .../src/main/resources/004-SENTRY-74.derby.sql  |   4 +
 .../src/main/resources/004-SENTRY-74.mysql.sql  |   4 +
 .../src/main/resources/004-SENTRY-74.oracle.sql |   4 +
 .../main/resources/004-SENTRY-74.postgres.sql   |   4 +
 .../src/main/resources/sentry-db2-1.5.0.sql     |   3 +-
 .../src/main/resources/sentry-derby-1.5.0.sql   |   3 +-
 .../src/main/resources/sentry-mysql-1.5.0.sql   |   6 +-
 .../src/main/resources/sentry-oracle-1.5.0.sql  |   5 +-
 .../main/resources/sentry-postgres-1.5.0.sql    |   5 +-
 .../sentry-upgrade-db2-1.4.0-to-1.5.0.sql       |   5 +
 .../sentry-upgrade-derby-1.4.0-to-1.5.0.sql     |   2 +
 .../sentry-upgrade-mysql-1.4.0-to-1.5.0.sql     |   1 +
 .../sentry-upgrade-oracle-1.4.0-to-1.5.0.sql    |   1 +
 .../sentry-upgrade-postgres-1.4.0-to-1.5.0.sql  |   1 +
 .../main/resources/sentry_policy_service.thrift |   9 +-
 .../log/entity/TestAuditMetadataLogEntity.java  |   3 +-
 .../db/log/entity/TestJsonLogEntityFactory.java |  58 +-
 .../provider/db/log/util/TestCommandUtil.java   |  50 +-
 .../service/persistent/TestSentryPrivilege.java |  47 +
 .../db/service/persistent/TestSentryStore.java  | 296 ++++++
 .../TestSentryStoreToAuthorizable.java          |  22 +-
 .../thrift/TestSentryServiceIntegration.java    |  64 ++
 .../e2e/dbprovider/TestColumnEndToEnd.java      | 247 +++++
 .../e2e/dbprovider/TestDatabaseProvider.java    | 141 ++-
 .../TestDbPrivilegesAtColumnScope.java          |  39 +
 .../AbstractTestWithStaticConfiguration.java    |  10 +-
 .../sentry/tests/e2e/hive/TestConfigTool.java   |  68 +-
 .../e2e/hive/TestExportImportPrivileges.java    |  10 +
 .../sentry/tests/e2e/hive/TestPolicyImport.java |  40 +-
 .../e2e/hive/TestPrivilegesAtColumnScope.java   | 992 +++++++++++++++++++
 .../e2e/hive/hiveserver/HiveServerFactory.java  |   1 +
 .../metastore/SentryPolicyProviderForDb.java    |   9 +-
 68 files changed, 3826 insertions(+), 551 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
index 6f83cc6..f5b1f4d 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
@@ -27,7 +27,6 @@ public class SentryHiveConstants {
       PrivilegeType.ALTER_METADATA, PrivilegeType.INDEX, PrivilegeType.LOCK);
 
   public static final String PRIVILEGE_NOT_SUPPORTED = "Sentry does not support privilege: ";
-  public static final String COLUMN_PRIVS_NOT_SUPPORTED = "Sentry users should use views to grant privileges on columns";
   public static final String PARTITION_PRIVS_NOT_SUPPORTED = "Sentry does not support partition level authorization";
   public static final String GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT = "Sentry does not allow grant/revoke on: ";
   public static final String GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL = "Sentry does not allow privileges to be granted/revoked to/from: ";

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
index ac45746..5357180 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.GrantDesc;
 import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
@@ -66,6 +67,7 @@ import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.common.utils.PathUtils;
 import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.core.model.db.AccessURI;
+import org.apache.sentry.core.model.db.Column;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Server;
 import org.apache.sentry.core.model.db.Table;
@@ -81,6 +83,8 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 
 public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable {
@@ -316,7 +320,16 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
       } else {
         SentryHivePrivilegeObjectDesc privSubjectDesc = toSentryHivePrivilegeObjectDesc(hiveObjectDesc);
         List<Authorizable> authorizableHeirarchy = toAuthorizable(privSubjectDesc);
-        privileges = sentryClient.listPrivilegesByRoleName(subject, principalName, authorizableHeirarchy);
+        if (privSubjectDesc.getColumns() != null && !privSubjectDesc.getColumns().isEmpty()) {
+          List<List<Authorizable>> ps = parseColumnToAuthorizable(authorizableHeirarchy, privSubjectDesc);
+          ImmutableSet.Builder<TSentryPrivilege> pbuilder = new ImmutableSet.Builder<TSentryPrivilege>();
+          for (List<Authorizable> p : ps) {
+            pbuilder.addAll(sentryClient.listPrivilegesByRoleName(subject, principalName, p));
+          }
+          privileges = pbuilder.build();
+        } else {
+          privileges = sentryClient.listPrivilegesByRoleName(subject, principalName, authorizableHeirarchy);
+        }
       }
       writeToFile(writeGrantInfo(privileges, principalName), desc.getResFile());
       return RETURN_CODE_SUCCESS;
@@ -343,7 +356,6 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
       String tableName = dbTable.getTable();
       authorizableHeirarchy.add(new Table(tableName));
       authorizableHeirarchy.add(new Database(dbName));
-
     } else if (privSubjectDesc.getUri()) {
       String uriPath = privSubjectDesc.getObject();
       String warehouseDir = conf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
@@ -359,6 +371,21 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
     return authorizableHeirarchy;
   }
 
+  private List<List<Authorizable>> parseColumnToAuthorizable(List<Authorizable> authorizableHeirarchy,
+      SentryHivePrivilegeObjectDesc privSubjectDesc) {
+    ImmutableList.Builder<List<Authorizable>> listsBuilder = ImmutableList.builder();
+    List<String> cols = privSubjectDesc.getColumns();
+    if ( cols != null && !cols.isEmpty() ) {
+      for ( String col : cols ) {
+        ImmutableList.Builder<Authorizable> listBuilder = ImmutableList.builder();
+        listBuilder.addAll(authorizableHeirarchy);
+        listBuilder.add(new Column(col));
+        listsBuilder.add(listBuilder.build());
+      }
+    }
+    return listsBuilder.build();
+  }
+
   private void writeToFile(String data, String file) throws IOException {
     Path resFile = new Path(file);
     FileSystem fs = resFile.getFileSystem(conf);
@@ -425,7 +452,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
       }
       appendNonNull(builder, privilege.getTableName());
       appendNonNull(builder, null);//getPartValues()
-      appendNonNull(builder, null);//getColumnName()
+      appendNonNull(builder, privilege.getColumnName());//getColumnName()
       appendNonNull(builder, roleName);//getPrincipalName()
       appendNonNull(builder, "ROLE");//getPrincipalType()
       appendNonNull(builder, privilege.getAction());
@@ -502,6 +529,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
 
     String dbName = null;
     String tableName = null;
+    List<String> columnNames = null;
     String uriPath = null;
     String serverName = null;
     try {
@@ -528,12 +556,18 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
       for (PrivilegeDesc privDesc : privileges) {
         List<String> columns = privDesc.getColumns();
         if (columns != null && !columns.isEmpty()) {
-          throw new HiveException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+          columnNames = columns;
         }
         if (!SentryHiveConstants.ALLOWED_PRIVS.contains(privDesc.getPrivilege().getPriv())) {
           String msg = SentryHiveConstants.PRIVILEGE_NOT_SUPPORTED + privDesc.getPrivilege().getPriv();
           throw new HiveException(msg);
         }
+        if (columnNames != null && (privDesc.getPrivilege().getPriv().equals(PrivilegeType.INSERT)
+            || privDesc.getPrivilege().getPriv().equals(PrivilegeType.ALL))) {
+          String msg = SentryHiveConstants.PRIVILEGE_NOT_SUPPORTED
+              + privDesc.getPrivilege().getPriv() + " on Column";
+          throw new SemanticException(msg);
+        }
       }
       for (PrincipalDesc princ : principals) {
         if (princ.getType() != PrincipalType.ROLE) {
@@ -550,9 +584,12 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
             } else if (tableName == null) {
               sentryClient.grantDatabasePrivilege(subject, princ.getName(), server, dbName,
                   toDbSentryAction(privDesc.getPrivilege().getPriv()), grantOption);
-            } else {
+            } else if (columnNames == null) {
               sentryClient.grantTablePrivilege(subject, princ.getName(), server, dbName,
                   tableName, toSentryAction(privDesc.getPrivilege().getPriv()), grantOption);
+            } else {
+              sentryClient.grantColumnsPrivileges(subject, princ.getName(), server, dbName,
+                  tableName, columnNames, toSentryAction(privDesc.getPrivilege().getPriv()), grantOption);
             }
           } else {
             if (serverName != null) {
@@ -562,9 +599,12 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
             } else if (tableName == null) {
               sentryClient.revokeDatabasePrivilege(subject, princ.getName(), server, dbName,
                   toDbSentryAction(privDesc.getPrivilege().getPriv()), grantOption);
-            } else {
+            } else if (columnNames == null) {
               sentryClient.revokeTablePrivilege(subject, princ.getName(), server, dbName,
                   tableName, toSentryAction(privDesc.getPrivilege().getPriv()), grantOption);
+            } else {
+              sentryClient.revokeColumnsPrivilege(subject, princ.getName(), server, dbName,
+                  tableName, columnNames, toSentryAction(privDesc.getPrivilege().getPriv()), grantOption);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java
index 1b5f557..0c59abb 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java
@@ -17,11 +17,15 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
 
 public class SentryHivePrivilegeObjectDesc extends PrivilegeObjectDesc {
   private boolean isUri;
   private boolean isServer;
+  private List<String> columns = new ArrayList<String>();
 
   public SentryHivePrivilegeObjectDesc() {
     // reset table type which is on by default
@@ -44,4 +48,12 @@ public class SentryHivePrivilegeObjectDesc extends PrivilegeObjectDesc {
     this.isServer = isServer;
   }
 
+  public List<String> getColumns() {
+    return columns;
+  }
+
+  public void setColumns(List<String> columns) {
+    this.columns = columns;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
index 1e59671..97ef3b8 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
@@ -56,6 +56,7 @@ import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.common.utils.PathUtils;
 import org.apache.sentry.core.model.db.AccessURI;
+import org.apache.sentry.core.model.db.Column;
 import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
@@ -417,24 +418,7 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
       inputHierarchy.add(dbHierarchy);
       outputHierarchy.add(dbHierarchy);
 
-      for(ReadEntity readEntity:inputs) {
-        // skip the tables/view that are part of expanded view definition.
-        if (isChildTabForView(readEntity)) {
-          continue;
-        }
-        // If this is a UDF, then check whether its allowed to be executed
-         // TODO: when we support execute privileges on UDF, this can be removed.
-        if (isUDF(readEntity)) {
-          if (isBuiltinUDF(readEntity)) {
-            checkUDFWhiteList(readEntity.getUDF().getDisplayName());
-          }
-          continue;
-        }
-        List<DBModelAuthorizable> entityHierarchy = new ArrayList<DBModelAuthorizable>();
-        entityHierarchy.add(hiveAuthzBinding.getAuthServer());
-        entityHierarchy.addAll(getAuthzHierarchyFromEntity(readEntity));
-        inputHierarchy.add(entityHierarchy);
-      }
+      getInputHierarchyFromInputs(inputHierarchy, inputs);
       break;
     case TABLE:
       // workaround for add partitions
@@ -442,24 +426,8 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
         inputHierarchy.add(ImmutableList.of(hiveAuthzBinding.getAuthServer(), partitionURI));
       }
 
-      for (ReadEntity readEntity: inputs) {
-        // skip the tables/view that are part of expanded view definition.
-        if (isChildTabForView(readEntity)) {
-          continue;
-        }
-        // If this is a UDF, then check whether its allowed to be executed
-        // TODO: when we support execute privileges on UDF, this can be removed.
-        if (isUDF(readEntity)) {
-          if (isBuiltinUDF(readEntity)) {
-            checkUDFWhiteList(readEntity.getUDF().getDisplayName());
-          }
-          continue;
-        }
-        List<DBModelAuthorizable> entityHierarchy = new ArrayList<DBModelAuthorizable>();
-        entityHierarchy.add(hiveAuthzBinding.getAuthServer());
-        entityHierarchy.addAll(getAuthzHierarchyFromEntity(readEntity));
-        inputHierarchy.add(entityHierarchy);
-      }
+      getInputHierarchyFromInputs(inputHierarchy, inputs);
+
       for (WriteEntity writeEntity: outputs) {
         if (filterWriteEntity(writeEntity)) {
           continue;
@@ -491,6 +459,7 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
       connectHierarchy.add(hiveAuthzBinding.getAuthServer());
       // by default allow connect access to default db
       Table currTbl = Table.ALL;
+      Column currCol = Column.ALL;
       if ((DEFAULT_DATABASE_NAME.equalsIgnoreCase(currDB.getName()) &&
           "false".equalsIgnoreCase(authzConf.
               get(HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(), "false")))
@@ -502,6 +471,7 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
 
       connectHierarchy.add(currDB);
       connectHierarchy.add(currTbl);
+      connectHierarchy.add(currCol);
 
       inputHierarchy.add(connectHierarchy);
       // check if this is a create temp function and we need to validate URI
@@ -600,6 +570,67 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
     return objectHierarchy;
   }
 
+  /**
+   * Add column level hierarchy to inputHierarchy
+   *
+   * @param inputHierarchy
+   * @param entity
+   * @param sentryContext
+   */
+  private void addColumnHierarchy(List<List<DBModelAuthorizable>> inputHierarchy,
+      ReadEntity entity) {
+    List<DBModelAuthorizable> entityHierarchy = new ArrayList<DBModelAuthorizable>();
+    entityHierarchy.add(hiveAuthzBinding.getAuthServer());
+    entityHierarchy.addAll(getAuthzHierarchyFromEntity(entity));
+
+    switch (entity.getType()) {
+    case TABLE:
+    case PARTITION:
+      List<String> cols = entity.getAccessedColumns();
+      for (String col : cols) {
+        List<DBModelAuthorizable> colHierarchy = new ArrayList<DBModelAuthorizable>(entityHierarchy);
+        colHierarchy.add(new Column(col));
+        inputHierarchy.add(colHierarchy);
+      }
+      break;
+    default:
+      inputHierarchy.add(entityHierarchy);
+    }
+  }
+
+  /**
+   * Get Authorizable from inputs and put into inputHierarchy
+   *
+   * @param inputHierarchy
+   * @param entity
+   * @param sentryContext
+   */
+  private void getInputHierarchyFromInputs(List<List<DBModelAuthorizable>> inputHierarchy,
+      Set<ReadEntity> inputs) {
+    for (ReadEntity readEntity: inputs) {
+      // skip the tables/view that are part of expanded view definition.
+      if (isChildTabForView(readEntity)) {
+        continue;
+      }
+      // If this is a UDF, then check whether its allowed to be executed
+      // TODO: when we support execute privileges on UDF, this can be removed.
+      if (isUDF(readEntity)) {
+        if (isBuiltinUDF(readEntity)) {
+          checkUDFWhiteList(readEntity.getUDF().getDisplayName());
+        }
+        continue;
+      }
+      if (readEntity.getAccessedColumns() != null && !readEntity.getAccessedColumns().isEmpty()) {
+        addColumnHierarchy(inputHierarchy, readEntity);
+      } else {
+        List<DBModelAuthorizable> entityHierarchy = new ArrayList<DBModelAuthorizable>();
+        entityHierarchy.add(hiveAuthzBinding.getAuthServer());
+        entityHierarchy.addAll(getAuthzHierarchyFromEntity(readEntity));
+        inputHierarchy.add(entityHierarchy);
+      }
+    }
+  }
+
   // Check if this write entity needs to skipped
   private boolean filterWriteEntity(WriteEntity writeEntity)
       throws AuthorizationException {
@@ -640,7 +671,7 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
     List<String> filteredResult = new ArrayList<String>();
     Subject subject = new Subject(userName);
     HiveAuthzPrivileges tableMetaDataPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
-        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
         setOperationScope(HiveOperationScope.TABLE).
         setOperationType(HiveOperationType.INFO).
         build();
@@ -657,6 +688,7 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
       externalAuthorizableHierarchy.add(hiveAuthzBinding.getAuthServer());
       externalAuthorizableHierarchy.add(database);
       externalAuthorizableHierarchy.add(table);
+      externalAuthorizableHierarchy.add(Column.ALL);
       inputHierarchy.add(externalAuthorizableHierarchy);
 
       try {
@@ -679,7 +711,7 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
     List<String> filteredResult = new ArrayList<String>();
     Subject subject = new Subject(userName);
     HiveAuthzPrivileges anyPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
-        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
         addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).
         setOperationScope(HiveOperationScope.CONNECT).
         setOperationType(HiveOperationType.QUERY).
@@ -707,6 +739,7 @@ hiveAuthzBinding.getAuthzConf().get(
       externalAuthorizableHierarchy.add(hiveAuthzBinding.getAuthServer());
       externalAuthorizableHierarchy.add(database);
       externalAuthorizableHierarchy.add(Table.ALL);
+      externalAuthorizableHierarchy.add(Column.ALL);
       inputHierarchy.add(externalAuthorizableHierarchy);
 
       try {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
index 39a22c6..9b44f15 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
 import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
 import org.apache.hadoop.hive.ql.security.authorization.Privilege;
 import org.apache.hadoop.hive.ql.security.authorization.PrivilegeRegistry;
+import org.apache.hadoop.hive.ql.security.authorization.PrivilegeType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.sentry.core.model.db.AccessConstants;
 import org.slf4j.Logger;
@@ -140,12 +141,6 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
     if (privilegeObj.getPartSpec() != null) {
       throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
     }
-    for (PrivilegeDesc privDesc : privilegeDesc) {
-      List<String> columns = privDesc.getColumns();
-      if (columns != null && !columns.isEmpty()) {
-        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
-      }
-    }
     for (PrincipalDesc princ : principalDesc) {
       if (princ.getType() != PrincipalType.ROLE) {
         String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + princ.getType();
@@ -169,12 +164,6 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
     if (privilegeObj.getPartSpec() != null) {
       throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
     }
-    for (PrivilegeDesc privDesc : privilegeDesc) {
-      List<String> columns = privDesc.getColumns();
-      if (columns != null && !columns.isEmpty()) {
-        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
-      }
-    }
     for (PrincipalDesc princ : principalDesc) {
       if (princ.getType() != PrincipalType.ROLE) {
         String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + princ.getType();
@@ -194,7 +183,7 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
   @Override
   public Task<? extends Serializable> createShowGrantTask(ASTNode ast, Path resultFile, HashSet<ReadEntity> inputs,
       HashSet<WriteEntity> outputs) throws SemanticException {
-    PrivilegeObjectDesc privHiveObj = null;
+    SentryHivePrivilegeObjectDesc privHiveObj = null;
 
     ASTNode principal = (ASTNode) ast.getChild(0);
     PrincipalType type = PrincipalType.USER;
@@ -216,18 +205,20 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
     String principalName = BaseSemanticAnalyzer.unescapeIdentifier(principal.getChild(0).getText());
     PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
 
-    //Column privileges and Partition privileges are not supported by Sentry
+    // Partition privileges are not supported by Sentry
+    List<String> cols = null;
     if (ast.getChildCount() > 1) {
       ASTNode child = (ASTNode) ast.getChild(1);
       if (child.getToken().getType() == HiveParser.TOK_PRIV_OBJECT_COL) {
         privHiveObj = analyzePrivilegeObject(child);
+        cols = privHiveObj.getColumns();
       }else {
         throw new SemanticException("Unrecognized Token: " + child.getToken().getType());
       }
     }
 
     ShowGrantDesc showGrant = new ShowGrantDesc(resultFile.toString(),
-        principalDesc, privHiveObj, null);
+        principalDesc, privHiveObj, cols);
     return createTask(new DDLWork(inputs, outputs, showGrant));
   }
 
@@ -297,8 +288,8 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
         if (astChild.getToken().getType() == HiveParser.TOK_PARTSPEC) {
           throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
         } else if (astChild.getToken().getType() == HiveParser.TOK_TABCOLNAME) {
-          throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
-        }else if (astChild.getToken().getType() == HiveParser.TOK_URI) {
+          subject.setColumns(BaseSemanticAnalyzer.getColumnNames(astChild));
+        } else if (astChild.getToken().getType() == HiveParser.TOK_URI) {
           privilegeObject = privilegeObject.replaceAll("'", "").replaceAll("\"", "");
           subject.setUri(true);
         } else if (astChild.getToken().getType() == HiveParser.TOK_SERVER) {
@@ -359,10 +350,16 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
         String msg = SentryHiveConstants.PRIVILEGE_NOT_SUPPORTED + privObj.getPriv();
         throw new SemanticException(msg);
       }
+      List<String> cols = null;
       if (privilegeDef.getChildCount() > 1) {
-        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+        cols = BaseSemanticAnalyzer.getColumnNames((ASTNode) privilegeDef.getChild(1));
+      }
+      if (cols != null && (privObj.getPriv().equals(PrivilegeType.INSERT)
+              || privObj.getPriv().equals(PrivilegeType.ALL))) {
+        String msg = SentryHiveConstants.PRIVILEGE_NOT_SUPPORTED + privObj.getPriv() + " on Column";
+        throw new SemanticException(msg);
       }
-      PrivilegeDesc privilegeDesc = new PrivilegeDesc(privObj, null);
+      PrivilegeDesc privilegeDesc = new PrivilegeDesc(privObj, cols);
       ret.add(privilegeDesc);
     }
     return ret;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
index 68b93aa..b4b69e1 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
@@ -322,9 +322,11 @@ public class HiveAuthzBinding {
           }
         }
       }
-      if(!found && !(key.equals(AuthorizableType.URI)) &&  !(hiveOp.equals(HiveOperation.QUERY))) {
+      if(!found && !(key.equals(AuthorizableType.URI)) &&  !(hiveOp.equals(HiveOperation.QUERY))
+          && !(hiveOp.equals(HiveOperation.CREATETABLE_AS_SELECT))) {
         //URI privileges are optional for some privileges: anyPrivilege, tableDDLAndOptionalUriPrivilege
         //Query can mean select/insert/analyze where all of them have different required privileges.
+        //CreateAsSelect can has table/columns privileges with select.
         //For these alone we skip if there is no equivalent input privilege
         //TODO: Even this case should be handled to make sure we do not skip the privilege check if we did not build
         //the input privileges correctly

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
index fa4efc9..b5c7aff 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
@@ -114,6 +114,7 @@ public class HiveAuthzPrivilegesMap {
      */
     HiveAuthzPrivileges tableQueryPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT)).
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT)).
         addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.INSERT)).
         addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
         setOperationScope(HiveOperationScope.TABLE).
@@ -168,8 +169,8 @@ public class HiveAuthzPrivilegesMap {
         build();
 
     HiveAuthzPrivileges anyPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
-        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT,
-            DBModelAction.ALTER, DBModelAction.CREATE, DBModelAction.DROP, DBModelAction.DROP,
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT,
+            DBModelAction.INSERT, DBModelAction.ALTER, DBModelAction.CREATE, DBModelAction.DROP,
             DBModelAction.INDEX, DBModelAction.LOCK)).
         addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)). //TODO: make them ||
         addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
@@ -261,6 +262,7 @@ public class HiveAuthzPrivilegesMap {
     hiveAuthzStmtPrivMap.put(HiveOperation.CREATETABLE_AS_SELECT,
         new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT)).
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT)).
         addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.CREATE)).
         setOperationScope(HiveOperationScope.DATABASE).
         setOperationType(HiveOperationType.DDL).

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
index 2b978d5..8bf1d86 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
@@ -290,6 +290,7 @@ public class SentryConfigTool {
           String server = null;
           String database = null;
           String table = null;
+          String column = null;
           String uri = null;
           String action = AccessConstants.ALL;
           for (String authorizable : PolicyFileConstants.AUTHORIZABLE_SPLITTER.
@@ -315,6 +316,9 @@ public class SentryConfigTool {
               case URI:
                 uri = a.getName();
                 break;
+              case Column:
+                column = a.getName();
+                break;
               default:
                 break;
             }
@@ -329,6 +333,17 @@ public class SentryConfigTool {
                 uri, roleName));
 
             client.grantURIPrivilege(requestorUserName, roleName, server, uri);
+          } else if (column != null && !AccessConstants.ALL.equals(column)) {
+            System.out.println(String.format(
+                "# server=%s, database=%s",
+                server, database));
+            System.out.println(String.format(
+                "GRANT %s (%s) ON TABLE %s TO ROLE %s;",
+                "*".equals(action) ? "ALL" : action.toUpperCase(), column,
+                table, roleName));
+
+            client.grantColumnPrivilege(requestorUserName, roleName, server,
+                database, table, column, action);
           } else if (table != null && !AccessConstants.ALL.equals(table)) {
             System.out.println(String.format(
                 "# server=%s, database=%s",

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java
index 84d7fba..d41f6cf 100644
--- a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java
@@ -37,6 +37,7 @@ import org.apache.sentry.binding.hive.conf.InvalidConfigurationException;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.core.model.db.AccessURI;
+import org.apache.sentry.core.model.db.Column;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Server;
@@ -75,6 +76,10 @@ public class TestHiveAuthzBindings {
   private static final String PURCHASES_TAB = "purchases";
   private static final String PAYMENT_TAB = "payments";
 
+  // Columns
+  private static final String AGE_COL = "age";
+  private static final String NAME_COL = "name";
+
   // Entities
   private List<List<DBModelAuthorizable>> inputTabHierarcyList = new ArrayList<List<DBModelAuthorizable>>();
   private List<List<DBModelAuthorizable>> outputTabHierarcyList = new ArrayList<List<DBModelAuthorizable>>();
@@ -279,7 +284,7 @@ public class TestHiveAuthzBindings {
    */
   @Test
   public void testValidateCreateFunctionForAdmin() throws Exception {
-    inputTabHierarcyList.add(buildObjectHierarchy(SERVER1, CUSTOMER_DB, PURCHASES_TAB));
+    inputTabHierarcyList.add(buildObjectHierarchy(SERVER1, CUSTOMER_DB, PURCHASES_TAB, AGE_COL));
     inputTabHierarcyList.add(Arrays.asList(new DBModelAuthorizable[] {
         new Server(SERVER1), new AccessURI("file:///some/path/to/a/jar")
     }));
@@ -289,7 +294,8 @@ public class TestHiveAuthzBindings {
   @Test
   public void testValidateCreateFunctionAppropiateURI() throws Exception {
     inputTabHierarcyList.add(Arrays.asList(new DBModelAuthorizable[] {
-        new Server(SERVER1), new Database(CUSTOMER_DB), new Table(AccessConstants.ALL)
+        new Server(SERVER1), new Database(CUSTOMER_DB), new Table(AccessConstants.ALL),
+        new Column(AccessConstants.ALL)
     }));
     inputTabHierarcyList.add(Arrays.asList(new DBModelAuthorizable[] {
         new Server(SERVER1), new AccessURI("file:///path/to/some/lib/dir/my.jar")
@@ -397,6 +403,14 @@ public class TestHiveAuthzBindings {
     return authList;
   }
 
+  private List <DBModelAuthorizable>  buildObjectHierarchy(String server, String db, String table, String column) {
+    List <DBModelAuthorizable> authList = buildObjectHierarchy(server, db, table);
+    if (server != null && db != null && table != null && column != null) {
+      authList.add(new Column(column));
+    }
+    return authList;
+  }
+
   /**
    * Turn off authentication and verify exception is raised in non-testing mode
    * @throws Exception

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Column.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Column.java b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Column.java
new file mode 100644
index 0000000..89aabfc
--- /dev/null
+++ b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Column.java
@@ -0,0 +1,51 @@
+/*
+ * 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.core.model.db;
+
+public class Column implements DBModelAuthorizable {
+
+  /**
+   * Represents all Column
+   */
+  public static final Column ALL = new Column(AccessConstants.ALL);
+
+  private final String name;
+
+  public Column(String name) {
+    this.name = name;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String toString() {
+    return "Column [name=" + name + "]";
+  }
+
+  @Override
+  public AuthorizableType getAuthzType() {
+    return AuthorizableType.Column;
+  }
+
+  @Override
+  public String getTypeName() {
+    return getAuthzType().name();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/DBModelAuthorizable.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/DBModelAuthorizable.java b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/DBModelAuthorizable.java
index de35bfa..4d74356 100644
--- a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/DBModelAuthorizable.java
+++ b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/DBModelAuthorizable.java
@@ -24,6 +24,7 @@ public interface DBModelAuthorizable extends Authorizable {
     Server,
     Db,
     Table,
+    Column,
     View,
     URI
   };

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java
index 873f789..e47c733 100644
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java
@@ -17,6 +17,7 @@
 package org.apache.sentry.policy.db;
 
 import org.apache.sentry.core.model.db.AccessURI;
+import org.apache.sentry.core.model.db.Column;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 import org.apache.sentry.core.model.db.Database;
@@ -55,6 +56,8 @@ public class DBModelAuthorizables {
       return new Table(name);
     case View:
       return new View(name);
+    case Column:
+      return new Column(name);
     case URI:
       return new AccessURI(name);
     default:

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
index 51d4248..6449405 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
@@ -135,12 +135,13 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
 
   private ImmutableSet<String> appendDefaultDBPriv(ImmutableSet<String> privileges, Authorizable[] authorizables) {
     // Only for switch db
-    if ((authorizables != null)&&(authorizables.length == 3)&&(authorizables[2].getName().equals("+"))) {
+    if ((authorizables != null)&&(authorizables.length == 4)&&(authorizables[2].getName().equals("+"))) {
       if ((privileges.size() == 1) && hasOnlyServerPrivilege(privileges.asList().get(0))) {
         // Assuming authorizable[0] will always be the server
         // This Code is only reachable only when user fires a 'use default'
         // and the user has a privilege on atleast 1 privilized Object
-        String defaultPriv = "Server=" + authorizables[0].getName() + "->Db=default->Table=*->action=select";
+        String defaultPriv = "Server=" + authorizables[0].getName()
+            + "->Db=default->Table=*->Column=*->action=select";
         HashSet<String> newPrivs = Sets.newHashSet(defaultPriv);
         return ImmutableSet.copyOf(newPrivs);
       }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java
index 62b6b31..6b051a1 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java
@@ -38,6 +38,7 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
   private static final org.apache.thrift.protocol.TField REQUESTOR_USER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("requestorUserName", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField ROLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("roleName", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField PRIVILEGE_FIELD_DESC = new org.apache.thrift.protocol.TField("privilege", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+  private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.SET, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -48,14 +49,16 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
   private int protocol_version; // required
   private String requestorUserName; // required
   private String roleName; // required
-  private TSentryPrivilege privilege; // required
+  private TSentryPrivilege privilege; // optional
+  private Set<TSentryPrivilege> privileges; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     PROTOCOL_VERSION((short)1, "protocol_version"),
     REQUESTOR_USER_NAME((short)2, "requestorUserName"),
     ROLE_NAME((short)3, "roleName"),
-    PRIVILEGE((short)5, "privilege");
+    PRIVILEGE((short)5, "privilege"),
+    PRIVILEGES((short)6, "privileges");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -78,6 +81,8 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
           return ROLE_NAME;
         case 5: // PRIVILEGE
           return PRIVILEGE;
+        case 6: // PRIVILEGES
+          return PRIVILEGES;
         default:
           return null;
       }
@@ -120,6 +125,7 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
   // isset id assignments
   private static final int __PROTOCOL_VERSION_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
+  private _Fields optionals[] = {_Fields.PRIVILEGE,_Fields.PRIVILEGES};
   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);
@@ -129,8 +135,11 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.ROLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("roleName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.PRIVILEGE, new org.apache.thrift.meta_data.FieldMetaData("privilege", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+    tmpMap.put(_Fields.PRIVILEGE, new org.apache.thrift.meta_data.FieldMetaData("privilege", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryPrivilege.class)));
+    tmpMap.put(_Fields.PRIVILEGES, new org.apache.thrift.meta_data.FieldMetaData("privileges", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryPrivilege.class))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAlterSentryRoleGrantPrivilegeRequest.class, metaDataMap);
   }
@@ -143,15 +152,13 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
   public TAlterSentryRoleGrantPrivilegeRequest(
     int protocol_version,
     String requestorUserName,
-    String roleName,
-    TSentryPrivilege privilege)
+    String roleName)
   {
     this();
     this.protocol_version = protocol_version;
     setProtocol_versionIsSet(true);
     this.requestorUserName = requestorUserName;
     this.roleName = roleName;
-    this.privilege = privilege;
   }
 
   /**
@@ -169,6 +176,13 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
     if (other.isSetPrivilege()) {
       this.privilege = new TSentryPrivilege(other.privilege);
     }
+    if (other.isSetPrivileges()) {
+      Set<TSentryPrivilege> __this__privileges = new HashSet<TSentryPrivilege>();
+      for (TSentryPrivilege other_element : other.privileges) {
+        __this__privileges.add(new TSentryPrivilege(other_element));
+      }
+      this.privileges = __this__privileges;
+    }
   }
 
   public TAlterSentryRoleGrantPrivilegeRequest deepCopy() {
@@ -182,6 +196,7 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
     this.requestorUserName = null;
     this.roleName = null;
     this.privilege = null;
+    this.privileges = null;
   }
 
   public int getProtocol_version() {
@@ -275,6 +290,44 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
     }
   }
 
+  public int getPrivilegesSize() {
+    return (this.privileges == null) ? 0 : this.privileges.size();
+  }
+
+  public java.util.Iterator<TSentryPrivilege> getPrivilegesIterator() {
+    return (this.privileges == null) ? null : this.privileges.iterator();
+  }
+
+  public void addToPrivileges(TSentryPrivilege elem) {
+    if (this.privileges == null) {
+      this.privileges = new HashSet<TSentryPrivilege>();
+    }
+    this.privileges.add(elem);
+  }
+
+  public Set<TSentryPrivilege> getPrivileges() {
+    return this.privileges;
+  }
+
+  public void setPrivileges(Set<TSentryPrivilege> privileges) {
+    this.privileges = privileges;
+  }
+
+  public void unsetPrivileges() {
+    this.privileges = null;
+  }
+
+  /** Returns true if field privileges is set (has been assigned a value) and false otherwise */
+  public boolean isSetPrivileges() {
+    return this.privileges != null;
+  }
+
+  public void setPrivilegesIsSet(boolean value) {
+    if (!value) {
+      this.privileges = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case PROTOCOL_VERSION:
@@ -309,6 +362,14 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
       }
       break;
 
+    case PRIVILEGES:
+      if (value == null) {
+        unsetPrivileges();
+      } else {
+        setPrivileges((Set<TSentryPrivilege>)value);
+      }
+      break;
+
     }
   }
 
@@ -326,6 +387,9 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
     case PRIVILEGE:
       return getPrivilege();
 
+    case PRIVILEGES:
+      return getPrivileges();
+
     }
     throw new IllegalStateException();
   }
@@ -345,6 +409,8 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
       return isSetRoleName();
     case PRIVILEGE:
       return isSetPrivilege();
+    case PRIVILEGES:
+      return isSetPrivileges();
     }
     throw new IllegalStateException();
   }
@@ -398,6 +464,15 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
         return false;
     }
 
+    boolean this_present_privileges = true && this.isSetPrivileges();
+    boolean that_present_privileges = true && that.isSetPrivileges();
+    if (this_present_privileges || that_present_privileges) {
+      if (!(this_present_privileges && that_present_privileges))
+        return false;
+      if (!this.privileges.equals(that.privileges))
+        return false;
+    }
+
     return true;
   }
 
@@ -425,6 +500,11 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
     if (present_privilege)
       builder.append(privilege);
 
+    boolean present_privileges = true && (isSetPrivileges());
+    builder.append(present_privileges);
+    if (present_privileges)
+      builder.append(privileges);
+
     return builder.toHashCode();
   }
 
@@ -476,6 +556,16 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetPrivileges()).compareTo(typedOther.isSetPrivileges());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPrivileges()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.privileges, typedOther.privileges);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -515,14 +605,26 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
       sb.append(this.roleName);
     }
     first = false;
-    if (!first) sb.append(", ");
-    sb.append("privilege:");
-    if (this.privilege == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.privilege);
+    if (isSetPrivilege()) {
+      if (!first) sb.append(", ");
+      sb.append("privilege:");
+      if (this.privilege == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.privilege);
+      }
+      first = false;
+    }
+    if (isSetPrivileges()) {
+      if (!first) sb.append(", ");
+      sb.append("privileges:");
+      if (this.privileges == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.privileges);
+      }
+      first = false;
     }
-    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -541,10 +643,6 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'roleName' is unset! Struct:" + toString());
     }
 
-    if (!isSetPrivilege()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'privilege' is unset! Struct:" + toString());
-    }
-
     // check for sub-struct validity
     if (privilege != null) {
       privilege.validate();
@@ -620,6 +718,25 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // PRIVILEGES
+            if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
+              {
+                org.apache.thrift.protocol.TSet _set16 = iprot.readSetBegin();
+                struct.privileges = new HashSet<TSentryPrivilege>(2*_set16.size);
+                for (int _i17 = 0; _i17 < _set16.size; ++_i17)
+                {
+                  TSentryPrivilege _elem18; // required
+                  _elem18 = new TSentryPrivilege();
+                  _elem18.read(iprot);
+                  struct.privileges.add(_elem18);
+                }
+                iprot.readSetEnd();
+              }
+              struct.setPrivilegesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -647,9 +764,25 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
         oprot.writeFieldEnd();
       }
       if (struct.privilege != null) {
-        oprot.writeFieldBegin(PRIVILEGE_FIELD_DESC);
-        struct.privilege.write(oprot);
-        oprot.writeFieldEnd();
+        if (struct.isSetPrivilege()) {
+          oprot.writeFieldBegin(PRIVILEGE_FIELD_DESC);
+          struct.privilege.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.privileges != null) {
+        if (struct.isSetPrivileges()) {
+          oprot.writeFieldBegin(PRIVILEGES_FIELD_DESC);
+          {
+            oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, struct.privileges.size()));
+            for (TSentryPrivilege _iter19 : struct.privileges)
+            {
+              _iter19.write(oprot);
+            }
+            oprot.writeSetEnd();
+          }
+          oprot.writeFieldEnd();
+        }
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -671,7 +804,26 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
       oprot.writeI32(struct.protocol_version);
       oprot.writeString(struct.requestorUserName);
       oprot.writeString(struct.roleName);
-      struct.privilege.write(oprot);
+      BitSet optionals = new BitSet();
+      if (struct.isSetPrivilege()) {
+        optionals.set(0);
+      }
+      if (struct.isSetPrivileges()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetPrivilege()) {
+        struct.privilege.write(oprot);
+      }
+      if (struct.isSetPrivileges()) {
+        {
+          oprot.writeI32(struct.privileges.size());
+          for (TSentryPrivilege _iter20 : struct.privileges)
+          {
+            _iter20.write(oprot);
+          }
+        }
+      }
     }
 
     @Override
@@ -683,9 +835,26 @@ public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.
       struct.setRequestorUserNameIsSet(true);
       struct.roleName = iprot.readString();
       struct.setRoleNameIsSet(true);
-      struct.privilege = new TSentryPrivilege();
-      struct.privilege.read(iprot);
-      struct.setPrivilegeIsSet(true);
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.privilege = new TSentryPrivilege();
+        struct.privilege.read(iprot);
+        struct.setPrivilegeIsSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TSet _set21 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.privileges = new HashSet<TSentryPrivilege>(2*_set21.size);
+          for (int _i22 = 0; _i22 < _set21.size; ++_i22)
+          {
+            TSentryPrivilege _elem23; // required
+            _elem23 = new TSentryPrivilege();
+            _elem23.read(iprot);
+            struct.privileges.add(_elem23);
+          }
+        }
+        struct.setPrivilegesIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/13e83d6e/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java
index 5246889..0cadf16 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java
@@ -36,6 +36,7 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
 
   private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1);
   private static final org.apache.thrift.protocol.TField PRIVILEGE_FIELD_DESC = new org.apache.thrift.protocol.TField("privilege", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.SET, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -45,11 +46,13 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
 
   private org.apache.sentry.service.thrift.TSentryResponseStatus status; // required
   private TSentryPrivilege privilege; // optional
+  private Set<TSentryPrivilege> privileges; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     STATUS((short)1, "status"),
-    PRIVILEGE((short)2, "privilege");
+    PRIVILEGE((short)2, "privilege"),
+    PRIVILEGES((short)3, "privileges");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -68,6 +71,8 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
           return STATUS;
         case 2: // PRIVILEGE
           return PRIVILEGE;
+        case 3: // PRIVILEGES
+          return PRIVILEGES;
         default:
           return null;
       }
@@ -108,7 +113,7 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
   }
 
   // isset id assignments
-  private _Fields optionals[] = {_Fields.PRIVILEGE};
+  private _Fields optionals[] = {_Fields.PRIVILEGE,_Fields.PRIVILEGES};
   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);
@@ -116,6 +121,9 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.sentry.service.thrift.TSentryResponseStatus.class)));
     tmpMap.put(_Fields.PRIVILEGE, new org.apache.thrift.meta_data.FieldMetaData("privilege", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryPrivilege.class)));
+    tmpMap.put(_Fields.PRIVILEGES, new org.apache.thrift.meta_data.FieldMetaData("privileges", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryPrivilege.class))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAlterSentryRoleGrantPrivilegeResponse.class, metaDataMap);
   }
@@ -140,6 +148,13 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
     if (other.isSetPrivilege()) {
       this.privilege = new TSentryPrivilege(other.privilege);
     }
+    if (other.isSetPrivileges()) {
+      Set<TSentryPrivilege> __this__privileges = new HashSet<TSentryPrivilege>();
+      for (TSentryPrivilege other_element : other.privileges) {
+        __this__privileges.add(new TSentryPrivilege(other_element));
+      }
+      this.privileges = __this__privileges;
+    }
   }
 
   public TAlterSentryRoleGrantPrivilegeResponse deepCopy() {
@@ -150,6 +165,7 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
   public void clear() {
     this.status = null;
     this.privilege = null;
+    this.privileges = null;
   }
 
   public org.apache.sentry.service.thrift.TSentryResponseStatus getStatus() {
@@ -198,6 +214,44 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
     }
   }
 
+  public int getPrivilegesSize() {
+    return (this.privileges == null) ? 0 : this.privileges.size();
+  }
+
+  public java.util.Iterator<TSentryPrivilege> getPrivilegesIterator() {
+    return (this.privileges == null) ? null : this.privileges.iterator();
+  }
+
+  public void addToPrivileges(TSentryPrivilege elem) {
+    if (this.privileges == null) {
+      this.privileges = new HashSet<TSentryPrivilege>();
+    }
+    this.privileges.add(elem);
+  }
+
+  public Set<TSentryPrivilege> getPrivileges() {
+    return this.privileges;
+  }
+
+  public void setPrivileges(Set<TSentryPrivilege> privileges) {
+    this.privileges = privileges;
+  }
+
+  public void unsetPrivileges() {
+    this.privileges = null;
+  }
+
+  /** Returns true if field privileges is set (has been assigned a value) and false otherwise */
+  public boolean isSetPrivileges() {
+    return this.privileges != null;
+  }
+
+  public void setPrivilegesIsSet(boolean value) {
+    if (!value) {
+      this.privileges = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case STATUS:
@@ -216,6 +270,14 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
       }
       break;
 
+    case PRIVILEGES:
+      if (value == null) {
+        unsetPrivileges();
+      } else {
+        setPrivileges((Set<TSentryPrivilege>)value);
+      }
+      break;
+
     }
   }
 
@@ -227,6 +289,9 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
     case PRIVILEGE:
       return getPrivilege();
 
+    case PRIVILEGES:
+      return getPrivileges();
+
     }
     throw new IllegalStateException();
   }
@@ -242,6 +307,8 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
       return isSetStatus();
     case PRIVILEGE:
       return isSetPrivilege();
+    case PRIVILEGES:
+      return isSetPrivileges();
     }
     throw new IllegalStateException();
   }
@@ -277,6 +344,15 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
         return false;
     }
 
+    boolean this_present_privileges = true && this.isSetPrivileges();
+    boolean that_present_privileges = true && that.isSetPrivileges();
+    if (this_present_privileges || that_present_privileges) {
+      if (!(this_present_privileges && that_present_privileges))
+        return false;
+      if (!this.privileges.equals(that.privileges))
+        return false;
+    }
+
     return true;
   }
 
@@ -294,6 +370,11 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
     if (present_privilege)
       builder.append(privilege);
 
+    boolean present_privileges = true && (isSetPrivileges());
+    builder.append(present_privileges);
+    if (present_privileges)
+      builder.append(privileges);
+
     return builder.toHashCode();
   }
 
@@ -325,6 +406,16 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetPrivileges()).compareTo(typedOther.isSetPrivileges());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPrivileges()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.privileges, typedOther.privileges);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -362,6 +453,16 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
       }
       first = false;
     }
+    if (isSetPrivileges()) {
+      if (!first) sb.append(", ");
+      sb.append("privileges:");
+      if (this.privileges == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.privileges);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -433,6 +534,25 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 3: // PRIVILEGES
+            if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
+              {
+                org.apache.thrift.protocol.TSet _set24 = iprot.readSetBegin();
+                struct.privileges = new HashSet<TSentryPrivilege>(2*_set24.size);
+                for (int _i25 = 0; _i25 < _set24.size; ++_i25)
+                {
+                  TSentryPrivilege _elem26; // required
+                  _elem26 = new TSentryPrivilege();
+                  _elem26.read(iprot);
+                  struct.privileges.add(_elem26);
+                }
+                iprot.readSetEnd();
+              }
+              struct.setPrivilegesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -458,6 +578,20 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
           oprot.writeFieldEnd();
         }
       }
+      if (struct.privileges != null) {
+        if (struct.isSetPrivileges()) {
+          oprot.writeFieldBegin(PRIVILEGES_FIELD_DESC);
+          {
+            oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, struct.privileges.size()));
+            for (TSentryPrivilege _iter27 : struct.privileges)
+            {
+              _iter27.write(oprot);
+            }
+            oprot.writeSetEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -480,10 +614,22 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
       if (struct.isSetPrivilege()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetPrivileges()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetPrivilege()) {
         struct.privilege.write(oprot);
       }
+      if (struct.isSetPrivileges()) {
+        {
+          oprot.writeI32(struct.privileges.size());
+          for (TSentryPrivilege _iter28 : struct.privileges)
+          {
+            _iter28.write(oprot);
+          }
+        }
+      }
     }
 
     @Override
@@ -492,12 +638,26 @@ public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift
       struct.status = new org.apache.sentry.service.thrift.TSentryResponseStatus();
       struct.status.read(iprot);
       struct.setStatusIsSet(true);
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.privilege = new TSentryPrivilege();
         struct.privilege.read(iprot);
         struct.setPrivilegeIsSet(true);
       }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TSet _set29 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.privileges = new HashSet<TSentryPrivilege>(2*_set29.size);
+          for (int _i30 = 0; _i30 < _set29.size; ++_i30)
+          {
+            TSentryPrivilege _elem31; // required
+            _elem31 = new TSentryPrivilege();
+            _elem31.read(iprot);
+            struct.privileges.add(_elem31);
+          }
+        }
+        struct.setPrivilegesIsSet(true);
+      }
     }
   }