You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ka...@apache.org on 2017/08/03 14:07:44 UTC

sentry git commit: SENTRY-1847: Integrate sentry with Hive 2.0.0. (kalyan kumar kalvagadda reviewed by Colm O hEigeartaigh)

Repository: sentry
Updated Branches:
  refs/heads/master a68386e3a -> 67d64d0d5


SENTRY-1847: Integrate sentry with Hive 2.0.0. (kalyan kumar kalvagadda reviewed by Colm O hEigeartaigh)


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

Branch: refs/heads/master
Commit: 67d64d0d57ebf108b2be62b674dfac89aaf8d0c9
Parents: a68386e
Author: Kalyan Kumar Kalvagadda <kk...@cloudera.com>
Authored: Thu Aug 3 09:07:33 2017 -0500
Committer: Kalyan Kumar Kalvagadda <kk...@cloudera.com>
Committed: Thu Aug 3 09:07:33 2017 -0500

----------------------------------------------------------------------
 .../json/SentryJSONAlterPartitionMessage.java   |   7 +-
 .../json/SentryJSONMessageDeserializer.java     |  16 +++
 .../json/SentryJSONMessageFactory.java          | 101 ++++++++-----------
 .../json/SentryJSONMessageFactory.java          |   4 +
 .../DefaultSentryAccessController.java          |   9 +-
 .../metastore/MetastoreAuthzBindingBaseV2.java  |  22 ++--
 .../SentryMetastorePostEventListenerBaseV2.java |  15 ++-
 .../sentry/service/thrift/TestHMSFollower.java  |   6 +-
 .../tests/e2e/hdfs/TestHDFSIntegration.java     |   5 -
 9 files changed, 102 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONAlterPartitionMessage.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONAlterPartitionMessage.java b/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONAlterPartitionMessage.java
index b29d727..64c1515 100644
--- a/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONAlterPartitionMessage.java
+++ b/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONAlterPartitionMessage.java
@@ -18,11 +18,12 @@
 
 package org.apache.sentry.binding.metastore.messaging.json;
 
-import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import org.apache.hive.hcatalog.messaging.json.JSONAlterPartitionMessage;
 import org.codehaus.jackson.annotate.JsonProperty;
 
 import java.util.List;
