You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ak...@apache.org on 2017/07/24 19:25:52 UTC

sentry git commit: SENTRY-1825: Dropping a Hive database/table doesn't cleanup the permissions associated with it (Na Li, reviewed by Alex Kolbasov)

Repository: sentry
Updated Branches:
  refs/heads/sentry-ha-redesign ad7837f92 -> 66686420b


SENTRY-1825: Dropping a Hive database/table doesn't cleanup the permissions associated with it (Na Li, reviewed by Alex Kolbasov)


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

Branch: refs/heads/sentry-ha-redesign
Commit: 66686420bb6fbca4c8922c4b502266da1e6551c1
Parents: ad7837f
Author: Alexander Kolbasov <ak...@cloudera.com>
Authored: Mon Jul 24 21:25:28 2017 +0200
Committer: Alexander Kolbasov <ak...@cloudera.com>
Committed: Mon Jul 24 21:25:28 2017 +0200

----------------------------------------------------------------------
 .../sentry/binding/hive/conf/HiveAuthzConf.java |   5 +-
 .../sentry/service/thrift/HMSFollower.java      |  27 +-
 .../thrift/HiveSimpleConnectionFactory.java     |   2 +-
 .../service/thrift/NotificationProcessor.java   |   5 +
 .../TestHMSFollowerSentryStoreIntegration.java  | 304 +++++++++++++++++++
 .../sentry/service/thrift/TestHMSFollower.java  |  66 +++-
 6 files changed, 395 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/66686420/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java b/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
