You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by ma...@apache.org on 2016/11/04 00:13:23 UTC

[2/3] incubator-ranger git commit: Added agents-common module changes, version tracking table creation schema for Postgres, database interface classes and code to update database table. Addressed review comments

Added agents-common module changes, version tracking table creation schema for Postgres, database interface classes and code to update database table. Addressed review comments

Signed-off-by: Madhan Neethiraj <ma...@apache.org>


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

Branch: refs/heads/master
Commit: 3bbc3702df19ab23e9efab5ca1730e1827a35589
Parents: 3d6476b
Author: Abhay Kulkarni <ak...@hortonworks.com>
Authored: Mon Aug 1 15:00:35 2016 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Thu Nov 3 17:11:28 2016 -0700

----------------------------------------------------------------------
 .../ranger/admin/client/RangerAdminClient.java  |   4 +-
 .../admin/client/RangerAdminRESTClient.java     |  27 +-
 .../RangerAdminTagRetriever.java                |   4 +-
 .../RangerFileBasedTagRetriever.java            |   6 +-
 .../contextenricher/RangerTagEnricher.java      |  17 +-
 .../RangerTagFileStoreRetriever.java            |   3 +-
 .../contextenricher/RangerTagRetriever.java     |   2 +-
 .../model/RangerPluginServiceVersionInfo.java   | 192 +++++++++++++
 .../ranger/plugin/util/PolicyRefresher.java     |  16 +-
 .../ranger/plugin/util/RangerRESTUtils.java     |  34 +++
 .../apache/ranger/plugin/util/SearchFilter.java |   6 +
 .../services/hdfs/RangerAdminClientImpl.java    |   4 +-
 .../services/hive/RangerAdminClientImpl.java    |   4 +-
 .../client/RangerAdminJersey2RESTClient.java    |  10 +-
 .../kms/authorizer/RangerAdminClientImpl.java   |   6 +-
 ...hema-for-plugin-service-version-tracking.sql |  36 +++
 ...hema-for-plugin-service-version-tracking.sql |  38 +++
 .../java/org/apache/ranger/biz/AssetMgr.java    | 211 +++++++++++----
 .../apache/ranger/common/RangerSearchUtil.java  |   5 +
 ...RangerTransactionSynchronizationAdapter.java | 118 ++++++++
 .../apache/ranger/db/RangerDaoManagerBase.java  |   7 +
 .../db/XXPluginServiceVersionInfoDao.java       |  95 +++++++
 .../entity/XXPluginServiceVersionInfo.java      | 271 +++++++++++++++++++
 .../java/org/apache/ranger/rest/AssetREST.java  |   2 +-
 .../org/apache/ranger/rest/PublicAPIsv2.java    |  17 ++
 .../org/apache/ranger/rest/ServiceREST.java     |  92 +++++--
 .../java/org/apache/ranger/rest/TagREST.java    |  55 +++-
 .../apache/ranger/rest/TagRESTConstants.java    |   1 +
 .../ranger/security/context/RangerAPIList.java  |   3 +
 .../security/context/RangerAPIMapping.java      |   2 +
 .../service/RangerPluginActivityLogger.java     |  68 +++++
 .../RangerPluginServiceVersionInfoService.java  | 170 ++++++++++++
 .../service/RangerTransactionService.java       | 105 +++++++
 .../RangerPluginServiceVersionInfoList.java     |  72 +++++
 .../resources/META-INF/jpa_named_queries.xml    |  25 ++
 .../src/main/resources/META-INF/persistence.xml |   2 +
 .../org/apache/ranger/rest/TestServiceREST.java |   2 +-
 .../org/apache/ranger/rest/TestTagREST.java     |  32 ++-
 .../storm/RangerAdminClientImpl.java            |   4 +-
 39 files changed, 1646 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminClient.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminClient.java b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminClient.java
index 5ae9854..6755e15 100644
--- a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminClient.java
+++ b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminClient.java
@@ -30,13 +30,13 @@ import java.util.List;
 public interface RangerAdminClient {
 	void init(String serviceName, String appId, String configPropertyPrefix);
 
-	ServicePolicies getServicePoliciesIfUpdated(long lastKnownVersion) throws Exception;
+	ServicePolicies getServicePoliciesIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception;
 
 	void grantAccess(GrantRevokeRequest request) throws Exception;
 
 	void revokeAccess(GrantRevokeRequest request) throws Exception;
 
-	ServiceTags getServiceTagsIfUpdated(long lastKnownVersion) throws Exception;
+	ServiceTags getServiceTagsIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception;
 
 	List<String> getTagTypes(String tagTypePattern) throws Exception;
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java
index 6ec44c4..646004a 100644
--- a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java
+++ b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java
@@ -31,7 +31,11 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.ranger.admin.client.datatype.RESTResponse;
 import org.apache.ranger.audit.provider.MiscUtil;
 import org.apache.ranger.authorization.hadoop.config.RangerConfiguration;
-import org.apache.ranger.plugin.util.*;
+import org.apache.ranger.plugin.util.GrantRevokeRequest;
+import org.apache.ranger.plugin.util.RangerRESTClient;
+import org.apache.ranger.plugin.util.RangerRESTUtils;
+import org.apache.ranger.plugin.util.ServicePolicies;
+import org.apache.ranger.plugin.util.ServiceTags;
 
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
@@ -44,7 +48,7 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 	private String           serviceName = null;
 	private String           pluginId    = null;
 	private RangerRESTClient restClient  = null;
-	private RangerRESTUtils  restUtils   = new RangerRESTUtils();
+	private RangerRESTUtils restUtils   = new RangerRESTUtils();
 
 	public RangerAdminRESTClient() {
 	}
@@ -82,9 +86,9 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 	}
 
 	@Override
-	public ServicePolicies getServicePoliciesIfUpdated(final long lastKnownVersion) throws Exception {
+	public ServicePolicies getServicePoliciesIfUpdated(final long lastKnownVersion, final long lastActivationTimeInMillis) throws Exception {
 		if(LOG.isDebugEnabled()) {
-			LOG.debug("==> RangerAdminRESTClient.getServicePoliciesIfUpdated(" + lastKnownVersion + ")");
+			LOG.debug("==> RangerAdminRESTClient.getServicePoliciesIfUpdated(" + lastKnownVersion + ", " + lastActivationTimeInMillis + ")");
 		}
 
 		ServicePolicies ret = null;
@@ -100,6 +104,7 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 				public ClientResponse run() {
 					WebResource secureWebResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SECURE_SERVICE_IF_UPDATED + serviceName)
 												.queryParam(RangerRESTUtils.REST_PARAM_LAST_KNOWN_POLICY_VERSION, Long.toString(lastKnownVersion))
+												.queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis))
 												.queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId);
 					return secureWebResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class);
 				};
@@ -111,6 +116,7 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 			}
 			WebResource webResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SERVICE_IF_UPDATED + serviceName)
                                                                                 .queryParam(RangerRESTUtils.REST_PARAM_LAST_KNOWN_POLICY_VERSION, Long.toString(lastKnownVersion))
+                                                                                .queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis))
                                                                                 .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId);
 			response = webResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class);
 		}
@@ -125,7 +131,7 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 		}
 
 		if(LOG.isDebugEnabled()) {
-			LOG.debug("<== RangerAdminRESTClient.getServicePoliciesIfUpdated(" + lastKnownVersion + "): " + ret);
+			LOG.debug("<== RangerAdminRESTClient.getServicePoliciesIfUpdated(" + lastKnownVersion + ", " + lastActivationTimeInMillis + "): " + ret);
 		}
 
 		return ret;
@@ -243,9 +249,9 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 	}
 
 	@Override
-	public ServiceTags getServiceTagsIfUpdated(final long lastKnownVersion) throws Exception {
+	public ServiceTags getServiceTagsIfUpdated(final long lastKnownVersion, final long lastActivationTimeInMillis) throws Exception {
 		if(LOG.isDebugEnabled()) {
-			LOG.debug("==> RangerAdminRESTClient.getServiceTagsIfUpdated(" + lastKnownVersion + "): ");
+			LOG.debug("==> RangerAdminRESTClient.getServiceTagsIfUpdated(" + lastKnownVersion + ", " + lastActivationTimeInMillis + "): ");
 		}
 
 		ServiceTags ret = null;
@@ -259,6 +265,7 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 				public ClientResponse run() {
 					WebResource secureWebResource = createWebResource(RangerRESTUtils.REST_URL_GET_SECURE_SERVICE_TAGS_IF_UPDATED + serviceName)
 							.queryParam(RangerRESTUtils.LAST_KNOWN_TAG_VERSION_PARAM, Long.toString(lastKnownVersion))
+							.queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis))
 							.queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId);
 					return secureWebResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class);
 				};
@@ -270,6 +277,7 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 		} else {
 			webResource = createWebResource(RangerRESTUtils.REST_URL_GET_SERVICE_TAGS_IF_UPDATED + serviceName)
 					.queryParam(RangerRESTUtils.LAST_KNOWN_TAG_VERSION_PARAM, Long.toString(lastKnownVersion))
+					.queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis))
 					.queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId);
 			response = webResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class);
 		}