+import java.util.Map;
 
 public class SentryJSONAlterPartitionMessage extends JSONAlterPartitionMessage {
   @JsonProperty
@@ -33,12 +34,12 @@ public class SentryJSONAlterPartitionMessage extends JSONAlterPartitionMessage {
   private List<String> newValues;
 
   public SentryJSONAlterPartitionMessage() {
-    super("", "", "", "", ImmutableList.<String>of(), null);
+    super("", "", "", "", ImmutableMap.<String, String>of(), null);
   }
 
   public SentryJSONAlterPartitionMessage(String server, String servicePrincipal,
                                          String db, String table,
-                                         List<String> values, List<String> newValues,
+                                         Map<String, String> values, List<String> newValues,
                                          Long timestamp, String oldlocation,
                                          String newLocation) {
     super(server, servicePrincipal, db, table, values, timestamp);

http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageDeserializer.java b/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageDeserializer.java
index cc0bbec..d11b261 100644
--- a/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageDeserializer.java
+++ b/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageDeserializer.java
@@ -19,6 +19,7 @@
 package org.apache.sentry.binding.metastore.messaging.json;
 
 import org.apache.hive.hcatalog.messaging.*;
+import org.apache.hive.hcatalog.messaging.json.JSONInsertMessage;
 import org.codehaus.jackson.map.DeserializationConfig;
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -128,6 +129,21 @@ public class SentryJSONMessageDeserializer extends MessageDeserializer {
     }
   }
 
+  /**
+   * Method to de-serialize JSONInsertMessage instance.
+   */
+  @Override
+  public InsertMessage getInsertMessage(String messageBody) {
+    // Sentry would be not be interested in InsertMessage as these are generated when is data is
+    // added inserted. This method is implemented for completeness. This is reason why, new sentry
+    // JSON class is not defined for InsertMessage.
+    try {
+      return mapper.readValue(messageBody, JSONInsertMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct InsertMessage", e);
+    }
+  }
+
   public static String serialize(Object object) {
     try {
       return mapper.writeValueAsString(object);

http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java b/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java
index f173243..19310b8 100644
--- a/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java
+++ b/sentry-binding/sentry-binding-hive-follower-v2/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java
@@ -21,13 +21,11 @@ package org.apache.sentry.binding.metastore.messaging.json;
 import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hive.hcatalog.messaging.*;
+import org.apache.hive.hcatalog.messaging.json.JSONInsertMessage;
 
 import java.util.*;
 
@@ -76,48 +74,59 @@ public class SentryJSONMessageFactory extends MessageFactory {
         table.getTableName(), now(), table.getSd().getLocation());
   }
 
-  public SentryJSONAddPartitionMessage buildAddPartitionMessage(Table table, List<Partition> partitions) {
-    return new SentryJSONAddPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, table.getDbName(),
-        table.getTableName(), getPartitionKeyValues(table, partitions), now(),
-        getPartitionLocations(partitions));
+  @Override
+  public SentryJSONAlterPartitionMessage buildAlterPartitionMessage(Table table,
+      Partition before, Partition after) {
+    return new SentryJSONAlterPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL,
+        before.getDbName(), before.getTableName(), getPartitionKeyValues(table, before),
+        after.getValues(), now(), before.getSd().getLocation(), after.getSd().getLocation());
   }
 
-  private List<String> getPartitionLocations(List<Partition> partitions) {
-    List<String> paths = Lists.newLinkedList();
-    for (Partition partition : partitions) {
-      paths.add(partition.getSd().getLocation());
-    }
-    return paths;
+  @Override
+  public DropPartitionMessage buildDropPartitionMessage(Table table, Iterator<Partition> partitions) {
+    return new SentryJSONDropPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL,
+        table.getDbName(), table.getTableName(),getPartitionKeyValues(table, partitions),
+        now(), getPartitionLocations(partitions));
   }
 
-  private List<String> getPartitionLocations(PartitionSpecProxy partitionSpec) {
-    Iterator<Partition> iterator = partitionSpec.getPartitionIterator();
-    List<String> locations = Lists.newLinkedList();
-    while (iterator.hasNext()) {
-      locations.add(iterator.next().getSd().getLocation());
-    }
-    return locations;
+  @Override
+  public InsertMessage buildInsertMessage(String db, String table, Map<String,String> partKeyVals,
+      List<String> files) {
+    // Sentry would be not be interested in InsertMessage as these are generated when is data is
+    // added inserted. This method is implemented for completeness. This is reason why, new sentry
+    // JSON class is not defined for InsertMessage.
+    return new JSONInsertMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, db, table, partKeyVals,
+        files, now());
   }
 
-  @InterfaceAudience.LimitedPrivate( {"Hive"})
-  @InterfaceStability.Evolving
-  public SentryJSONAddPartitionMessage buildAddPartitionMessage(Table table, PartitionSpecProxy partitionSpec) {
+  @Override
+  public AddPartitionMessage buildAddPartitionMessage(Table table,
+      Iterator<Partition> partitionsIterator) {
     return new SentryJSONAddPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, table.getDbName(),
-        table.getTableName(), getPartitionKeyValues(table, partitionSpec), now(),
-        getPartitionLocations(partitionSpec));
+        table.getTableName(), getPartitionKeyValues(table, partitionsIterator), now(),
+        getPartitionLocations(partitionsIterator));
   }
 
-  @Override
-  public SentryJSONAlterPartitionMessage buildAlterPartitionMessage(Partition before, Partition after) {
-    return new SentryJSONAlterPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, before.getDbName(),
-        before.getTableName(), before.getValues(), after.getValues(), now(), before.getSd().getLocation(),
-        after.getSd().getLocation());
+  public AddPartitionMessage buildAddPartitionMessage(Table table,
+                                                      List<Partition> partitions) {
+    return buildAddPartitionMessage (table, partitions.iterator());
   }
 
-  public SentryJSONDropPartitionMessage buildDropPartitionMessage(Table table, Partition partition) {
-    return new SentryJSONDropPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, partition.getDbName(),
-        partition.getTableName(), Arrays.asList(getPartitionKeyValues(table, partition)),
-        now(), Arrays.asList(partition.getSd().getLocation()));
+  private static List<Map<String, String>> getPartitionKeyValues(Table table,
+      Iterator<Partition> partitions) {
+    List<Map<String, String>> partitionList = Lists.newLinkedList();
+    while(partitions.hasNext()) {
+      partitionList.add(getPartitionKeyValues(table, partitions.next()));
+    }
+    return partitionList;
+  }
+
+  private List<String> getPartitionLocations(Iterator<Partition> partitionsIterator) {
+    List<String> locations = Lists.newLinkedList();
+    while(partitionsIterator.hasNext()) {
+      locations.add(partitionsIterator.next().getSd().getLocation());
+    }
+    return locations;
   }
 
   private static Map<String, String> getPartitionKeyValues(Table table, Partition partition) {
@@ -130,30 +139,6 @@ public class SentryJSONMessageFactory extends MessageFactory {
     return partitionKeys;
   }
 
-  private static List<Map<String, String>> getPartitionKeyValues(Table table, List<Partition> partitions) {
-    List<Map<String, String>> partitionList = Lists.newLinkedList();
-
-    for (Partition partition : partitions) {
-      partitionList.add(getPartitionKeyValues(table, partition));
-    }
-
-    return partitionList;
-  }
-
-  @InterfaceAudience.LimitedPrivate( {"Hive"})
-  @InterfaceStability.Evolving
-  private static List<Map<String, String>> getPartitionKeyValues(Table table, PartitionSpecProxy partitionSpec) {
-    ArrayList partitionList = new ArrayList();
-    PartitionSpecProxy.PartitionIterator iterator = partitionSpec.getPartitionIterator();
-
-    while (iterator.hasNext()) {
-      Partition partition = iterator.next();
-      partitionList.add(getPartitionKeyValues(table, partition));
-    }
-
-    return partitionList;
-  }
-
   //This is private in parent class
   private long now() {
     return System.currentTimeMillis() / 1000L;

http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/sentry-binding/sentry-binding-hive-follower/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-follower/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java b/sentry-binding/sentry-binding-hive-follower/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java
index f173243..efdf8b8 100644
--- a/sentry-binding/sentry-binding-hive-follower/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java
+++ b/sentry-binding/sentry-binding-hive-follower/src/main/java/org/apache/sentry/binding/metastore/messaging/json/SentryJSONMessageFactory.java
@@ -114,6 +114,10 @@ public class SentryJSONMessageFactory extends MessageFactory {
         after.getSd().getLocation());
   }
 
+  public SentryJSONAlterPartitionMessage buildAlterPartitionMessage(Table table, Partition before, Partition after) {
+    return buildAlterPartitionMessage(before, after);
+  }
+
   public SentryJSONDropPartitionMessage buildDropPartitionMessage(Table table, Partition partition) {
     return new SentryJSONDropPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, partition.getDbName(),
         partition.getTableName(), Arrays.asList(getPartitionKeyValues(table, partition)),

http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java
index c5a2f42..456c8d6 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryAccessController.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilege;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeInfo;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveRoleGrant;
-import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.sentry.binding.hive.SentryOnFailureHookContext;
 import org.apache.sentry.binding.hive.SentryOnFailureHookContextImpl;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
@@ -514,8 +513,14 @@ public class DefaultSentryAccessController extends SentryHiveAccessController {
 
   private void executeOnFailureHooks(HiveOperation hiveOp, SentryAccessDeniedException e)
       throws HiveAccessControlException {
+
+    // With Hive 2.x cmd information is not available from SessionState. More over cmd information
+    // is not used in SentryOnFailureHookContextImpl. If this information is really needed an issue
+    // should be raised with  Hive community to update HiveAccessController interface to pass
+    // HiveSemanticAnalyzerHookContext, which has cmd information. For now, empty string is used for
+    // cmd.
     SentryOnFailureHookContext hookCtx =
-        new SentryOnFailureHookContextImpl(SessionState.get().getCmd(), null, null, hiveOp, null,
+        new SentryOnFailureHookContextImpl("", null, null, hiveOp, null,
             null, null, null, authenticator.getUserName(), null, new AuthorizationException(e),
             authzConf);
     SentryAuthorizerUtil.executeOnFailureHooks(hookCtx, authzConf);

http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingBaseV2.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingBaseV2.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingBaseV2.java
index 67413fa..88fa4a1 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingBaseV2.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingBaseV2.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.util.Iterator;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
@@ -337,14 +338,19 @@ public abstract class MetastoreAuthzBindingBaseV2 extends MetaStorePreEventListe
 
   protected void authorizeDropPartition(PreDropPartitionEvent context)
       throws InvalidOperationException, MetaException {
+    Iterator<Partition> partitionIterator = context.getPartitionIterator();
+    HierarcyBuilder inputHierarchy = new HierarcyBuilder();
+    HierarcyBuilder outputHierarchy = new HierarcyBuilder();
+    Partition partition;
+    while(partitionIterator.hasNext()) {
+      partition = partitionIterator.next();
+      inputHierarchy.addTableToOutput(getAuthServer(), partition.getDbName(),
+          partition.getTableName());
+      outputHierarchy.addTableToOutput(getAuthServer(), partition.getDbName(),
+          partition.getTableName());
+    }
     authorizeMetastoreAccess(
-        HiveOperation.ALTERTABLE_DROPPARTS,
-        new HierarcyBuilder().addTableToOutput(getAuthServer(),
-            context.getPartition().getDbName(),
-            context.getPartition().getTableName()).build(),
-        new HierarcyBuilder().addTableToOutput(getAuthServer(),
-            context.getPartition().getDbName(),
-            context.getPartition().getTableName()).build());
+        HiveOperation.ALTERTABLE_DROPPARTS, inputHierarchy.build(), outputHierarchy.build());
   }
 
   private void authorizeAlterPartition(PreAlterPartitionEvent context)
@@ -447,7 +453,7 @@ public abstract class MetastoreAuthzBindingBaseV2 extends MetaStorePreEventListe
   }
 
   public static void setSentryCacheOutOfSync(boolean sentryCacheOutOfSync) {
-    MetastoreAuthzBindingBase.sentryCacheOutOfSync = sentryCacheOutOfSync;
+    MetastoreAuthzBindingBaseV2.sentryCacheOutOfSync = sentryCacheOutOfSync;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerBaseV2.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerBaseV2.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerBaseV2.java
index 918bf8f..567e9fa 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerBaseV2.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerBaseV2.java
@@ -19,6 +19,7 @@ package org.apache.sentry.binding.metastore;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -276,7 +277,9 @@ public class SentryMetastorePostEventListenerBaseV2 extends MetaStoreEventListen
       return;
     }
 
-    for (Partition part : partitionEvent.getPartitions()) {
+    Iterator<Partition> partitionIterator = partitionEvent.getPartitionIterator();
+    while(partitionIterator.hasNext()) {
+      Partition part = partitionIterator.next();
       if (part.getSd() != null && part.getSd().getLocation() != null) {
         String authzObj = part.getDbName() + "." + part.getTableName();
         String path = part.getSd().getLocation();
@@ -301,9 +304,13 @@ public class SentryMetastorePostEventListenerBaseV2 extends MetaStoreEventListen
 
     String authzObj = partitionEvent.getTable().getDbName() + "."
         + partitionEvent.getTable().getTableName();
-    String path = partitionEvent.getPartition().getSd().getLocation();
-    for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
-      plugin.removePath(authzObj, path);
+    Iterator<Partition> partitionIterator = partitionEvent.getPartitionIterator();
+    while (partitionIterator.hasNext()) {
+      Partition part = partitionIterator.next();
+      String path = part.getSd().getLocation();
+      for (SentryMetastoreListenerPlugin plugin : sentryPlugins) {
+        plugin.removePath(authzObj, path);
+      }
     }
     super.onDropPartition(partitionEvent);
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/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 9b31b3c..23f8de8 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
@@ -506,7 +506,7 @@ public class TestHMSFollower {
         0, 0, sd, null);
     partitions.add(partition);
     notificationEvent = new NotificationEvent(inputEventId, 0, EventType.ADD_PARTITION.toString(),
-        messageFactory.buildAddPartitionMessage(table, partitions).toString());
+       messageFactory.buildAddPartitionMessage(table, partitions).toString());
     notificationEvent.setDbName(dbName);
     notificationEvent.setTableName(tableName1);
     events.add(notificationEvent);
@@ -526,7 +526,7 @@ public class TestHMSFollower {
     // This is an invalid event and should be processed by sentry store.
     // Event Id should be explicitly persisted using persistLastProcessedNotificationID
     notificationEvent = new NotificationEvent(inputEventId, 0, EventType.ALTER_PARTITION.toString(),
-        messageFactory.buildAlterPartitionMessage(partition, partition).toString());
+        messageFactory.buildAlterPartitionMessage(table, partition, partition).toString());
     notificationEvent.setDbName(dbName);
     notificationEvent.setTableName(tableName1);
     events.add(notificationEvent);
@@ -544,7 +544,7 @@ public class TestHMSFollower {
     Partition updatedPartition = new Partition(partition);
     updatedPartition.setSd(sd);
     notificationEvent = new NotificationEvent(inputEventId, 0, EventType.ALTER_PARTITION.toString(),
-        messageFactory.buildAlterPartitionMessage(partition, updatedPartition).toString());
+      messageFactory.buildAlterPartitionMessage(table, partition, updatedPartition).toString());
     notificationEvent.setDbName(dbName);
     notificationEvent.setTableName(tableName1);
     events.add(notificationEvent);

http://git-wip-us.apache.org/repos/asf/sentry/blob/67d64d0d/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java b/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java
index 85902ed..c19ccbd 100644
--- a/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java
+++ b/sentry-tests/sentry-tests-hive-v2/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java
@@ -67,13 +67,9 @@ import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
-import org.apache.sentry.binding.hive.v2.HiveAuthzBindingSessionHookV2;
 import org.apache.sentry.binding.hive.v2.SentryHiveAuthorizationTaskFactoryImplV2;
-import org.apache.sentry.binding.hive.v2.metastore.MetastoreAuthzBindingV2;
-import org.apache.sentry.binding.hive.v2.metastore.SentryMetastorePostEventListenerV2;
 import org.apache.sentry.hdfs.PathsUpdate;
 import org.apache.sentry.hdfs.SentryINodeAttributesProvider;
-import org.apache.sentry.core.common.exception.SentryAlreadyExistsException;
 import org.apache.sentry.provider.db.SimpleDBProviderBackend;
 import org.apache.sentry.provider.file.LocalGroupResourceAuthorizationProvider;
 import org.apache.sentry.provider.file.PolicyFile;
@@ -405,7 +401,6 @@ public class TestHDFSIntegration {
           properties.put(ServerConfig.RPC_ADDRESS, "localhost");
           properties.put(ServerConfig.RPC_PORT, String.valueOf(sentryPort > 0 ? sentryPort : 0));
           properties.put(ServerConfig.SENTRY_VERIFY_SCHEM_VERSION, "false");
-          properties.put(ServerConfig.SENTRY_NOTIFICATION_LOG_ENABLED,"true");
 
           properties.put(ServerConfig.SENTRY_STORE_GROUP_MAPPING, ServerConfig.SENTRY_STORE_LOCAL_GROUP_MAPPING);
           properties.put(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE, policyFileLocation.getPath());