index e1312bf..d651691 100644
--- a/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
+++ b/sentry-binding/sentry-binding-hive-conf/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
@@ -87,6 +87,7 @@ public class HiveAuthzConf extends Configuration {
                     "noopstreaming,noopwithmapstreaming,windowingtablefunction,matchpath";
 
     public static final String HIVE_UDF_BLACK_LIST = "reflect,reflect2,java_method";
+    public static final String SENTRY_HIVE_SERVER_DEFAULT = "";
 
     /**
      * Config setting definitions
@@ -100,7 +101,7 @@ public class HiveAuthzConf extends Configuration {
         AUTHZ_POLICY_FILE_FORMATTER(
                 "sentry.hive.policy.file.formatter",
                 "org.apache.sentry.binding.hive.SentryIniPolicyFileFormatter"),
-        AUTHZ_SERVER_NAME("sentry.hive.server", ""),
+        AUTHZ_SERVER_NAME("sentry.hive.server", SENTRY_HIVE_SERVER_DEFAULT),
         AUTHZ_RESTRICT_DEFAULT_DB("sentry.hive.restrict.defaultDB", "false"),
         SENTRY_TESTING_MODE("sentry.hive.testing.mode", "false"),
         AUTHZ_ALLOW_HIVE_IMPERSONATION("sentry.hive.allow.hive.impersonation", "false"),
@@ -115,7 +116,7 @@ public class HiveAuthzConf extends Configuration {
         AUTHZ_PROVIDER_DEPRECATED2("sentry.provider",
                 "org.apache.sentry.provider.common.HadoopGroupResourceAuthorizationProvider"),
         AUTHZ_PROVIDER_RESOURCE_DEPRECATED("hive.sentry.provider.resource", ""),
-        AUTHZ_SERVER_NAME_DEPRECATED("hive.sentry.server", ""),
+        AUTHZ_SERVER_NAME_DEPRECATED("hive.sentry.server", SENTRY_HIVE_SERVER_DEFAULT),
         AUTHZ_RESTRICT_DEFAULT_DB_DEPRECATED("hive.sentry.restrict.defaultDB", "false"),
         SENTRY_TESTING_MODE_DEPRECATED("hive.sentry.testing.mode", "false"),
         AUTHZ_ALLOW_HIVE_IMPERSONATION_DEPRECATED("hive.sentry.allow.hive.impersonation", "false"),

http://git-wip-us.apache.org/repos/asf/sentry/blob/66686420/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
index 547a61f..9e8e0e7 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
@@ -18,17 +18,16 @@
 
 package org.apache.sentry.service.thrift;
 
-import com.google.common.annotations.VisibleForTesting;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.net.SocketException;
-
 import java.util.Collection;
 import java.util.List;
 import javax.jdo.JDODataStoreException;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
-import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SERVER_NAME;
+import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SERVER_NAME_DEPRECATED;
 import org.apache.sentry.provider.db.service.persistent.PathsImage;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.thrift.TException;
@@ -73,16 +72,18 @@ public class HMSFollower implements Runnable, AutoCloseable {
    * @param leaderMonitor
    * @param authServerName Server that sentry is Authorizing
    */
-  HMSFollower(Configuration conf, SentryStore store, LeaderStatusMonitor leaderMonitor,
+  public HMSFollower(Configuration conf, SentryStore store, LeaderStatusMonitor leaderMonitor,
               HiveSimpleConnectionFactory hiveConnectionFactory, String authServerName) {
     LOGGER.info("HMSFollower is being initialized");
     authzConf = conf;
     this.leaderMonitor = leaderMonitor;
     sentryStore = store;
-   if (authServerName == null) {
-     HiveConf hiveConf = new HiveConf();
-     authServerName = hiveConf.get(HiveAuthzConf.AuthzConfVars.AUTHZ_SERVER_NAME.getVar());
-   }
+
+    if (authServerName == null) {
+      authServerName = conf.get(AUTHZ_SERVER_NAME.getVar(),
+        conf.get(AUTHZ_SERVER_NAME_DEPRECATED.getVar(), AUTHZ_SERVER_NAME_DEPRECATED.getDefault()));
+    }
+
     notificationProcessor = new NotificationProcessor(sentryStore, authServerName, authzConf);
     client = new SentryHMSClient(authzConf, hiveConnectionFactory);
   }
@@ -135,6 +136,11 @@ public class HMSFollower implements Runnable, AutoCloseable {
     return (leaderMonitor == null) || leaderMonitor.isLeader();
   }
 
+  @VisibleForTesting
+  String getAuthServerName() {
+    return notificationProcessor.getAuthServerName();
+  }
+
   /**
    * Processes new Hive Metastore notifications.
    *
@@ -301,11 +307,12 @@ public class HMSFollower implements Runnable, AutoCloseable {
    * @param events list of event to be processed
    * @throws Exception if the complete notification list is not processed because of JDO Exception
    */
-  void processNotifications(Collection<NotificationEvent> events) throws Exception {
+  public void processNotifications(Collection<NotificationEvent> events) throws Exception {
     boolean isNotificationProcessed;
     if (events.isEmpty()) {
       return;
     }
+
     for (NotificationEvent event : events) {
       isNotificationProcessed = false;
       try {

http://git-wip-us.apache.org/repos/asf/sentry/blob/66686420/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HiveSimpleConnectionFactory.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HiveSimpleConnectionFactory.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HiveSimpleConnectionFactory.java
index 3d67401..77634cf 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HiveSimpleConnectionFactory.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HiveSimpleConnectionFactory.java
@@ -56,7 +56,7 @@ public final class HiveSimpleConnectionFactory implements HiveConnectionFactory
   private final boolean insecure;
   private SentryKerberosContext kerberosContext = null;
 
-  HiveSimpleConnectionFactory(Configuration sentryConf, HiveConf hiveConf) {
+  public HiveSimpleConnectionFactory(Configuration sentryConf, HiveConf hiveConf) {
     this.conf = sentryConf;
     this.hiveConf = hiveConf;
     insecure = !ServerConfig.SECURITY_MODE_KERBEROS.equalsIgnoreCase(

http://git-wip-us.apache.org/repos/asf/sentry/blob/66686420/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
index 62fde2c..1f34d05 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
@@ -124,6 +124,11 @@ final class NotificationProcessor {
     return update;
   }
 
+  @VisibleForTesting
+  String getAuthServerName() {
+    return authServerName;
+  }
+
   /**
    * Constructs permission update to be persisted for rename event that can be persisted from thrift
    * object.

http://git-wip-us.apache.org/repos/asf/sentry/blob/66686420/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java
new file mode 100644
index 0000000..82f600b
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java
@@ -0,0 +1,304 @@
+/**
+ * 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.provider.db.service.persistent;
+
+import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SERVER_NAME;
+
+import com.google.common.io.Files;
+import java.io.File;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.UserProvider;
+import org.apache.hive.hcatalog.messaging.HCatEventMessage;
+import org.apache.hive.hcatalog.messaging.HCatEventMessage.EventType;
+import org.apache.sentry.binding.metastore.messaging.json.SentryJSONMessageFactory;
+import org.apache.sentry.core.common.exception.SentryNoSuchObjectException;
+import org.apache.sentry.provider.db.service.thrift.TSentryAuthorizable;
+
+import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
+import org.apache.sentry.service.thrift.HiveSimpleConnectionFactory;
+import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.service.thrift.HMSFollower;
+import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test integration of HMSFollower with SentryStore
+ */
+public class TestHMSFollowerSentryStoreIntegration {
+
+  // SentryStore related member
+  private static File dataDir;
+  private static SentryStore sentryStore;
+  private static String[] adminGroups = { "adminGroup1" };
+  private static PolicyFile policyFile;
+  private static File policyFilePath;
+  private static Configuration conf = null;
+  private static char[] passwd = new char[] { '1', '2', '3'};
+  private static String dbName1 = "db1";
+  private static String tableName1 = "table1";
+  private static String serverName1 = "server1";
+
+  // HMSFollower related member
+  SentryJSONMessageFactory messageFactory = new SentryJSONMessageFactory();
+  final static String hiveInstance = serverName1;
+
+
+  // SentryStore related
+  @BeforeClass
+  public static void setup() throws Exception {
+    conf = new Configuration(false);
+    final String ourUrl = UserProvider.SCHEME_NAME + ":///";
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    // THis should be a UserGroupInformation provider
+    CredentialProvider provider = CredentialProviderFactory.getProviders(conf).get(0);
+
+
+    // The user credentials are stored as a static variable by UserGrouoInformation provider.
+    // We need to only set the password the first time, an attempt to set it for the second
+    // time fails with an exception.
+    if(provider.getCredentialEntry(ServerConfig.SENTRY_STORE_JDBC_PASS) == null) {
+      provider.createCredentialEntry(ServerConfig.SENTRY_STORE_JDBC_PASS, passwd);
+      provider.flush();
+    }
+
+    dataDir = new File(Files.createTempDir(), "sentry_policy_db");
+    conf.set(ServerConfig.SENTRY_VERIFY_SCHEM_VERSION, "false");
+    conf.set(ServerConfig.SENTRY_STORE_JDBC_URL,
+        "jdbc:derby:;databaseName=" + dataDir.getPath() + ";create=true");
+    conf.set(ServerConfig.SENTRY_STORE_JDBC_PASS, "dummy");
+    conf.setStrings(ServerConfig.ADMIN_GROUPS, adminGroups);
+    conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING,
+        ServerConfig.SENTRY_STORE_LOCAL_GROUP_MAPPING);
+    policyFilePath = new File(dataDir, "local_policy_file.ini");
+    conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE,
+        policyFilePath.getPath());
+    conf.setInt(ServerConfig.SENTRY_STORE_TRANSACTION_RETRY, 10);
+
+  }
+
+  @Before
+  public void before() throws Exception {
+    sentryStore = new SentryStore(conf);
+    policyFile = new PolicyFile();
+    String adminUser = "g1";
+    addGroupsToUser(adminUser, adminGroups);
+    writePolicyFile();
+  }
+
+  @After
+  public void after() {
+    if (sentryStore != null) {
+      sentryStore.clearAllTables();
+      sentryStore.stop();
+    }
+  }
+
+  @AfterClass
+  public static void teardown() {
+
+    if (dataDir != null) {
+      FileUtils.deleteQuietly(dataDir);
+    }
+  }
+
+  /**
+   * Fail test if role already exists
+   * @param roleName Role name to checl
+   * @throws Exception
+   */
+  private void checkRoleDoesNotExist(String roleName) throws Exception {
+    try {
+      sentryStore.getMSentryRoleByName(roleName);
+      Assert.fail("Role " + roleName + "already exists");
+    } catch (SentryNoSuchObjectException e) {
+      // Ok
+    }
+  }
+
+  /**
+   * Fail test if role doesn't exist
+   * @param roleName Role name to checl
+   * @throws Exception
+   */
+  private void checkRoleExists(String roleName) throws Exception {
+    Assert.assertEquals(roleName.toLowerCase(),
+        sentryStore.getMSentryRoleByName(roleName).getRoleName());
+  }
+
+  /**
+   * Create a role with the given name and verify that it is created
+   * @param roleName
+   * @throws Exception
+   */
+  private void createRole(String roleName) throws Exception {
+    checkRoleDoesNotExist(roleName);
+    sentryStore.createSentryRole(roleName);
+    checkRoleExists(roleName);
+  }
+
+  private TSentryAuthorizable toTSentryAuthorizable(
+      TSentryPrivilege tSentryPrivilege) {
+    TSentryAuthorizable tSentryAuthorizable = new TSentryAuthorizable();
+    tSentryAuthorizable.setServer(tSentryPrivilege.getServerName());
+    tSentryAuthorizable.setDb(tSentryPrivilege.getDbName());
+    tSentryAuthorizable.setTable(tSentryPrivilege.getTableName());
+    tSentryAuthorizable.setUri(tSentryPrivilege.getURI());
+    return tSentryAuthorizable;
+  }
+
+  protected static void addGroupsToUser(String user, String... groupNames) {
+    policyFile.addGroupsToUser(user, groupNames);
+  }
+
+  protected static void writePolicyFile() throws Exception {
+    policyFile.write(policyFilePath);
+  }
+
+  /**
+   * Test that SentryStore droppes the permission associated with the table when HMSFollower
+   * processes the drop table event
+   * @throws Exception
+   */
+  @Test
+  public void testDropTableDropRelatedPermission() throws Exception {
+    String serverName = "server1";
+
+    // create HMSFollower
+    Configuration configuration = new Configuration();
+    configuration.set(AUTHZ_SERVER_NAME.getVar(), serverName);
+    HMSFollower hmsFollower = new HMSFollower(configuration, sentryStore, null,
+      new HiveSimpleConnectionFactory(conf, new HiveConf()), null);
+
+    // configure permission of the table
+    String roleName1 = "list-privs-r1";
+    String grantor = "g1";
+    sentryStore.createSentryRole(roleName1);
+
+    TSentryPrivilege privilege_tbl1 = new TSentryPrivilege();
+    privilege_tbl1.setPrivilegeScope("TABLE");
+    privilege_tbl1.setServerName(serverName);
+    privilege_tbl1.setDbName(dbName1);
+    privilege_tbl1.setTableName(tableName1);
+    privilege_tbl1.setCreateTime(System.currentTimeMillis());
+
+    TSentryPrivilege privilege1 = new TSentryPrivilege(privilege_tbl1);
+    privilege1.setAction("SELECT");
+
+    TSentryPrivilege privilege1_2 = new TSentryPrivilege(privilege_tbl1);
+    privilege1_2.setAction("INSERT");
+    TSentryPrivilege privilege1_3 = new TSentryPrivilege(privilege_tbl1);
+    privilege1_3.setAction("*");
+
+    TSentryPrivilege privilege_server = new TSentryPrivilege();
+    privilege_server.setPrivilegeScope("SERVER");
+    privilege_server.setServerName(serverName1);
+    privilege_server.setCreateTime(System.currentTimeMillis());
+
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege1);
+
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege1_2);
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege_server);
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege1_3);
+
+    // Create notification events to drop the table
+    StorageDescriptor sd = new StorageDescriptor();
+    sd.setLocation("hdfs:///db1.db/table1");
+    NotificationEvent notificationEvent = new NotificationEvent(1, 0, HCatEventMessage.EventType.DROP_TABLE.toString(),
+        messageFactory.buildDropTableMessage(new Table(tableName1, dbName1, null, 0, 0, 0, sd, null, null, null, null, null)).toString());
+    List<NotificationEvent> events = new ArrayList<>();
+    events.add(notificationEvent);
+
+    hmsFollower.processNotifications(events);
+
+    Assert.assertEquals(1, sentryStore.getAllTSentryPrivilegesByRoleName(roleName1)
+        .size());
+  }
+
+  /**
+   * Test that SentryStore droppes the permission associated with the database when HMSFollower
+   * processes the drop database event
+   * @throws Exception
+   */
+  @Test
+  public void testDropDatabaseDropRelatedPermission() throws Exception {
+    String serverName = "server1";
+
+    // create HMSFollower
+    Configuration configuration = new Configuration();
+    configuration.set(AUTHZ_SERVER_NAME.getVar(), serverName);
+    HMSFollower hmsFollower = new HMSFollower(configuration, sentryStore, null,
+        new HiveSimpleConnectionFactory(conf, new HiveConf()), null);
+
+    // configure permission of the database
+    String roleName1 = "list-privs-r1";
+    String grantor = "g1";
+    sentryStore.createSentryRole(roleName1);
+
+    TSentryPrivilege privilege_tbl1 = new TSentryPrivilege();
+    privilege_tbl1.setPrivilegeScope("DATABASE");
+    privilege_tbl1.setServerName(serverName);
+    privilege_tbl1.setDbName(dbName1);
+    privilege_tbl1.setTableName(tableName1);
+    privilege_tbl1.setCreateTime(System.currentTimeMillis());
+
+    TSentryPrivilege privilege1 = new TSentryPrivilege(privilege_tbl1);
+    privilege1.setAction("SELECT");
+
+    TSentryPrivilege privilege1_2 = new TSentryPrivilege(privilege_tbl1);
+    privilege1_2.setAction("INSERT");
+    TSentryPrivilege privilege1_3 = new TSentryPrivilege(privilege_tbl1);
+    privilege1_3.setAction("*");
+
+    TSentryPrivilege privilege_server = new TSentryPrivilege();
+    privilege_server.setPrivilegeScope("SERVER");
+    privilege_server.setServerName(serverName1);
+    privilege_server.setCreateTime(System.currentTimeMillis());
+
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege1);
+
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege1_2);
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege_server);
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege1_3);
+
+    // Create notification events to drop the database
+    NotificationEvent notificationEvent = new NotificationEvent(1, 0, EventType.DROP_DATABASE.toString(),
+        messageFactory.buildDropDatabaseMessage(new Database(dbName1, null, "hdfs:///" + dbName1, null)).toString());
+    List<NotificationEvent> events = new ArrayList<>();
+    events.add(notificationEvent);
+
+    hmsFollower.processNotifications(events);
+
+    Assert.assertEquals(1, sentryStore.getAllTSentryPrivilegesByRoleName(roleName1)
+        .size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/66686420/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java
index fdf52bf..9b31b3c 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java
@@ -28,6 +28,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
@@ -38,6 +39,8 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hive.hcatalog.messaging.HCatEventMessage;
 import org.apache.hive.hcatalog.messaging.HCatEventMessage.EventType;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
 import org.apache.sentry.binding.metastore.messaging.json.SentryJSONMessageFactory;
 import org.apache.sentry.hdfs.Updateable;
 import org.apache.sentry.provider.db.service.persistent.PathsImage;
@@ -190,7 +193,8 @@ public class TestHMSFollower {
     hmsFollower.setSentryHmsClient(sentryHmsClient);
 
     // 1st run should get a full snapshot
-    Mockito.when(sentryStore.getLastProcessedNotificationID()).thenReturn(SENTRY_PROCESSED_EVENT_ID);
+    Mockito.when(sentryStore.getLastProcessedNotificationID())
+        .thenReturn(SENTRY_PROCESSED_EVENT_ID);
     Mockito.when(sentryStore.isAuthzPathsMappingEmpty()).thenReturn(false);
     hmsFollower.run();
     Mockito.verify(sentryStore, times(1)).persistFullPathsImage(Mockito.anyMap());
@@ -207,6 +211,66 @@ public class TestHMSFollower {
   }
 
   /**
+   * Test that HMSFollower uses the input authentication server name when it is not null
+   */
+  @Test
+  public void testInputConfigurationGetInputAuthServerName() {
+    Configuration sentryConfiguration = new Configuration();
+    HMSFollower hmsFollower = new HMSFollower(sentryConfiguration, sentryStore, null,
+        hiveConnectionFactory, hiveInstance);
+    String authServerName = hmsFollower.getAuthServerName();
+
+    Assert.assertEquals(true, authServerName.equals(hiveInstance));
+  }
+
+  /**
+   * Test that HMSFollower uses the default authentication server name when its constructor input
+   * value is null and the configuration does not configure AUTHZ_SERVER_NAME nor
+   * AUTHZ_SERVER_NAME_DEPRECATED
+   */
+  @Test
+  public void testNoConfigurationGetDefaultAuthServerName() {
+    Configuration sentryConfiguration = new Configuration();
+    HMSFollower hmsFollower = new HMSFollower(sentryConfiguration, sentryStore, null,
+        hiveConnectionFactory, null);
+    String authServerName = hmsFollower.getAuthServerName();
+
+    Assert.assertEquals(true, authServerName.equals(AuthzConfVars.AUTHZ_SERVER_NAME_DEPRECATED.getDefault()));
+  }
+
+  /**
+   * Test that HMSFollower uses the configured authentication server name when its constructor input
+   * value is null and the configuration contains configuration for AUTHZ_SERVER_NAME
+   */
+  @Test
+  public void testNewNameConfigurationGetAuthServerName() {
+    String serverName = "newServer";
+    Configuration sentryConfiguration = new Configuration();
+    sentryConfiguration.set(HiveAuthzConf.AuthzConfVars.AUTHZ_SERVER_NAME.getVar(), serverName);
+    HMSFollower hmsFollower = new HMSFollower(sentryConfiguration, sentryStore, null,
+        hiveConnectionFactory, null);
+    String authServerName = hmsFollower.getAuthServerName();
+
+    Assert.assertEquals(true, authServerName.equals(serverName));
+  }
+
+  /**
+   * Test that HMSFollower uses the configured deprecated authentication server name when its constructor input
+   * value is null and the configuration contains configuration for AUTHZ_SERVER_NAME_DEPRECATED
+   */
+  @Test
+  public void testOldNameConfigurationGetAuthServerName() {
+    String serverName = "oldServer";
+    Configuration sentryConfiguration = new Configuration();
+    sentryConfiguration.set(AuthzConfVars.AUTHZ_SERVER_NAME_DEPRECATED.getVar(), serverName);
+    HMSFollower hmsFollower = new HMSFollower(sentryConfiguration, sentryStore, null,
+        hiveConnectionFactory, null);
+    String authServerName = hmsFollower.getAuthServerName();
+
+    Assert.assertEquals(true, authServerName.equals(serverName));
+  }
+
+  /**
    * Constructs create database event and makes sure that appropriate sentry store API's
    * are invoke when the event is processed by hms follower.
    *