@@ -280,12 +288,13 @@ public class RangerAdminRESTClient implements RangerAdminClient {
 			RESTResponse resp = RESTResponse.fromClientResponse(response);
 			LOG.error("Error getting taggedResources. secureMode=" + isSecureMode + ", user=" + user
 					+ ", response=" + resp.toString() + ", serviceName=" + serviceName
-					+ ", " + "lastKnownVersion=" + lastKnownVersion);
+					+ ", " + "lastKnownVersion=" + lastKnownVersion
+					+ ", " + "lastActivationTimeInMillis=" + lastActivationTimeInMillis);
 			throw new Exception(resp.getMessage());
 		}
 
 		if(LOG.isDebugEnabled()) {
-			LOG.debug("<== RangerAdminRESTClient.getServiceTagsIfUpdated(" + lastKnownVersion + "): ");
+			LOG.debug("<== RangerAdminRESTClient.getServiceTagsIfUpdated(" + lastKnownVersion + ", " + lastActivationTimeInMillis + "): ");
 		}
 
 		return ret;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerAdminTagRetriever.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerAdminTagRetriever.java b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerAdminTagRetriever.java
index 7c5b378..9c336bf 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerAdminTagRetriever.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerAdminTagRetriever.java
@@ -48,13 +48,13 @@ public class RangerAdminTagRetriever extends RangerTagRetriever {
 	}
 
 	@Override
-	public ServiceTags retrieveTags(long lastKnownVersion) throws InterruptedException {
+	public ServiceTags retrieveTags(long lastKnownVersion, long lastActivationTimeInMillis) throws InterruptedException {
 
 		ServiceTags serviceTags = null;
 
 		if (adminClient != null) {
 			try {
-				serviceTags = adminClient.getServiceTagsIfUpdated(lastKnownVersion);
+				serviceTags = adminClient.getServiceTagsIfUpdated(lastKnownVersion, lastActivationTimeInMillis);
 			} catch (InterruptedException interruptedException) {
 				LOG.error("Tag-retriever thread was interrupted");
 				throw interruptedException;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerFileBasedTagRetriever.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerFileBasedTagRetriever.java b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerFileBasedTagRetriever.java
index db4c6e4..248aafa 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerFileBasedTagRetriever.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerFileBasedTagRetriever.java
@@ -124,10 +124,10 @@ public class RangerFileBasedTagRetriever extends RangerTagRetriever {
 	}
 
 	@Override
-	public ServiceTags retrieveTags(long lastKnownVersion) throws InterruptedException {
+	public ServiceTags retrieveTags(long lastKnownVersion, long lastActivationTimeInMillis) throws InterruptedException {
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("==> retrieveTags(lastKnownVersion=" + lastKnownVersion + ", serviceTagsFilePath=" + serviceTagsFileName);
+			LOG.debug("==> retrieveTags(lastKnownVersion=" + lastKnownVersion + ", lastActivationTimeInMillis=" + lastActivationTimeInMillis + ", serviceTagsFilePath=" + serviceTagsFileName);
 		}
 
 		ServiceTags serviceTags = null;
@@ -152,7 +152,7 @@ public class RangerFileBasedTagRetriever extends RangerTagRetriever {
 		}
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("<== retrieveTags(lastKnownVersion=" + lastKnownVersion);
+			LOG.debug("<== retrieveTags(lastKnownVersion=" + lastKnownVersion + ", lastActivationTimeInMillis=" + lastActivationTimeInMillis);
 		}
 
 		return serviceTags;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java
index 00e46ea..62e7c99 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java
@@ -357,6 +357,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 		private final RangerTagRetriever tagRetriever;
 		private final RangerTagEnricher tagEnricher;
 		private long lastKnownVersion = -1L;
+		private long lastActivationTimeInMillis = 0L;
 
 		private final long pollingIntervalMs;
 		private final String cacheFile;
@@ -381,6 +382,14 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 			}
 		}
 
+		public long getLastActivationTimeInMillis() {
+			return lastActivationTimeInMillis;
+		}
+
+		public void setLastActivationTimeInMillis(long lastActivationTimeInMillis) {
+			this.lastActivationTimeInMillis = lastActivationTimeInMillis;
+		}
+
 		@Override
 		public void run() {
 
@@ -423,7 +432,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 			if (tagEnricher != null) {
 				ServiceTags serviceTags = null;
 
-				serviceTags = tagRetriever.retrieveTags(lastKnownVersion);
+				serviceTags = tagRetriever.retrieveTags(lastKnownVersion, lastActivationTimeInMillis);
 
 				if (serviceTags == null) {
 					if (!hasProvidedTagsToReceiver) {
@@ -435,9 +444,11 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 
 				if (serviceTags != null) {
 					tagEnricher.setServiceTags(serviceTags);
-					lastKnownVersion = serviceTags.getTagVersion() == null ? -1L : serviceTags.getTagVersion();
-					LOG.info("RangerTagRefresher.populateTags() - Updated tags-cache to new version of tags, lastKnownVersion=" + lastKnownVersion + "; newVersion=" + serviceTags.getTagVersion());
+					LOG.info("RangerTagRefresher.populateTags() - Updated tags-cache to new version of tags, lastKnownVersion=" + lastKnownVersion + "; newVersion="
+							+ (serviceTags.getTagVersion() == null ? -1L : serviceTags.getTagVersion()));
 					hasProvidedTagsToReceiver = true;
+					lastKnownVersion = serviceTags.getTagVersion() == null ? -1L : serviceTags.getTagVersion();
+					setLastActivationTimeInMillis(System.currentTimeMillis());
 				} else {
 					if (LOG.isDebugEnabled()) {
 						LOG.debug("RangerTagRefresher.populateTags() - No need to update tags-cache. lastKnownVersion=" + lastKnownVersion);

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagFileStoreRetriever.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagFileStoreRetriever.java b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagFileStoreRetriever.java
index 1ee00d3..85cf420 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagFileStoreRetriever.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagFileStoreRetriever.java
@@ -47,12 +47,13 @@ public class RangerTagFileStoreRetriever extends RangerTagRetriever {
 	}
 
 	@Override
-	public ServiceTags retrieveTags(long lastKnownVersion) throws InterruptedException {
+	public ServiceTags retrieveTags(long lastKnownVersion, long lastActivationTimeInMillis) throws InterruptedException {
 
 		ServiceTags serviceTags = null;
 
 		if (tagStore != null) {
 			try {
+				// Ignore lastActivationTimeInMillis for TagFileStore
 				serviceTags = tagStore.getServiceTagsIfUpdated(serviceName, lastKnownVersion);
 			} catch (InterruptedException interruptedException) {
 				LOG.error("Tag-retriever thread was interrupted");

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagRetriever.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagRetriever.java b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagRetriever.java
index 3ec1754..a5eeeaa 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagRetriever.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagRetriever.java
@@ -32,7 +32,7 @@ public abstract class RangerTagRetriever {
 
 	public abstract void init(Map<String, String> options);
 
-	public abstract ServiceTags retrieveTags(long lastKnownVersion) throws InterruptedException;
+	public abstract ServiceTags retrieveTags(long lastKnownVersion, long lastActivationTimeInMillis) throws InterruptedException;
 
 	public String getServiceName() {
 		return serviceName;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerPluginServiceVersionInfo.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerPluginServiceVersionInfo.java b/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerPluginServiceVersionInfo.java
new file mode 100644
index 0000000..8625756
--- /dev/null
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerPluginServiceVersionInfo.java
@@ -0,0 +1,192 @@
+/*
+ * 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.ranger.plugin.model;
+
+import org.codehaus.jackson.annotate.JsonAutoDetect;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.io.Serializable;
+import java.util.Date;
+
+@JsonAutoDetect(fieldVisibility=JsonAutoDetect.Visibility.ANY)
+@JsonSerialize(include=JsonSerialize.Inclusion.NON_NULL)
+@JsonIgnoreProperties(ignoreUnknown=true)
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class RangerPluginServiceVersionInfo implements Serializable {
+	private static final long serialVersionUID = 1L;
+
+	public static final int ENTITY_TYPE_POLICIES = 0;
+	public static final int ENTITY_TYPE_TAGS     = 1;
+
+	private Long    id         = null;
+	private Date    createTime = null;
+
+	private String serviceName;
+	private String hostName;
+	private String appType;
+	private Integer entityType;
+	private String ipAddress;
+	private Long downloadedVersion;
+	private Date downloadTime;
+	private Long activeVersion;
+	private Date activationTime;
+
+	public RangerPluginServiceVersionInfo(Long id, Date createTime, String serviceName, String hostName, String appType, Integer entityType, String ipAddress, Long downloadedVersion, Date downloadTime, Long activeVersion, Date activationTime) {
+		super();
+
+		setId(id);
+		setCreateTime(createTime);
+		setServiceName(serviceName);
+		setHostName(hostName);
+		setAppType(appType);
+		setEntityType(entityType);
+		setIpAddress(ipAddress);
+		setDownloadedVersion(downloadedVersion);
+		setDownloadTime(downloadTime);
+		setActiveVersion(activeVersion);
+		setActivationTime(activationTime);
+	}
+
+	public RangerPluginServiceVersionInfo() {
+	}
+
+	public Long getId() {
+		return id;
+	}
+
+	public void setId(Long id) {
+		this.id = id;
+	}
+
+	public Date getCreateTime() {
+		return createTime;
+	}
+
+	public void setCreateTime(Date createTime) {
+		this.createTime = createTime;
+	}
+
+	public String getServiceName() {
+		return serviceName;
+	}
+
+	public void setServiceName(String serviceName) {
+		this.serviceName = serviceName;
+	}
+
+	public String getHostName() {
+		return hostName;
+	}
+
+	public void setHostName(String hostName) {
+		this.hostName = hostName;
+	}
+
+	public String getAppType() {
+		return appType;
+	}
+
+	public void setAppType(String appType) {
+		this.appType = appType;
+	}
+
+	public Integer getEntityType() {
+		return entityType;
+	}
+
+	public void setEntityType(Integer entityType) {
+		this.entityType = entityType;
+	}
+
+	public String getIpAddress() {
+		return ipAddress;
+	}
+
+	public void setIpAddress(String ipAddress) {
+		this.ipAddress = ipAddress;
+	}
+
+	public Long getDownloadedVersion() {
+		return downloadedVersion;
+	}
+
+	public void setDownloadedVersion(Long downloadedVersion) {
+		this.downloadedVersion = downloadedVersion;
+	}
+
+	public Date getDownloadTime() {
+		return downloadTime;
+	}
+
+	public void setDownloadTime(Date downloadTime) {
+		this.downloadTime = downloadTime;
+	}
+
+	public Long getActiveVersion() {
+		return activeVersion;
+	}
+
+	public void setActiveVersion(Long activeVersion) {
+		this.activeVersion = activeVersion;
+	}
+
+	public Date getActivationTime() {
+		return activationTime;
+	}
+
+	public void setActivationTime(Date activationTime) {
+		this.activationTime = activationTime;
+	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+
+		toString(sb);
+
+		return sb.toString();
+	}
+
+	public StringBuilder toString(StringBuilder sb) {
+		sb.append("RangerPluginServiceVersionInfo={");
+
+		sb.append("id={").append(id).append("} ");
+		sb.append("createTime={").append(createTime).append("} ");
+		sb.append("serviceName={").append(serviceName).append("} ");
+		sb.append("hostName={").append(hostName).append("} ");
+		sb.append("appType={").append(appType).append("} ");
+		sb.append("entityType={").append(entityType).append("} ");
+		sb.append("ipAddress={").append(ipAddress).append("} ");
+		sb.append("downloadedVersion={").append(downloadedVersion).append("} ");
+		sb.append("downloadTime={").append(downloadTime).append("} ");
+		sb.append("activeVersion={").append(activeVersion).append("} ");
+		sb.append("activationTime={").append(activationTime).append("} ");
+
+		sb.append(" }");
+
+		return sb;
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/util/PolicyRefresher.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/util/PolicyRefresher.java b/agents-common/src/main/java/org/apache/ranger/plugin/util/PolicyRefresher.java
index 014e866..91c24c6 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/util/PolicyRefresher.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/util/PolicyRefresher.java
@@ -49,7 +49,8 @@ public class PolicyRefresher extends Thread {
 	private final Gson              gson;
 
 	private long 	pollingIntervalMs   = 30 * 1000;
-	private long 	lastKnownVersion    = -1;
+	private long 	lastKnownVersion    = -1L;
+	private long	lastActivationTimeInMillis  = 0L;
 	private boolean policiesSetInPlugin = false;
 
 
@@ -130,6 +131,13 @@ public class PolicyRefresher extends Thread {
 		this.pollingIntervalMs = pollingIntervalMilliSeconds;
 	}
 
+	public long getLastActivationTimeInMillis() {
+		return lastActivationTimeInMillis;
+	}
+
+	public void setLastActivationTimeInMillis(long lastActivationTimeInMillis) {
+		this.lastActivationTimeInMillis = lastActivationTimeInMillis;
+	}
 
 	public void startRefresher() {
 
@@ -207,6 +215,8 @@ public class PolicyRefresher extends Thread {
 		if (svcPolicies != null) {
 			plugIn.setPolicies(svcPolicies);
 			policiesSetInPlugin = true;
+			setLastActivationTimeInMillis(System.currentTimeMillis());
+			lastKnownVersion = svcPolicies.getPolicyVersion();
 		}
 
 		if(LOG.isDebugEnabled()) {
@@ -229,7 +239,7 @@ public class PolicyRefresher extends Thread {
 		}
 
 		try {
-			svcPolicies = rangerAdmin.getServicePoliciesIfUpdated(lastKnownVersion);
+			svcPolicies = rangerAdmin.getServicePoliciesIfUpdated(lastKnownVersion, lastActivationTimeInMillis);
 
 			boolean isUpdated = svcPolicies != null;
 
@@ -244,8 +254,6 @@ public class PolicyRefresher extends Thread {
 
 				LOG.info("PolicyRefresher(serviceName=" + serviceName + "): found updated version. lastKnownVersion=" + lastKnownVersion + "; newVersion=" + newVersion);
 
-			   	lastKnownVersion = newVersion;
-
 			} else {
 				if(LOG.isDebugEnabled()) {
 					LOG.debug("PolicyRefresher(serviceName=" + serviceName + ").run(): no update found. lastKnownVersion=" + lastKnownVersion);

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTUtils.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTUtils.java b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTUtils.java
index ed674ee..fa81f0b 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTUtils.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTUtils.java
@@ -56,6 +56,7 @@ public class RangerRESTUtils {
 	public static final String REST_MIME_TYPE_JSON     = "application/json";
 
 	public static final String REST_PARAM_LAST_KNOWN_POLICY_VERSION = "lastKnownVersion";
+	public static final String REST_PARAM_LAST_ACTIVATION_TIME = "lastActivationTime";
 	public static final String REST_PARAM_PLUGIN_ID                 = "pluginId";
 
 	private static final int MAX_PLUGIN_ID_LEN = 255;
@@ -130,4 +131,37 @@ public class RangerRESTUtils {
 
         return ret ;
     }
+
+    public String getHostnameFromPluginId(String pluginId, String serviceName) {
+    	String ret = "";
+
+    	if (StringUtils.isNotBlank(pluginId)) {
+			int lastIndex;
+			String[] parts = pluginId.split("@");
+			int index = parts.length > 1 ? 1 : 0;
+			if (StringUtils.isNotBlank(serviceName)) {
+				lastIndex = StringUtils.lastIndexOf(parts[index], serviceName);
+				if (lastIndex > 1) {
+					ret = parts[index].substring(0, lastIndex-1);
+				}
+			} else {
+				lastIndex = StringUtils.lastIndexOf(parts[index], "-");
+				if (lastIndex > 0) {
+					ret = parts[index].substring(0, lastIndex);
+				}
+			}
+		}
+
+		return ret;
+	}
+	public String getAppIdFromPluginId(String pluginId) {
+		String ret = "**Unknown**";
+
+		if (StringUtils.isNotBlank(pluginId)) {
+			String[] parts = pluginId.split("@");
+			ret = parts[0];
+		}
+
+		return ret;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/agents-common/src/main/java/org/apache/ranger/plugin/util/SearchFilter.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/util/SearchFilter.java b/agents-common/src/main/java/org/apache/ranger/plugin/util/SearchFilter.java
index 49a48cd..73ea6e9 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/util/SearchFilter.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/util/SearchFilter.java
@@ -65,6 +65,12 @@ public class SearchFilter {
 	public static final String TAG_MAP_GUID              = "tagResourceMapGuid";  // search
 
 	public static final String SERVICE_NAME_PARTIAL      = "serviceNamePartial";
+
+	public static final String PLUGIN_HOST_NAME          = "pluginHostName";
+	public static final String PLUGIN_APP_TYPE           = "pluginAppType";
+	public static final String PLUGIN_ENTITY_TYPE        = "pluginEntityType";
+	public static final String PLUGIN_IP_ADDRESS         = "pluginIpAddress";
+
 	private Map<String, String> params     = null;
 	private int                 startIndex = 0;
 	private int                 maxRows    = Integer.MAX_VALUE;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerAdminClientImpl.java
----------------------------------------------------------------------
diff --git a/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerAdminClientImpl.java b/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerAdminClientImpl.java
index 5c612e9..99d3f78 100644
--- a/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerAdminClientImpl.java
+++ b/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerAdminClientImpl.java
@@ -50,7 +50,7 @@ public class RangerAdminClientImpl implements RangerAdminClient {
         this.gson = gson;
     }
 
-    public ServicePolicies getServicePoliciesIfUpdated(long lastKnownVersion) throws Exception {
+    public ServicePolicies getServicePoliciesIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception {
 
         String basedir = System.getProperty("basedir");
         if (basedir == null) {
@@ -71,7 +71,7 @@ public class RangerAdminClientImpl implements RangerAdminClient {
 
     }
 
-    public ServiceTags getServiceTagsIfUpdated(long lastKnownVersion) throws Exception {
+    public ServiceTags getServiceTagsIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception {
         return null;
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/hive-agent/src/test/java/org/apache/ranger/services/hive/RangerAdminClientImpl.java
----------------------------------------------------------------------
diff --git a/hive-agent/src/test/java/org/apache/ranger/services/hive/RangerAdminClientImpl.java b/hive-agent/src/test/java/org/apache/ranger/services/hive/RangerAdminClientImpl.java
index 43770c2..8b48dd8 100644
--- a/hive-agent/src/test/java/org/apache/ranger/services/hive/RangerAdminClientImpl.java
+++ b/hive-agent/src/test/java/org/apache/ranger/services/hive/RangerAdminClientImpl.java
@@ -50,7 +50,7 @@ public class RangerAdminClientImpl implements RangerAdminClient {
         this.gson = gson;
     }
 
-    public ServicePolicies getServicePoliciesIfUpdated(long lastKnownVersion) throws Exception {
+    public ServicePolicies getServicePoliciesIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception {
 
         String basedir = System.getProperty("basedir");
         if (basedir == null) {
@@ -71,7 +71,7 @@ public class RangerAdminClientImpl implements RangerAdminClient {
 
     }
 
-    public ServiceTags getServiceTagsIfUpdated(long lastKnownVersion) throws Exception {
+    public ServiceTags getServiceTagsIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception {
         return null;
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java
----------------------------------------------------------------------
diff --git a/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java b/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java
index aeff332..6c0b3e9 100644
--- a/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java
+++ b/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java
@@ -91,9 +91,9 @@ public class RangerAdminJersey2RESTClient implements RangerAdminClient {
 	}
 
 	@Override
-	public ServicePolicies getServicePoliciesIfUpdated(final long lastKnownVersion) throws Exception {
+	public ServicePolicies getServicePoliciesIfUpdated(final long lastKnownVersion, final long lastActivationTimeInMillis) throws Exception {
 		if(LOG.isDebugEnabled()) {
-			LOG.debug("==> RangerAdminJersey2RESTClient.getServicePoliciesIfUpdated(" + lastKnownVersion + ")");
+			LOG.debug("==> RangerAdminJersey2RESTClient.getServicePoliciesIfUpdated(" + lastKnownVersion + ", " + lastActivationTimeInMillis + ")");
 		}
 
 		UserGroupInformation user = MiscUtil.getUGILoginUser();
@@ -162,11 +162,11 @@ public class RangerAdminJersey2RESTClient implements RangerAdminClient {
 			}
 
 			if(LOG.isDebugEnabled()) {
-				LOG.debug("<== RangerAdminJersey2RESTClient.getServicePoliciesIfUpdated(" + lastKnownVersion + "): " + servicePolicies);
+				LOG.debug("<== RangerAdminJersey2RESTClient.getServicePoliciesIfUpdated(" + lastKnownVersion + ", " + lastActivationTimeInMillis + "): " + servicePolicies);
 			}
 			return servicePolicies;
 		} catch (Exception ex) {
-			LOG.error("Failed getting policies from server. url=" + url + ", pluginId=" + _pluginId + ", lastKnownVersion=" + lastKnownVersion );
+			LOG.error("Failed getting policies from server. url=" + url + ", pluginId=" + _pluginId + ", lastKnownVersion=" + lastKnownVersion + ", " + lastActivationTimeInMillis);
 			throw ex;
 		}
 	}
@@ -242,7 +242,7 @@ public class RangerAdminJersey2RESTClient implements RangerAdminClient {
 	}
 
 	@Override
-	public ServiceTags getServiceTagsIfUpdated(long lastKnownVersion) throws Exception {
+	public ServiceTags getServiceTagsIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception {
 		throw new Exception("RangerAdminjersey2RESTClient.getServiceTagsIfUpdated() -- *** NOT IMPLEMENTED *** ");
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/plugin-kms/src/test/java/org/apache/ranger/authorization/kms/authorizer/RangerAdminClientImpl.java
----------------------------------------------------------------------
diff --git a/plugin-kms/src/test/java/org/apache/ranger/authorization/kms/authorizer/RangerAdminClientImpl.java b/plugin-kms/src/test/java/org/apache/ranger/authorization/kms/authorizer/RangerAdminClientImpl.java
index e889447..6196727 100644
--- a/plugin-kms/src/test/java/org/apache/ranger/authorization/kms/authorizer/RangerAdminClientImpl.java
+++ b/plugin-kms/src/test/java/org/apache/ranger/authorization/kms/authorizer/RangerAdminClientImpl.java
@@ -50,7 +50,7 @@ public class RangerAdminClientImpl implements RangerAdminClient {
         this.gson = gson;
     }
 
-    public ServicePolicies getServicePoliciesIfUpdated(long lastKnownVersion) throws Exception {
+    public ServicePolicies getServicePoliciesIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception {
 
         String basedir = System.getProperty("basedir");
         if (basedir == null) {
@@ -71,7 +71,7 @@ public class RangerAdminClientImpl implements RangerAdminClient {
 
     }
 
-    public ServiceTags getServiceTagsIfUpdated(long lastKnownVersion) throws Exception {
+    public ServiceTags getServiceTagsIfUpdated(long lastKnownVersion, long lastActivationTimeInMillis) throws Exception {
         return null;
 
     }
@@ -81,4 +81,4 @@ public class RangerAdminClientImpl implements RangerAdminClient {
     }
 
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/db/mysql/patches/025-create-schema-for-plugin-service-version-tracking.sql
----------------------------------------------------------------------
diff --git a/security-admin/db/mysql/patches/025-create-schema-for-plugin-service-version-tracking.sql b/security-admin/db/mysql/patches/025-create-schema-for-plugin-service-version-tracking.sql
new file mode 100644
index 0000000..d2f3d11
--- /dev/null
+++ b/security-admin/db/mysql/patches/025-create-schema-for-plugin-service-version-tracking.sql
@@ -0,0 +1,36 @@
+-- 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.
+
+DROP TABLE IF EXISTS `x_plugin_service_version_info`;
+
+
+CREATE TABLE x_plugin_service_version_info (
+`id` bigint(20) NOT NULL AUTO_INCREMENT,
+`create_time` datetime NOT NULL,
+`service_name` varchar(255) NOT  NULL,
+`host_name` varchar(64) NOT NULL,
+`app_type` varchar(128) NOT NULL,
+`entity_type` smallint(10) NOT NULL DEFAULT '0',
+`ip_address` varchar(64) NOT NULL,
+`downloaded_version` bigint NOT NULL DEFAULT '0',
+`download_time` datetime NOT NULL,
+`active_version` bigint(20) NOT NULL DEFAULT '0',
+`activation_time` datetime DEFAULT NULL,
+ PRIMARY KEY (`id`),
+ CREATE UNIQUE INDEX x_plugin_service_version_info_IDX_unique ON x_plugin_service_version_info(service_name, host_name, app_type, entity_type),
+ CREATE INDEX x_plugin_service_version_info_IDX_service_name ON x_plugin_service_version_info(service_name),
+ CREATE INDEX x_plugin_service_version_info_IDX_host_name ON x_plugin_service_version_info(host_name)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/db/postgres/patches/025-create-schema-for-plugin-service-version-tracking.sql
----------------------------------------------------------------------
diff --git a/security-admin/db/postgres/patches/025-create-schema-for-plugin-service-version-tracking.sql b/security-admin/db/postgres/patches/025-create-schema-for-plugin-service-version-tracking.sql
new file mode 100644
index 0000000..f8de7fa
--- /dev/null
+++ b/security-admin/db/postgres/patches/025-create-schema-for-plugin-service-version-tracking.sql
@@ -0,0 +1,38 @@
+-- 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.
+
+DROP TABLE IF EXISTS x_plugin_service_version_info;
+DROP SEQUENCE IF EXISTS x_plugin_service_version_info_seq;
+
+CREATE SEQUENCE x_plugin_service_version_info_seq;
+
+CREATE TABLE x_plugin_service_version_info (
+id BIGINT DEFAULT nextval('x_plugin_service_version_info_seq'::regclass),
+create_time TIMESTAMP NOT NULL,
+service_name varchar(256) NOT NULL,
+host_name varchar(64) NOT NULL,
+app_type varchar(128) NOT NULL,
+entity_type smallint NOT NULL DEFAULT '0',
+ip_address varchar(64) NOT NULL,
+downloaded_version bigint NOT NULL DEFAULT '0',
+download_time TIMESTAMP NOT NULL,
+active_version bigint NOT NULL DEFAULT '0',
+activation_time TIMESTAMP DEFAULT NULL,
+primary key (id)
+);
+CREATE UNIQUE INDEX x_plugin_service_version_info_IDX_unique ON x_plugin_service_version_info(service_name, host_name, app_type, entity_type);
+CREATE INDEX x_plugin_service_version_info_IDX_service_name ON x_plugin_service_version_info(service_name);
+CREATE INDEX x_plugin_service_version_info_IDX_host_name ON x_plugin_service_version_info(host_name);
+

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/src/main/java/org/apache/ranger/biz/AssetMgr.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/biz/AssetMgr.java b/security-admin/src/main/java/org/apache/ranger/biz/AssetMgr.java
index 931356e..7b9fc59 100644
--- a/security-admin/src/main/java/org/apache/ranger/biz/AssetMgr.java
+++ b/security-admin/src/main/java/org/apache/ranger/biz/AssetMgr.java
@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.IOException;
 import java.security.cert.X509Certificate;
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -32,8 +33,10 @@ import java.util.Set;
 import javax.naming.InvalidNameException;
 import javax.naming.ldap.LdapName;
 import javax.naming.ldap.Rdn;
+import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 import org.apache.ranger.common.AppConstants;
 import org.apache.ranger.common.DateUtil;
@@ -45,10 +48,15 @@ import org.apache.ranger.common.SearchCriteria;
 import org.apache.ranger.common.StringUtil;
 import org.apache.ranger.db.RangerDaoManager;
 import org.apache.ranger.entity.XXPermMap;
+import org.apache.ranger.entity.XXPluginServiceVersionInfo;
 import org.apache.ranger.entity.XXPolicyExportAudit;
 import org.apache.ranger.entity.XXPortalUser;
 import org.apache.ranger.entity.XXTrxLog;
 import org.apache.ranger.entity.XXUser;
+import org.apache.ranger.plugin.model.RangerPluginServiceVersionInfo;
+import org.apache.ranger.plugin.util.RangerRESTUtils;
+import org.apache.ranger.service.RangerPluginActivityLogger;
+import org.apache.ranger.service.RangerPluginServiceVersionInfoService;
 import org.apache.ranger.service.XAccessAuditService;
 import org.apache.ranger.service.XAuditMapService;
 import org.apache.ranger.service.XGroupService;
@@ -58,25 +66,26 @@ import org.apache.ranger.service.XTrxLogService;
 import org.apache.ranger.service.XUserService;
 import org.apache.ranger.solr.SolrAccessAuditsService;
 import org.apache.ranger.util.RestUtil;
-import org.apache.ranger.view.*;
+import org.apache.ranger.view.VXAccessAuditList;
+import org.apache.ranger.view.VXAsset;
+import org.apache.ranger.view.VXAuditMap;
+import org.apache.ranger.view.VXPermMap;
+import org.apache.ranger.view.VXPolicyExportAuditList;
+import org.apache.ranger.view.VXResource;
+import org.apache.ranger.view.VXTrxLog;
+import org.apache.ranger.view.VXTrxLogList;
+import org.apache.ranger.view.VXUser;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.map.JsonMappingException;
 import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.beans.factory.annotation.Qualifier;
 import org.springframework.stereotype.Component;
-import org.springframework.transaction.PlatformTransactionManager;
-import org.springframework.transaction.TransactionDefinition;
-import org.springframework.transaction.TransactionStatus;
-import org.springframework.transaction.support.TransactionCallback;
-import org.springframework.transaction.support.TransactionTemplate;
 
 @Component
 public class AssetMgr extends AssetMgrBase {
-	
-	
+
 	@Autowired
 	XPermMapService xPermMapService;
-	
+
 	@Autowired
 	XAuditMapService xAuditMapService;
 
@@ -114,14 +123,12 @@ public class AssetMgr extends AssetMgrBase {
 	SolrAccessAuditsService solrAccessAuditsService;
 
 	@Autowired
-	@Qualifier(value = "transactionManager")
-	PlatformTransactionManager txManager;
-	
-	@Autowired
 	XPolicyService xPolicyService;
-	
-	static Logger logger = Logger.getLogger(AssetMgr.class);
 
+	@Autowired
+	RangerPluginActivityLogger activityLogger;
+
+	static Logger logger = Logger.getLogger(AssetMgr.class);
 
 	public File getXResourceFile(Long id, String fileType) {
 		VXResource xResource = xResourceService.readResource(id);
@@ -171,8 +178,8 @@ public class AssetMgr extends AssetMgrBase {
 	}
 
 	public String getLatestRepoPolicy(VXAsset xAsset, List<VXResource> xResourceList, Long updatedTime,
-			X509Certificate[] certchain, boolean httpEnabled, String epoch,
-			String ipAddress, boolean isSecure, String count, String agentId) {
+									  X509Certificate[] certchain, boolean httpEnabled, String epoch,
+									  String ipAddress, boolean isSecure, String count, String agentId) {
 		if(xAsset==null){
 			logger.error("Requested repository not found");
 			throw restErrorUtil.createRESTException("No Data Found.",
@@ -631,26 +638,138 @@ public class AssetMgr extends AssetMgrBase {
 
 	public XXPolicyExportAudit createPolicyAudit(
 			final XXPolicyExportAudit xXPolicyExportAudit) {
-		TransactionTemplate txTemplate = new TransactionTemplate(txManager);
-		txTemplate
-				.setPropagationBehavior(TransactionDefinition.PROPAGATION_REQUIRED);
-		XXPolicyExportAudit policyExportAudit = (XXPolicyExportAudit) txTemplate
-				.execute(new TransactionCallback<Object>() {
-					public Object doInTransaction(TransactionStatus status) {
-						if (xXPolicyExportAudit.getHttpRetCode() == HttpServletResponse.SC_NOT_MODIFIED) {
-							boolean logNotModified = PropertiesUtil.getBooleanProperty("ranger.log.SC_NOT_MODIFIED", false);
-							if (!logNotModified) {
-								logger.debug("Not logging HttpServletResponse."
-										+ "SC_NOT_MODIFIED, to enable, update "
-										+ ": ranger.log.SC_NOT_MODIFIED");
-								return null;
-							}
-						}
-						return rangerDaoManager.getXXPolicyExportAudit().create(
-								xXPolicyExportAudit);
+
+		XXPolicyExportAudit ret = null;
+		if (xXPolicyExportAudit.getHttpRetCode() == HttpServletResponse.SC_NOT_MODIFIED) {
+			boolean logNotModified = PropertiesUtil.getBooleanProperty("ranger.log.SC_NOT_MODIFIED", false);
+			if (!logNotModified) {
+				logger.debug("Not logging HttpServletResponse."
+						+ "SC_NOT_MODIFIED, to enable, update "
+						+ ": ranger.log.SC_NOT_MODIFIED");
+			} else {
+				Runnable commitWork = new Runnable() {
+					@Override
+					public void run() {
+						rangerDaoManager.getXXPolicyExportAudit().create(xXPolicyExportAudit);
+
+					}
+				};
+				activityLogger.commitAfterTransactionComplete(commitWork);
+			}
+		} else {
+			ret = rangerDaoManager.getXXPolicyExportAudit().create(xXPolicyExportAudit);
+		}
+
+		return ret;
+	}
+
+	public void createPluginServiceVersionInfo(String serviceName, String pluginId, HttpServletRequest request, int entityType, long downloadedVersion, long lastKnownVersion, long lastActivationTime, int httpCode) {
+		RangerRESTUtils restUtils = new RangerRESTUtils();
+
+		final String ipAddress = request != null ? request.getRemoteAddr() : null;
+		final String appType = restUtils.getAppIdFromPluginId(pluginId);
+
+		String tmpHostName = null;
+		if (StringUtils.isNotBlank(pluginId)) {
+			tmpHostName = restUtils.getHostnameFromPluginId(pluginId, serviceName);
+		}
+		if (StringUtils.isBlank(tmpHostName) && request != null) {
+			tmpHostName = request.getRemoteHost();
+		}
+
+		final String hostName = (StringUtils.isBlank(tmpHostName)) ? ipAddress : tmpHostName;
+
+		RangerPluginServiceVersionInfo pluginSvcVersionInfo = new RangerPluginServiceVersionInfo();
+		pluginSvcVersionInfo.setServiceName(serviceName);
+		pluginSvcVersionInfo.setEntityType(RangerPluginServiceVersionInfo.ENTITY_TYPE_POLICIES);
+		pluginSvcVersionInfo.setActiveVersion(lastKnownVersion);
+		pluginSvcVersionInfo.setActivationTime(new Date(lastActivationTime));
+		pluginSvcVersionInfo.setDownloadedVersion(downloadedVersion);
+		pluginSvcVersionInfo.setDownloadTime(new Date());
+		pluginSvcVersionInfo.setHostName(hostName);
+		pluginSvcVersionInfo.setIpAddress(ipAddress);
+		pluginSvcVersionInfo.setAppType(appType);
+		createOrUpdatePluginServiceVersionInfo(pluginSvcVersionInfo, httpCode);
+	}
+
+	void createOrUpdatePluginServiceVersionInfo(final RangerPluginServiceVersionInfo pluginServiceVersionInfo, final int httpCode) {
+		if (logger.isDebugEnabled()) {
+			logger.debug("==> createOrUpdatePluginServiceVersionInfo(pluginServiceVersionInfo=" + pluginServiceVersionInfo + ", httpCode=" + httpCode + ")");
+		}
+
+		if (httpCode == HttpServletResponse.SC_NOT_MODIFIED) {
+			Runnable commitWork = new Runnable() {
+				@Override
+				public void run() {
+					doCreateOrUpdateXXPluginServiceVersionInfo(pluginServiceVersionInfo);
+				}
+			};
+			activityLogger.commitAfterTransactionComplete(commitWork);
+		} else {
+			doCreateOrUpdateXXPluginServiceVersionInfo(pluginServiceVersionInfo);
+		}
+		if (logger.isDebugEnabled()) {
+			logger.debug("<== createOrUpdatePluginServiceVersionInfo(pluginServiceVersionInfo=" + pluginServiceVersionInfo + ", httpCode=" + httpCode + ")");
+		}
+
+	}
+
+	XXPluginServiceVersionInfo doCreateOrUpdateXXPluginServiceVersionInfo(RangerPluginServiceVersionInfo pluginServiceVersionInfo) {
+		XXPluginServiceVersionInfo ret = null;
+
+		if (StringUtils.isNotBlank(pluginServiceVersionInfo.getServiceName())) {
+
+			XXPluginServiceVersionInfo xObj = rangerDaoManager.getXXPluginServiceVersionInfo().find(pluginServiceVersionInfo.getServiceName(),
+					pluginServiceVersionInfo.getHostName(), pluginServiceVersionInfo.getAppType(), pluginServiceVersionInfo.getEntityType());
+
+			if (xObj == null) {
+				// If the ranger-admin is restarted, plugin contains latest version and there is no row for this pluginServiceVersionInfo
+				if (pluginServiceVersionInfo.getDownloadedVersion().equals(pluginServiceVersionInfo.getActiveVersion())) {
+					// This is our best guess of when policies may have been downloaded
+					pluginServiceVersionInfo.setDownloadTime(pluginServiceVersionInfo.getActivationTime());
+				}
+				xObj = RangerPluginServiceVersionInfoService.populateDBObject(pluginServiceVersionInfo);
+
+				if (logger.isDebugEnabled()) {
+					logger.debug("Creating RangerPluginServiceVersionInfo record for service-version");
+				}
+				ret = rangerDaoManager.getXXPluginServiceVersionInfo().create(xObj);
+			} else {
+				boolean needsUpdating = false;
+
+				if (!xObj.getIpAddress().equals(pluginServiceVersionInfo.getIpAddress())) {
+					xObj.setIpAddress(pluginServiceVersionInfo.getIpAddress());
+					needsUpdating = true;
+				}
+				if (!xObj.getDownloadedVersion().equals(pluginServiceVersionInfo.getDownloadedVersion())) {
+					xObj.setDownloadedVersion(pluginServiceVersionInfo.getDownloadedVersion());
+					xObj.setDownloadTime(pluginServiceVersionInfo.getDownloadTime());
+					needsUpdating = true;
+				}
+				long lastKnownVersion = pluginServiceVersionInfo.getActiveVersion();
+				long lastActivationTime = pluginServiceVersionInfo.getActivationTime().getTime();
+
+				if (lastKnownVersion > 0 && !xObj.getActiveVersion().equals(lastKnownVersion)) {
+					xObj.setActiveVersion(lastKnownVersion);
+					if (lastActivationTime > 0) {
+						xObj.setActivationTime(new Date(lastActivationTime));
 					}
-				});
-		return policyExportAudit;
+					needsUpdating = true;
+				}
+
+				if (needsUpdating) {
+					if (logger.isDebugEnabled()) {
+						logger.debug("Updating XXPluginServiceVersionInfo record for service-version");
+					}
+
+					ret = rangerDaoManager.getXXPluginServiceVersionInfo().update(xObj);
+				}
+			}
+		} else {
+			logger.error("Invalid parameters: pluginServiceVersionInfo=" + pluginServiceVersionInfo + ")");
+		}
+
+		return ret;
 	}
 
 	public VXTrxLogList getReportLogs(SearchCriteria searchCriteria) {
@@ -665,17 +784,17 @@ public class AssetMgr extends AssetMgrBase {
 		if (searchCriteria.getParamList() != null
 				&& searchCriteria.getParamList().size() > 0) {
 			int clientTimeOffsetInMinute = RestUtil.getClientTimeOffset();
-			java.util.Date temp = null;
+			Date temp = null;
 			DateUtil dateUtil = new DateUtil();
 			if (searchCriteria.getParamList().containsKey("startDate")) {
-				temp = (java.util.Date) searchCriteria.getParamList().get(
+				temp = (Date) searchCriteria.getParamList().get(
 						"startDate");
 				temp = dateUtil.getDateFromGivenDate(temp, 0, 0, 0, 0);
 				temp = dateUtil.addTimeOffset(temp, clientTimeOffsetInMinute);
 				searchCriteria.getParamList().put("startDate", temp);
 			}
 			if (searchCriteria.getParamList().containsKey("endDate")) {
-				temp = (java.util.Date) searchCriteria.getParamList().get(
+				temp = (Date) searchCriteria.getParamList().get(
 						"endDate");
 				temp = dateUtil.getDateFromGivenDate(temp, 0, 23, 59, 59);
 				temp = dateUtil.addTimeOffset(temp, clientTimeOffsetInMinute);
@@ -713,17 +832,17 @@ public class AssetMgr extends AssetMgrBase {
         if (searchCriteria.getParamList() != null
                 && searchCriteria.getParamList().size() > 0) {
             int clientTimeOffsetInMinute = RestUtil.getClientTimeOffset();
-            java.util.Date temp = null;
+            Date temp = null;
             DateUtil dateUtil = new DateUtil();
             if (searchCriteria.getParamList().containsKey("startDate")) {
-                temp = (java.util.Date) searchCriteria.getParamList().get(
+                temp = (Date) searchCriteria.getParamList().get(
                         "startDate");
                 temp = dateUtil.getDateFromGivenDate(temp, 0, 0, 0, 0);
                 temp = dateUtil.addTimeOffset(temp, clientTimeOffsetInMinute);
                 searchCriteria.getParamList().put("startDate", temp);
             }
             if (searchCriteria.getParamList().containsKey("endDate")) {
-                temp = (java.util.Date) searchCriteria.getParamList().get(
+                temp = (Date) searchCriteria.getParamList().get(
                         "endDate");
                 temp = dateUtil.getDateFromGivenDate(temp, 0, 23, 59, 59);
                 temp = dateUtil.addTimeOffset(temp, clientTimeOffsetInMinute);
@@ -828,17 +947,17 @@ public class AssetMgr extends AssetMgrBase {
                 && searchCriteria.getParamList().size() > 0) {
 
             int clientTimeOffsetInMinute = RestUtil.getClientTimeOffset();
-            java.util.Date temp = null;
+            Date temp = null;
             DateUtil dateUtil = new DateUtil();
             if (searchCriteria.getParamList().containsKey("startDate")) {
-                temp = (java.util.Date) searchCriteria.getParamList().get(
+                temp = (Date) searchCriteria.getParamList().get(
                         "startDate");
                 temp = dateUtil.getDateFromGivenDate(temp, 0, 0, 0, 0);
                 temp = dateUtil.addTimeOffset(temp, clientTimeOffsetInMinute);
                 searchCriteria.getParamList().put("startDate", temp);
             }
             if (searchCriteria.getParamList().containsKey("endDate")) {
-                temp = (java.util.Date) searchCriteria.getParamList().get(
+                temp = (Date) searchCriteria.getParamList().get(
                         "endDate");
                 temp = dateUtil.getDateFromGivenDate(temp, 0, 23, 59, 59);
                 temp = dateUtil.addTimeOffset(temp, clientTimeOffsetInMinute);

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/src/main/java/org/apache/ranger/common/RangerSearchUtil.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/common/RangerSearchUtil.java b/security-admin/src/main/java/org/apache/ranger/common/RangerSearchUtil.java
index 2d049a4..4fb52a4 100644
--- a/security-admin/src/main/java/org/apache/ranger/common/RangerSearchUtil.java
+++ b/security-admin/src/main/java/org/apache/ranger/common/RangerSearchUtil.java
@@ -67,6 +67,11 @@ public class RangerSearchUtil extends SearchUtil {
 		ret.setParam(SearchFilter.RESOURCE_SIGNATURE, request.getParameter(SearchFilter.RESOURCE_SIGNATURE));
 		ret.setParam(SearchFilter.POLICY_TYPE, request.getParameter(SearchFilter.POLICY_TYPE));
 
+		ret.setParam(SearchFilter.PLUGIN_HOST_NAME, request.getParameter(SearchFilter.PLUGIN_HOST_NAME));
+		ret.setParam(SearchFilter.PLUGIN_APP_TYPE, request.getParameter(SearchFilter.PLUGIN_APP_TYPE));
+		ret.setParam(SearchFilter.PLUGIN_ENTITY_TYPE, request.getParameter(SearchFilter.PLUGIN_ENTITY_TYPE));
+		ret.setParam(SearchFilter.PLUGIN_IP_ADDRESS, request.getParameter(SearchFilter.PLUGIN_IP_ADDRESS));
+
 		for (Map.Entry<String, String[]> e : request.getParameterMap().entrySet()) {
 			String name = e.getKey();
 			String[] values = e.getValue();

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/src/main/java/org/apache/ranger/common/db/RangerTransactionSynchronizationAdapter.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/common/db/RangerTransactionSynchronizationAdapter.java b/security-admin/src/main/java/org/apache/ranger/common/db/RangerTransactionSynchronizationAdapter.java
new file mode 100644
index 0000000..3f5536f
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/common/db/RangerTransactionSynchronizationAdapter.java
@@ -0,0 +1,118 @@
+/*
+ * 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.ranger.common.db;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Qualifier;
+import org.springframework.stereotype.Component;
+import org.springframework.transaction.PlatformTransactionManager;
+import org.springframework.transaction.TransactionDefinition;
+import org.springframework.transaction.TransactionStatus;
+import org.springframework.transaction.support.TransactionCallback;
+import org.springframework.transaction.support.TransactionSynchronizationAdapter;
+import org.springframework.transaction.support.TransactionSynchronizationManager;
+import org.springframework.transaction.support.TransactionTemplate;
+
+@Component
+public class RangerTransactionSynchronizationAdapter extends TransactionSynchronizationAdapter {
+
+    @Autowired
+    @Qualifier(value = "transactionManager")
+    PlatformTransactionManager txManager;
+
+    private static final Log LOG = LogFactory.getLog(RangerTransactionSynchronizationAdapter.class);
+
+    private static final ThreadLocal<List<Runnable>> RUNNABLES = new ThreadLocal<List<Runnable>>();
+
+    public void executeOnTransactionCompletion(Runnable runnable) {
+        LOG.info("Submitting new runnable {" + runnable + "} to run after completion");
+
+        /*
+        From TransactionSynchronizationManager documentation:
+        TransactionSynchronizationManager is a central helper that manages resources and transaction synchronizations per thread.
+        Resource management code should only register synchronizations when this manager is active,
+        which can be checked via isSynchronizationActive(); it should perform immediate resource cleanup else.
+        If transaction synchronization isn't active, there is either no current transaction,
+        or the transaction manager doesn't support transaction synchronization.
+
+        Note: Synchronization is an Interface for transaction synchronization callbacks which is implemented by
+        TransactionSynchronizationAdapter
+        */
+
+        if (!TransactionSynchronizationManager.isSynchronizationActive()) {
+            LOG.info("Transaction synchronization is NOT ACTIVE. Executing right now runnable {" + runnable + "}");
+            runnable.run();
+            return;
+        }
+        List<Runnable> threadRunnables = RUNNABLES.get();
+        if (threadRunnables == null) {
+            threadRunnables = new ArrayList<Runnable>();
+            RUNNABLES.set(threadRunnables);
+            // Register a new transaction synchronization for the current thread.
+            // TransactionSynchronizationManage will call afterCompletion() when current transaction completes.
+            TransactionSynchronizationManager.registerSynchronization(this);
+        }
+        threadRunnables.add(runnable);
+    }
+
+    @Override
+    public void afterCompletion(int status) {
+        LOG.info("Transaction completed with status {" + (status == STATUS_COMMITTED ? "COMMITTED" : "ROLLED_BACK") + "}");
+        if (status == STATUS_ROLLED_BACK) {
+            final List<Runnable> threadRunnables = RUNNABLES.get();
+            LOG.info("Transaction completed, executing {" + threadRunnables.size() + "} runnables");
+            if (threadRunnables != null) {
+                try {
+                    //Create new  transaction
+                    TransactionTemplate txTemplate = new TransactionTemplate(txManager);
+                    txTemplate.setPropagationBehavior(TransactionDefinition.PROPAGATION_REQUIRES_NEW);
+
+                    txTemplate.execute(new TransactionCallback<Object>() {
+                        public Object doInTransaction(TransactionStatus status) {
+                            for (Runnable runnable : threadRunnables) {
+                                LOG.info("Executing runnable {" + runnable + "}");
+                                try {
+                                    runnable.run();
+                                } catch (RuntimeException e) {
+                                    LOG.error("Failed to execute runnable " + runnable, e);
+                                    break;
+                                }
+                            }
+
+                            return null;
+                        }
+                    });
+                } catch (Exception e) {
+                    LOG.error("Failed to commit TransactionService transaction", e);
+                    LOG.error("Ignoring...");
+                }
+            }
+
+        }
+        RUNNABLES.remove();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/src/main/java/org/apache/ranger/db/RangerDaoManagerBase.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/db/RangerDaoManagerBase.java b/security-admin/src/main/java/org/apache/ranger/db/RangerDaoManagerBase.java
index 6cd5c9f..aeb0fd7 100644
--- a/security-admin/src/main/java/org/apache/ranger/db/RangerDaoManagerBase.java
+++ b/security-admin/src/main/java/org/apache/ranger/db/RangerDaoManagerBase.java
@@ -363,6 +363,9 @@ public abstract class RangerDaoManagerBase {
 		if (className.equals("XXServiceVersionInfo")) {
 			return getXXServiceVersionInfo();
 		}
+		if (className.equals("XXPluginServiceVersionInfo")) {
+			return getXXPluginServiceVersionInfo();
+		}
 		logger.error("No DaoManager found for className=" + className, new Throwable());
 		return null;
 	}
@@ -583,5 +586,9 @@ public abstract class RangerDaoManagerBase {
 	public XXServiceVersionInfoDao getXXServiceVersionInfo() {
 		return new XXServiceVersionInfoDao(this);
 	}
+
+	public XXPluginServiceVersionInfoDao getXXPluginServiceVersionInfo() {
+		return new XXPluginServiceVersionInfoDao(this);
+	}
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/src/main/java/org/apache/ranger/db/XXPluginServiceVersionInfoDao.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/db/XXPluginServiceVersionInfoDao.java b/security-admin/src/main/java/org/apache/ranger/db/XXPluginServiceVersionInfoDao.java
new file mode 100644
index 0000000..2aca246
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/db/XXPluginServiceVersionInfoDao.java
@@ -0,0 +1,95 @@
+/*
+ * 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.ranger.db;
+
+import java.util.List;
+
+import javax.persistence.NoResultException;
+
+import org.apache.ranger.common.db.BaseDao;
+import org.apache.ranger.entity.XXPluginServiceVersionInfo;
+
+/**
+ */
+
+public class XXPluginServiceVersionInfoDao extends BaseDao<XXPluginServiceVersionInfo> {
+	/**
+	 * Default Constructor
+	 */
+	public XXPluginServiceVersionInfoDao(RangerDaoManagerBase daoManager) {
+		super(daoManager);
+	}
+
+	public XXPluginServiceVersionInfo find(String serviceName, String hostName, String appType, Integer entityType) {
+		if (serviceName == null || hostName == null || appType == null || entityType == null) {
+			return null;
+		}
+		try {
+			return getEntityManager()
+					.createNamedQuery("XXPluginServiceVersionInfo.find", tClass)
+					.setParameter("serviceName", serviceName)
+					.setParameter("hostName", hostName)
+					.setParameter("appType", appType)
+					.setParameter("entityType", entityType)
+					.getSingleResult();
+		} catch (NoResultException e) {
+			return null;
+		}
+	}
+	public List<XXPluginServiceVersionInfo> findByServiceName(String serviceName) {
+		if (serviceName == null) {
+			return null;
+		}
+		try {
+			return getEntityManager()
+					.createNamedQuery("XXPluginServiceVersionInfo.findByServiceName", tClass)
+					.setParameter("serviceName", serviceName).getResultList();
+		} catch (NoResultException e) {
+			return null;
+		}
+	}
+
+	public List<XXPluginServiceVersionInfo> findByServiceId(Long serviceId) {
+		if (serviceId == null) {
+			return null;
+		}
+		try {
+			return getEntityManager()
+					.createNamedQuery("XXPluginServiceVersionInfo.findByServiceId", tClass)
+					.setParameter("serviceId", serviceId).getResultList();
+		} catch (NoResultException e) {
+			return null;
+		}
+	}
+
+	public List<XXPluginServiceVersionInfo> findByServiceAndHostName(String serviceName, String hostName) {
+		if (serviceName == null || hostName == null) {
+			return null;
+		}
+		try {
+			return getEntityManager()
+					.createNamedQuery("XXPluginServiceVersionInfo.findByServiceAndHostName", tClass)
+					.setParameter("serviceName", serviceName)
+					.setParameter("hostName", hostName)
+					.getResultList();
+		} catch (NoResultException e) {
+			return null;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/src/main/java/org/apache/ranger/entity/XXPluginServiceVersionInfo.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/entity/XXPluginServiceVersionInfo.java b/security-admin/src/main/java/org/apache/ranger/entity/XXPluginServiceVersionInfo.java
new file mode 100644
index 0000000..ff0a85c
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/entity/XXPluginServiceVersionInfo.java
@@ -0,0 +1,271 @@
+/*
+ * 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.ranger.entity;
+
+import java.util.Date;
+
+import javax.persistence.Cacheable;
+import javax.persistence.Entity;
+import javax.persistence.Column;
+import javax.persistence.GeneratedValue;
+import javax.persistence.GenerationType;
+import javax.persistence.Id;
+import javax.persistence.SequenceGenerator;
+import javax.persistence.Table;
+import javax.persistence.Temporal;
+import javax.persistence.TemporalType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.ranger.common.AppConstants;
+import org.apache.ranger.common.DateUtil;
+
+@Entity
+@Cacheable
+@XmlRootElement
+@Table(name = "x_plugin_service_version_info")
+public class XXPluginServiceVersionInfo implements java.io.Serializable {
+	private static final long serialVersionUID = 1L;
+
+	@Id
+	@SequenceGenerator(name = "X_PLUGIN_SERVICE_VERSION_INFO_SEQ", sequenceName = "X_PLUGIN_SERVICE_VERSION_INFO_SEQ", allocationSize = 1)
+	@GeneratedValue(strategy = GenerationType.AUTO, generator = "X_PLUGIN_SERVICE_VERSION_INFO_SEQ")
+	@Column(name = "id")
+	protected Long id;
+
+	@Temporal(TemporalType.TIMESTAMP)
+	@Column(name="CREATE_TIME"   )
+	protected Date createTime = DateUtil.getUTCDate();
+
+	@Column(name = "service_name")
+	protected String serviceName;
+
+	@Column(name = "host_name")
+	protected String hostName;
+
+	@Column(name = "app_type")
+	protected String appType;
+
+	@Column(name = "entity_type")
+	protected Integer entityType;
+
+	@Column(name = "ip_address")
+	protected String ipAddress;
+
+	@Column(name = "downloaded_version")
+	protected Long downloadedVersion;
+
+	@Temporal(TemporalType.TIMESTAMP)
+	@Column(name="download_time"   )
+	protected Date downloadTime = DateUtil.getUTCDate();
+
+	@Column(name = "active_version")
+	protected Long activeVersion;
+
+	@Temporal(TemporalType.TIMESTAMP)
+	@Column(name="activation_time"   )
+	protected Date activationTime = DateUtil.getUTCDate();
+
+	/**
+	 * Default constructor. This will set all the attributes to default value.
+	 */
+	public XXPluginServiceVersionInfo ( ) {
+	}
+
+	public int getMyClassType( ) {
+	    return AppConstants.CLASS_TYPE_NONE;
+	}
+
+	public String getMyDisplayValue() {
+		return null;
+	}
+
+	public void setId(Long id) {
+		this.id = id;
+	}
+
+	public Long getId() {
+		return this.id;
+	}
+
+	public void setCreateTime( Date createTime ) {
+		this.createTime = createTime;
+	}
+
+	public Date getCreateTime( ) {
+		return this.createTime;
+	}
+
+	public void setServiceName(String serviceName) {
+		this.serviceName = serviceName;
+	}
+
+	public String getServiceName() {
+		return this.serviceName;
+	}
+
+	public void setHostName(String hostName) {
+		this.hostName = hostName;
+	}
+
+	public String getHostName() {
+		return this.hostName;
+	}
+
+	public void setAppType(String appType) {
+		this.appType = appType;
+	}
+
+	public String getAppType() {
+		return this.appType;
+	}
+
+	public void setEntityType(Integer entityType) {
+		this.entityType = entityType;
+	}
+
+	public Integer getEntityType() {
+		return this.entityType;
+	}
+
+	public void setIpAddress(String ipAddress) {
+		this.ipAddress = ipAddress;
+	}
+
+	public String getIpAddress() {
+		return this.ipAddress;
+	}
+
+	public void setDownloadedVersion(Long downloadedVersion) {
+		this.downloadedVersion = downloadedVersion;
+	}
+
+	public Long getDownloadedVersion() {
+		return this.downloadedVersion;
+	}
+
+	public void setDownloadTime( Date downloadTime ) {
+		this.downloadTime = downloadTime;
+	}
+
+	public Date getDownloadTime( ) {
+		return this.downloadTime;
+	}
+
+	public void setActiveVersion(Long activeVersion) {
+		this.activeVersion = activeVersion;
+	}
+
+	public Long getActiveVersion() {
+		return this.activeVersion;
+	}
+
+	public void setActivationTime( Date activationTime ) {
+		this.activationTime = activationTime;
+	}
+
+	public Date getActivationTime( ) {
+		return this.activationTime;
+	}
+
+
+	/**
+	 * This return the bean content in string format
+	 * @return formatedStr
+	*/
+	@Override
+	public String toString( ) {
+		String str = "XXPluginServiceVersionInfo={";
+		str += "id={" + id + "} ";
+		str += "createTime={" + createTime + "} ";
+		str += "serviceName={" + serviceName + "} ";
+		str += "hostName={" + hostName + "} ";
+		str += "appType={" + appType + "} ";
+		str += "entityType={" + entityType + "} ";
+		str += "ipAddress={" + ipAddress + "} ";
+		str += "downloadedVersion={" + downloadedVersion + "} ";
+		str += "downloadTime={" + downloadTime + "} ";
+		str += "activeVersion={" + activeVersion + "} ";
+		str += "activationTime={" + activationTime + "} ";
+		str += "}";
+		return str;
+	}
+
+	/**
+	 * Checks for all attributes except referenced db objects
+	 * @return true if all attributes match
+	*/
+	@Override
+	public boolean equals( Object obj) {
+		if (obj == null)
+			return false;
+		if (this == obj)
+			return true;
+		if (!super.equals(obj))
+			return false;
+		if (getClass() != obj.getClass())
+			return false;
+		XXPluginServiceVersionInfo other = (XXPluginServiceVersionInfo) obj;
+		if ((this.id == null && other.id != null) || (this.id != null && !this.id.equals(other.id))) {
+			return false;
+		}
+		if ((this.createTime == null && other.createTime != null) || (this.createTime != null && !this.createTime.equals(other.createTime))) {
+			return false;
+		}
+		if ((this.serviceName == null && other.serviceName != null) || (this.serviceName != null && !this.serviceName.equals(other.serviceName))) {
+			return false;
+		}
+		if ((this.hostName == null && other.hostName != null) || (this.hostName != null && !this.hostName.equals(other.hostName))) {
+			return false;
+		}
+		if ((this.appType == null && other.appType != null) || (this.appType != null && !this.appType.equals(other.appType))) {
+			return false;
+		}
+		if ((this.entityType == null && other.entityType != null) || (this.entityType != null && !this.entityType.equals(other.entityType))) {
+			return false;
+		}
+		if ((this.ipAddress == null && other.ipAddress != null) || (this.ipAddress != null && !this.ipAddress.equals(other.ipAddress))) {
+			return false;
+		}
+		if ((this.downloadedVersion == null && other.downloadedVersion != null) || (this.downloadedVersion != null && !this.downloadedVersion.equals(other.downloadedVersion))) {
+			return false;
+		}
+		if ((this.downloadTime == null && other.downloadTime != null) || (this.downloadTime != null && !this.downloadTime.equals(other.downloadTime))) {
+			return false;
+		}
+		if ((this.activeVersion == null && other.activeVersion != null) || (this.activeVersion != null && !this.activeVersion.equals(other.activeVersion))) {
+			return false;
+		}
+		if ((this.activationTime == null && other.activationTime != null) || (this.activationTime != null && !this.activationTime.equals(other.activationTime))) {
+			return false;
+		}
+		return true;
+	}
+
+	public static boolean equals(Object object1, Object object2) {
+		if (object1 == object2) {
+			return true;
+		}
+		if ((object1 == null) || (object2 == null)) {
+			return false;
+		}
+		return object1.equals(object2);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/src/main/java/org/apache/ranger/rest/AssetREST.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/rest/AssetREST.java b/security-admin/src/main/java/org/apache/ranger/rest/AssetREST.java
index 3d2997b..52c141b 100644
--- a/security-admin/src/main/java/org/apache/ranger/rest/AssetREST.java
+++ b/security-admin/src/main/java/org/apache/ranger/rest/AssetREST.java
@@ -535,7 +535,7 @@ public class AssetREST {
 		ServicePolicies servicePolicies = null;
 
 		try {
-			servicePolicies = serviceREST.getServicePoliciesIfUpdated(repository, lastKnowPolicyVersion, agentId, request);
+			servicePolicies = serviceREST.getServicePoliciesIfUpdated(repository, lastKnowPolicyVersion, 0L, agentId, request);
 		} catch(Exception excp) {
 			logger.error("failed to retrieve policies for repository " + repository, excp);
 		}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/3bbc3702/security-admin/src/main/java/org/apache/ranger/rest/PublicAPIsv2.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/rest/PublicAPIsv2.java b/security-admin/src/main/java/org/apache/ranger/rest/PublicAPIsv2.java
index e2a6435..cba4877 100644
--- a/security-admin/src/main/java/org/apache/ranger/rest/PublicAPIsv2.java
+++ b/security-admin/src/main/java/org/apache/ranger/rest/PublicAPIsv2.java
@@ -23,10 +23,12 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 import org.apache.ranger.common.RESTErrorUtil;
 import org.apache.ranger.common.annotation.RangerAnnotationJSMgrName;
+import org.apache.ranger.plugin.model.RangerPluginServiceVersionInfo;
 import org.apache.ranger.plugin.model.RangerPolicy;
 import org.apache.ranger.plugin.model.RangerService;
 import org.apache.ranger.plugin.model.RangerServiceDef;
 import org.apache.ranger.plugin.util.SearchFilter;
+import org.apache.ranger.view.RangerPluginServiceVersionInfoList;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.context.annotation.Scope;
 import org.springframework.security.access.prepost.PreAuthorize;
@@ -432,4 +434,19 @@ public class PublicAPIsv2 {
 			logger.debug("<== PublicAPIsv2.deletePolicyByName(" + serviceName + "," + policyName + ")");
 		}
 	}
+
+	@GET
+	@Path("/api/plugins/versions")
+	public List<RangerPluginServiceVersionInfo> getPluginServiceVersions(@Context HttpServletRequest request) {
+		if (logger.isDebugEnabled()) {
+			logger.debug("==> PublicAPIsv2.getPluginServiceVersions()");
+		}
+
+		RangerPluginServiceVersionInfoList pluginServiceVersionInfos = serviceREST.getPluginServiceVersions(request);
+
+		if (logger.isDebugEnabled()) {
+			logger.debug("<== PublicAPIsv2.getPluginServiceVersions()");
+		}
+		return pluginServiceVersionInfos.getPluginServiceVersionInfos();
+	}
 }