You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by bo...@apache.org on 2015/05/16 00:04:51 UTC

incubator-ranger git commit: RANGER-178 - Solr Plugin - Initial code working

Repository: incubator-ranger
Updated Branches:
  refs/heads/master 6222b9d09 -> a7ca7c674


RANGER-178 - Solr Plugin - Initial code working

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

Branch: refs/heads/master
Commit: a7ca7c67488cddcd85364bbfcb91fc9f36d1b504
Parents: 6222b9d
Author: Don Bosco Durai <bo...@apache.org>
Authored: Fri May 15 14:06:42 2015 -0700
Committer: Don Bosco Durai <bo...@apache.org>
Committed: Fri May 15 14:06:42 2015 -0700

----------------------------------------------------------------------
 .../plugin/audit/RangerDefaultAuditHandler.java |  37 ++-
 .../audit/RangerMultiResourceAuditHandler.java  |  73 +++++
 .../service-defs/ranger-servicedef-solr.json    |  68 ++---
 plugin-solr/pom.xml                             | 107 ++++---
 .../solr/authorizer/RangerSolrAuthorizer.java   | 303 ++++++++++++++++++-
 .../ranger/services/solr/RangerServiceSolr.java |   2 +-
 .../services/solr/client/ServiceSolrClient.java |   2 +-
 .../solr/client/ServiceSolrConnectionMgr.java   |   4 +-
 plugin-yarn/.gitignore                          |   1 +
 pom.xml                                         |  11 +-
 src/main/assembly/plugin-solr.xml               |   9 -
 11 files changed, 486 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java b/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java
index fd22852..0d38224 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java
@@ -100,13 +100,12 @@ public class RangerDefaultAuditHandler implements RangerAccessResultProcessor {
 			ret.setAccessType(request.getAction());
 			ret.setAccessResult((short)(result.getIsAllowed() ? 1 : 0));
 			ret.setPolicyId(result.getPolicyId());
-			ret.setAclEnforcer("ranger-acl"); // TODO: review
 			ret.setAction(request.getAccessType());
 			ret.setClientIP(request.getClientIPAddress());
 			ret.setClientType(request.getClientType());
-			ret.setAgentHostname(null);
-			ret.setAgentId(null);
-			ret.setEventId(null);
+
+			populateDefaults(ret);
+
 		}
 
 		if(LOG.isDebugEnabled()) {
@@ -153,17 +152,7 @@ public class RangerDefaultAuditHandler implements RangerAccessResultProcessor {
 		}
 
 		if(auditEvent != null) {
-			if (auditEvent.getAgentHostname() == null || auditEvent.getAgentHostname().isEmpty()) {
-				auditEvent.setAgentHostname(MiscUtil.getHostname());
-			}
-
-			if (auditEvent.getLogType() == null || auditEvent.getLogType().isEmpty()) {
-				auditEvent.setLogType("RangerAudit");
-			}
-
-			if (auditEvent.getEventId() == null || auditEvent.getEventId().isEmpty()) {
-				auditEvent.setEventId(MiscUtil.generateUniqueId());
-			}
+			populateDefaults(auditEvent);
 			AuditProviderFactory.getAuditProvider().log(auditEvent);
 		}
 
@@ -172,6 +161,24 @@ public class RangerDefaultAuditHandler implements RangerAccessResultProcessor {
 		}
 	}
 
+	private void populateDefaults(AuthzAuditEvent auditEvent) {
+		if( auditEvent.getAclEnforcer() == null || auditEvent.getAclEnforcer().isEmpty()) {
+			auditEvent.setAclEnforcer("ranger-acl"); // TODO: review
+		}
+
+		if (auditEvent.getAgentHostname() == null || auditEvent.getAgentHostname().isEmpty()) {
+			auditEvent.setAgentHostname(MiscUtil.getHostname());
+		}
+
+		if (auditEvent.getLogType() == null || auditEvent.getLogType().isEmpty()) {
+			auditEvent.setLogType("RangerAudit");
+		}
+
+		if (auditEvent.getEventId() == null || auditEvent.getEventId().isEmpty()) {
+			auditEvent.setEventId(MiscUtil.generateUniqueId());
+		}		
+	}
+
 	public void logAuthzAudits(Collection<AuthzAuditEvent> auditEvents) {
 		if(LOG.isDebugEnabled()) {
 			LOG.debug("==> RangerDefaultAuditHandler.logAuthzAudits(" + auditEvents + ")");

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerMultiResourceAuditHandler.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerMultiResourceAuditHandler.java b/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerMultiResourceAuditHandler.java
new file mode 100644
index 0000000..f40d39f
--- /dev/null
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerMultiResourceAuditHandler.java
@@ -0,0 +1,73 @@
+/*
+ * 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.audit;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.ranger.audit.model.AuthzAuditEvent;
+import org.apache.ranger.plugin.policyengine.RangerAccessResult;
+
+/**
+ * This class should be generated per request and flushed at the end of the
+ * request
+ */
+public class RangerMultiResourceAuditHandler extends RangerDefaultAuditHandler {
+
+	Collection<AuthzAuditEvent> auditEvents = new ArrayList<AuthzAuditEvent>();
+	boolean deniedExists = false;
+
+	public RangerMultiResourceAuditHandler() {
+	}
+
+
+	@Override
+	public void logAuthzAudit(AuthzAuditEvent auditEvent) {
+		auditEvents.add(auditEvent);
+	}
+
+	@Override
+	public void logAuthzAudits(Collection<AuthzAuditEvent> auditEvents) {
+		auditEvents.addAll(auditEvents);
+	}
+
+	public void flushAudit() {
+		try {
+			boolean deniedExists = false;
+			// First iterate to see if there are any denied
+			for (AuthzAuditEvent auditEvent : auditEvents) {
+				if (auditEvent.getAccessResult() == 0) {
+					deniedExists = true;
+					break;
+				}
+			}
+
+			for (AuthzAuditEvent auditEvent : auditEvents) {
+				if (deniedExists && auditEvent.getAccessResult() != 0) {
+					continue;
+				}
+
+				super.logAuthzAudit(auditEvent);
+			}
+		} catch (Throwable t) {
+
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/agents-common/src/main/resources/service-defs/ranger-servicedef-solr.json
----------------------------------------------------------------------
diff --git a/agents-common/src/main/resources/service-defs/ranger-servicedef-solr.json b/agents-common/src/main/resources/service-defs/ranger-servicedef-solr.json
index c2710e7..2b492ae 100644
--- a/agents-common/src/main/resources/service-defs/ranger-servicedef-solr.json
+++ b/agents-common/src/main/resources/service-defs/ranger-servicedef-solr.json
@@ -5,7 +5,7 @@
 	"description":"Solr",
 	"resources":[
 		{
-			"itemId": 1,
+			"itemId":100,
 			"name":"collection",
 			"type":"string",
 			"level":10,
@@ -15,51 +15,36 @@
 			"recursiveSupported":false,
 			"excludesSupported":true,
 			"matcher":"org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher",
-			"matcherOptions":{ "wildCard":true, "ignoreCase":true },
+			"matcherOptions":{
+				"wildCard":true,
+				"ignoreCase":true
+			},
 			"validationRegEx":"",
 			"validationMessage":"",
 			"uiHint":"",
 			"label":"Solr Collection",
 			"description":"Solr Collection"
-		},
-		{
-			"itemId": 2,
-			"name":"field",
-			"type":"string",
-			"level":20,
-			"parent":"collection",
-			"mandatory":true,
-			"lookupSupported":true,
-			"recursiveSupported":false,
-			"excludesSupported":true,
-			"matcher":"org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher",
-			"matcherOptions":{ "wildCard":true, "ignoreCase":true },
-			"validationRegEx":"",
-			"validationMessage":"",
-			"uiHint":"",
-			"label":"Field",
-			"description":"Field"
 		}
 		
 	],
 	"accessTypes":[
 		{
-			"itemId": 1,
-			"name":"create",
-			"label":"Create"
+			"itemId":100,
+			"name":"query",
+			"label":"Query"
 		},
 		{
-			"itemId": 2,
+			"itemId":200,
 			"name":"update",
 			"label":"Update"
 		},
 		{
-			"itemId": 3,
-			"name":"query",
-			"label":"Query"
+			"itemId":300,
+			"name":"others",
+			"label":"Others"
 		},
 		{
-			"itemId": 4,
+			"itemId":900,
 			"name":"solr_admin",
 			"label":"Solr Admin"
 		}
@@ -67,7 +52,7 @@
 	],
 	"configs":[
 		{
-			"itemId": 1,
+			"itemId":100,
 			"name":"username",
 			"type":"string",
 			"mandatory":true,
@@ -77,7 +62,7 @@
 			"label":"Username"
 		},
 		{
-			"itemId": 2,
+			"itemId":200,
 			"name":"password",
 			"type":"password",
 			"mandatory":true,
@@ -87,7 +72,7 @@
 			"label":"Password"
 		},
 		{
-			"itemId": 3,
+			"itemId":400,
 			"name":"solr.url",
 			"type":"string",
 			"mandatory":true,
@@ -98,14 +83,14 @@
 			"label":"Solr URL"
 		},
 		{
-			"itemId": 4,
+			"itemId":500,
 			"name":"commonNameForCertificate",
 			"type":"string",
 			"mandatory":false,
 			"validationRegEx":"",
 			"validationMessage":"",
 			"uiHint":"",
-			"label":"Common Name for Certificate"
+			"label":"Ranger Plugin SSL CName"
 		}
 		
 	],
@@ -117,15 +102,18 @@
 	],
 	"policyConditions":[
 		{
-			"itemId": 1,
-			"name": "ip-range",
-			"evaluator": "org.apache.ranger.plugin.conditionevaluator.RangerIpMatcher",
-			"evaluatorOptions": { },
+			"itemId":100,
+			"name":"ip-range",
+			"evaluator":"org.apache.ranger.plugin.conditionevaluator.RangerIpMatcher",
+			"evaluatorOptions":{
+				
+			},
 			"validationRegEx":"",
-			"validationMessage": "",
+			"validationMessage":"",
 			"uiHint":"",
-			"label": "IP Address Range",
-			"description": "IP Address Range"
+			"label":"IP Address Range",
+			"description":"IP Address Range"
 		}
+		
 	]
 }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/plugin-solr/pom.xml
----------------------------------------------------------------------
diff --git a/plugin-solr/pom.xml b/plugin-solr/pom.xml
index 54bcafa..e49865e 100644
--- a/plugin-solr/pom.xml
+++ b/plugin-solr/pom.xml
@@ -1,56 +1,55 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <groupId>security_plugins.ranger-solr-plugin</groupId>
-  <artifactId>ranger-solr-plugin</artifactId>
-  <name>SOLR Security Plugin</name>
-  <description>SOLR Security Plugin</description>
-  <packaging>jar</packaging>
-  <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-  </properties>
-  <parent>
-     <groupId>org.apache.ranger</groupId>
-     <artifactId>ranger</artifactId>
-     <version>0.5.0</version>
-     <relativePath>..</relativePath>
-  </parent>
-  <dependencies>
-    <dependency>
-      <groupId>security_plugins.ranger-plugins-common</groupId>
-      <artifactId>ranger-plugins-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>security_plugins.ranger-plugins-audit</groupId>
-      <artifactId>ranger-plugins-audit</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.ranger</groupId>
-      <artifactId>credentialbuilder</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.ranger</groupId>
-      <artifactId>ranger_solrj</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-  </dependencies>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<groupId>security_plugins.ranger-solr-plugin</groupId>
+	<artifactId>ranger-solr-plugin</artifactId>
+	<name>SOLR Security Plugin</name>
+	<description>SOLR Security Plugin</description>
+	<packaging>jar</packaging>
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+	</properties>
+	<parent>
+		<groupId>org.apache.ranger</groupId>
+		<artifactId>ranger</artifactId>
+		<version>0.5.0</version>
+		<relativePath>..</relativePath>
+	</parent>
+	<dependencies>
+		<dependency>
+			<groupId>security_plugins.ranger-plugins-common</groupId>
+			<artifactId>ranger-plugins-common</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>security_plugins.ranger-plugins-audit</groupId>
+			<artifactId>ranger-plugins-audit</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.ranger</groupId>
+			<artifactId>credentialbuilder</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.solr</groupId>
+			<artifactId>solr-core</artifactId>
+			<version>6.0</version>
+			<scope>system</scope>
+			<systemPath>/Users/bdurai/svn/solr/trunk/solr/build/solr-core/solr-core-6.0.0-SNAPSHOT.jar</systemPath>
+		</dependency>
+		<!-- <dependency> <groupId>org.apache.solr</groupId> <artifactId>solr-core</artifactId> 
+			<version>${solr.version}</version> </dependency> -->
+	</dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/plugin-solr/src/main/java/org/apache/ranger/authorization/solr/authorizer/RangerSolrAuthorizer.java
----------------------------------------------------------------------
diff --git a/plugin-solr/src/main/java/org/apache/ranger/authorization/solr/authorizer/RangerSolrAuthorizer.java b/plugin-solr/src/main/java/org/apache/ranger/authorization/solr/authorizer/RangerSolrAuthorizer.java
index 8ccc703..de786d7 100644
--- a/plugin-solr/src/main/java/org/apache/ranger/authorization/solr/authorizer/RangerSolrAuthorizer.java
+++ b/plugin-solr/src/main/java/org/apache/ranger/authorization/solr/authorizer/RangerSolrAuthorizer.java
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -20,18 +19,306 @@
 
 package org.apache.ranger.authorization.solr.authorizer;
 
+import java.io.IOException;
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.ranger.authorization.hadoop.config.RangerConfiguration;
+import org.apache.ranger.authorization.utils.StringUtil;
+import org.apache.ranger.plugin.audit.RangerMultiResourceAuditHandler;
+import org.apache.ranger.plugin.policyengine.RangerAccessRequestImpl;
+import org.apache.ranger.plugin.policyengine.RangerAccessResourceImpl;
+import org.apache.ranger.plugin.policyengine.RangerAccessResult;
+import org.apache.ranger.plugin.service.RangerBasePlugin;
+import org.apache.solr.security.AuthorizationContext.RequestType;
+import org.apache.solr.security.AuthorizationPlugin;
+import org.apache.solr.security.AuthorizationResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.AuthorizationContext.CollectionRequest;
+
+public class RangerSolrAuthorizer implements AuthorizationPlugin {
+	private static final Log logger = LogFactory
+			.getLog(RangerSolrAuthorizer.class);
+
+	public static final String PROP_USE_PROXY_IP = "xasecure.solr.use_proxy_ip";
+	public static final String PROP_PROXY_IP_HEADER = "xasecure.solr.proxy_ip_header";
+
+	public static final String KEY_COLLECTION = "collection";
+
+	public static final String ACCESS_TYPE_CREATE = "create";
+	public static final String ACCESS_TYPE_UPDATE = "update";
+	public static final String ACCESS_TYPE_QUERY = "query";
+	public static final String ACCESS_TYPE_OTHER = "other";
+	public static final String ACCESS_TYPE_ADMIN = "admin";
+
+	private static volatile RangerBasePlugin solrPlugin = null;
+
+	boolean useProxyIP = false;
+	String proxyIPHeader = "HTTP_X_FORWARDED_FOR";
+
+	public RangerSolrAuthorizer() {
+		logger.info("RangerSolrAuthorizer()");
+		if (solrPlugin == null) {
+			logger.info("RangerSolrAuthorizer(): init called");
+			solrPlugin = new RangerBasePlugin("solr", "solr");
+		}
+	}
+
+	/*
+	 * (non-Javadoc)
+	 * 
+	 * @see org.apache.solr.security.SolrAuthorizationPlugin#init(java.util.Map)
+	 */
+	@Override
+	public void init(Map<String, Object> initInfo) {
+		logger.info("init()");
+
+		try {
+			solrPlugin.init();
+
+			useProxyIP = RangerConfiguration.getInstance().getBoolean(
+					PROP_USE_PROXY_IP, useProxyIP);
+			proxyIPHeader = RangerConfiguration.getInstance().get(
+					PROP_PROXY_IP_HEADER, proxyIPHeader);
+
+		} catch (Throwable t) {
+			logger.fatal("Error init", t);
+		}
+	}
+
+	/*
+	 * (non-Javadoc)
+	 * 
+	 * @see java.io.Closeable#close()
+	 */
+	@Override
+	public void close() throws IOException {
+		logger.info("close() called");
+		try {
+			solrPlugin.cleanup();
+		} catch (Throwable t) {
+			logger.error("Error cleaning up Ranger plugin. Ignoring error", t);
+		}
+	}
+
+	/*
+	 * (non-Javadoc)
+	 * 
+	 * @see
+	 * org.apache.solr.security.SolrAuthorizationPlugin#authorize(org.apache
+	 * .solr.security.SolrRequestContext)
+	 */
+	@Override
+	public AuthorizationResponse authorize(AuthorizationContext context) {
+		// TODO: Change this to Debug only
+		if (logger.isInfoEnabled()) {
+			logAuthorizationConext(context);
+		}
+
+		RangerMultiResourceAuditHandler auditHandler = new RangerMultiResourceAuditHandler();
+
+		String userName = null;
+		Set<String> userGroups = null;
+		String ip = null;
+		Date eventTime = StringUtil.getUTCDate();
+
+		// Set the User and Groups
+		Principal principal = context.getUserPrincipal();
+		if (principal != null) {
+			userName = StringUtils.substringBefore(principal.getName(), "@");
+			userGroups = getGroupsForUser(userName);
+		}
+
+		// // Set the IP
+		if (useProxyIP) {
+			ip = context.getHttpHeader(proxyIPHeader);
+		}
+		if (ip == null) {
+			ip = context.getHttpHeader("REMOTE_ADDR");
+		}
+
+		String requestData = context.getResource() + ":" + context.getParams();
+
+		// Create the list of requests for access check. Each field is broken
+		// into a request
+		List<RangerAccessRequestImpl> rangerRequests = new ArrayList<RangerAccessRequestImpl>();
+		for (CollectionRequest collectionRequest : context
+				.getCollectionRequests()) {
+
+			List<RangerAccessRequestImpl> requestsForCollection = createRequests(
+					userName, userGroups, ip, eventTime, context,
+					collectionRequest, requestData);
+			rangerRequests.addAll(requestsForCollection);
+		}
+
+		boolean isDenied = false;
+		try {
+			// Let's check the access for each request/resource
+			for (RangerAccessRequestImpl rangerRequest : rangerRequests) {
+				RangerAccessResult result = solrPlugin.isAccessAllowed(
+						rangerRequest, auditHandler);
+				if (result == null || !result.getIsAllowed()) {
+					isDenied = true;
+					// rejecting on first failure
+					break;
+				}
+			}
+		} finally {
+			auditHandler.flushAudit();
+		}
+
+		AuthorizationResponse response = null;
+		if (isDenied) {
+			response = new AuthorizationResponse(403);
+		} else {
+			response = new AuthorizationResponse(200);
+		}
+		return response;
+	}
+
+	/**
+	 * @param context
+	 */
+	private void logAuthorizationConext(AuthorizationContext context) {
+		String collections = "";
+		int i = -1;
+		for (CollectionRequest collectionRequest : context
+				.getCollectionRequests()) {
+			i++;
+			if (i > 0) {
+				collections += ",";
+			}
+			collections += collectionRequest.collectionName;
+		}
+
+		String headers = "";
+		i = -1;
+		@SuppressWarnings("unchecked")
+		Enumeration<String> eList = context.getHeaderNames();
+		while (eList.hasMoreElements()) {
+			i++;
+			if (i > 0) {
+				headers += ",";
+			}
+			String header = eList.nextElement();
+			String value = context.getHttpHeader(header);
+			headers += header + "=" + value;
+		}
+
+		String ipAddress = context.getHttpHeader("HTTP_X_FORWARDED_FOR");
+
+		if (ipAddress == null) {
+			ipAddress = context.getHttpHeader("REMOTE_ADDR");
+		}
+
+		Principal principal = context.getUserPrincipal();
+		String userName = null;
+		if (principal != null) {
+			userName = principal.getName();
+			userName = StringUtils.substringBefore(userName, "@");
+		}
+
+		logger.info("AuthorizationContext: context.getResource()="
+				+ context.getResource() + ", solarParams="
+				+ context.getParams() + ", requestType="
+				+ context.getRequestType() + ", userPrincipal="
+				+ context.getUserPrincipal() + ", userName=" + userName
+				+ ", ipAddress=" + ipAddress + ", collections=" + collections
+				+ ", headers=" + headers);
+
+	}
+
+	/**
+	 * @param userName
+	 * @param userGroups
+	 * @param ip
+	 * @param eventTime
+	 * @param context
+	 * @param collectionRequest
+	 * @param requestData
+	 * @return
+	 */
+	private List<RangerAccessRequestImpl> createRequests(String userName,
+			Set<String> userGroups, String ip, Date eventTime,
+			AuthorizationContext context, CollectionRequest collectionRequest,
+			String requestData) {
+
+		List<RangerAccessRequestImpl> requests = new ArrayList<RangerAccessRequestImpl>();
+		String accessType = mapToRangerAccessType(context);
+		String action = accessType;
+
+		if (collectionRequest.collectionName != null) {
+			RangerAccessRequestImpl rangerRequest = createBaseRequest(userName,
+					userGroups, ip, eventTime);
+			RangerAccessResourceImpl rangerResource = new RangerAccessResourceImpl();
+			rangerResource.setValue(KEY_COLLECTION,
+					collectionRequest.collectionName);
+			rangerRequest.setResource(rangerResource);
+			rangerRequest.setAccessType(accessType);
+			rangerRequest.setAction(action);
+
+			requests.add(rangerRequest);
+		} else {
+			logger.fatal("Can't create RangerRequest oject. userName="
+					+ userName + ", accessType=" + accessType + ", ip=" + ip
+					+ ", collectionRequest=" + collectionRequest);
+		}
+
+		return requests;
+	}
+
+	private RangerAccessRequestImpl createBaseRequest(String userName,
+			Set<String> userGroups, String ip, Date eventTime) {
+		RangerAccessRequestImpl rangerRequest = new RangerAccessRequestImpl();
+		if (userName != null && !userName.isEmpty()) {
+			rangerRequest.setUser(userName);
+		}
+		if (userGroups != null && userGroups.size() > 0) {
+			rangerRequest.setUserGroups(userGroups);
+		}
+		if (ip != null && !ip.isEmpty()) {
+			rangerRequest.setClientIPAddress(ip);
+		}
+		rangerRequest.setAccessTime(eventTime);
+		return rangerRequest;
+	}
 
-public class RangerSolrAuthorizer /*SolrAuthorizationPlugin*/ {
-    public static final String ACCESS_TYPE_CREATE = "create";
-    public static final String ACCESS_TYPE_UPDATE  = "update";
-    public static final String ACCESS_TYPE_QUERY  = "query";
-    public static final String ACCESS_TYPE_ADMIN       = "admin";
+	/**
+	 * @param name
+	 * @return
+	 */
+	private Set<String> getGroupsForUser(String name) {
+		// TODO: Need to implement this method
 
-	private static final Log LOG = LogFactory.getLog(RangerSolrAuthorizer.class);
+		return null;
+	}
 
-    //private static volatile RangerSolrPlugin solrPlugin = null;
+	String mapToRangerAccessType(AuthorizationContext context) {
+		String accessType = ACCESS_TYPE_OTHER;
 
+		RequestType requestType = context.getRequestType();
+		if (requestType.equals(RequestType.ADMIN)) {
+			accessType = ACCESS_TYPE_ADMIN;
+		} else if (requestType.equals(RequestType.READ)) {
+			accessType = ACCESS_TYPE_QUERY;
+		} else if (requestType.equals(RequestType.WRITE)) {
+			accessType = ACCESS_TYPE_UPDATE;
+		} else if (requestType.equals(RequestType.UNKNOWN)) {
+			logger.info("UNKNOWN request type. Mapping it to " + accessType);
+			accessType = ACCESS_TYPE_OTHER;
+		} else {
+			logger.info("Request type is not supported. requestType="
+					+ requestType + ". Mapping it to " + accessType);
+		}
+		return accessType;
+	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/plugin-solr/src/main/java/org/apache/ranger/services/solr/RangerServiceSolr.java
----------------------------------------------------------------------
diff --git a/plugin-solr/src/main/java/org/apache/ranger/services/solr/RangerServiceSolr.java b/plugin-solr/src/main/java/org/apache/ranger/services/solr/RangerServiceSolr.java
index 3a43a9e..1474b29 100644
--- a/plugin-solr/src/main/java/org/apache/ranger/services/solr/RangerServiceSolr.java
+++ b/plugin-solr/src/main/java/org/apache/ranger/services/solr/RangerServiceSolr.java
@@ -56,7 +56,7 @@ public class RangerServiceSolr extends RangerBaseService {
 				ret = ServiceSolrConnectionMgr.testConnection(serviceName,
 						configs);
 			} catch (Exception e) {
-				LOG.error("<== RangerServiceSolr.validateConfig Error:" + e);
+				LOG.error("<== RangerServiceSolr.validateConfig Error:", e);
 				throw e;
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrClient.java
----------------------------------------------------------------------
diff --git a/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrClient.java b/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrClient.java
index 2e6d0ac..6a192f4 100644
--- a/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrClient.java
+++ b/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrClient.java
@@ -51,7 +51,7 @@ public class ServiceSolrClient {
 	}
 
 	SolrClient solrClient = null;
-	boolean isSolrCloud = false;
+	boolean isSolrCloud = true;
 
 	String serviceName = null;
 	private static final String errMessage = " You can still save the repository and start creating "

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrConnectionMgr.java
----------------------------------------------------------------------
diff --git a/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrConnectionMgr.java b/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrConnectionMgr.java
index 874fca5..7eeb5cd 100644
--- a/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrConnectionMgr.java
+++ b/plugin-solr/src/main/java/org/apache/ranger/services/solr/client/ServiceSolrConnectionMgr.java
@@ -34,9 +34,11 @@ public class ServiceSolrConnectionMgr {
 			Map<String, String> configs) throws Exception {
 		String url = configs.get("solr.url");
 		if (url != null) {
+			//TODO: Determine whether the instance is SolrCloud
+			boolean isSolrCloud = true;
 			SolrClient solrClient = new HttpSolrClient(url);
 			ServiceSolrClient serviceSolrClient = new ServiceSolrClient(
-					serviceName, solrClient, false);
+					serviceName, solrClient, isSolrCloud);
 			return serviceSolrClient;
 		}
 		// TODO: Need to add method to create SolrClient using ZooKeeper for

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/plugin-yarn/.gitignore
----------------------------------------------------------------------
diff --git a/plugin-yarn/.gitignore b/plugin-yarn/.gitignore
index ea8c4bf..d9d66d8 100644
--- a/plugin-yarn/.gitignore
+++ b/plugin-yarn/.gitignore
@@ -1 +1,2 @@
 /target
+/bin/

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 27dda4d..0b5608a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -88,7 +88,7 @@
   <module>knox-agent</module>
   <module>storm-agent</module>
   <module>plugin-kafka</module>
-  <module>plugin-solr</module>
+  <!-- <module>plugin-solr</module> -->
   <module>plugin-yarn</module>
   <module>ranger_solrj</module>
   <module>security-admin</module>
@@ -160,7 +160,7 @@
 		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
 		<security-agent-install-dir>hadoop-security/plugins</security-agent-install-dir>
 		<slf4j-api.version>1.7.5</slf4j-api.version>
-		<!--<solr.version>5.0.0</solr.version>-->
+		<solr.version>5.1.0</solr.version>
 		<ranger.solrj.version>${project.version}</ranger.solrj.version>
 		<springframework.spring.version>2.5.6</springframework.spring.version>
 		<!--
@@ -218,6 +218,12 @@
             <module>unixauthnative</module>
           </modules>
       </profile>
+      <profile>
+          <id>solr-security</id>
+         <modules>
+        	 <module>plugin-solr</module>         
+         </modules>
+      </profile>
   </profiles>
   <distributionManagement>
         <repository>
@@ -482,6 +488,7 @@
           <exclude>**/.externalToolBuilders/*</exclude>
           <exclude>*.patch</exclude>
           <exclude>atlassian-ide-plugin.xml</exclude>
+	  <exclude>**/.pydevproject</exclude>
         </excludes>
 
       </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/a7ca7c67/src/main/assembly/plugin-solr.xml
----------------------------------------------------------------------
diff --git a/src/main/assembly/plugin-solr.xml b/src/main/assembly/plugin-solr.xml
index 954ea52..06844ae 100644
--- a/src/main/assembly/plugin-solr.xml
+++ b/src/main/assembly/plugin-solr.xml
@@ -35,20 +35,11 @@
                 <outputDirectory>/lib</outputDirectory>
                 <unpack>false</unpack>
                 <includes>
-                    <include>commons-configuration:commons-configuration:jar:${commons.configuration.version}</include>
-                    <include>org.apache.hadoop:hadoop-common:jar:${hadoop-common.version}</include>
-                    <include>org.apache.hadoop:hadoop-common-plus:jar:${hadoop-common.version}</include>
                     <include>com.google.code.gson:gson</include>
                     <include>org.eclipse.persistence:eclipselink</include>
                     <include>org.eclipse.persistence:javax.persistence</include>
-                    <include>commons-collections:commons-collections</include>
 					<include>com.sun.jersey:jersey-bundle</include>
-                    <include>commons-logging:commons-logging:jar:${commons.logging.version}</include>
                     <include>com.google.guava:guava:jar:${guava.version}</include>
-		    <include>org.apache.httpcomponents:httpclient:jar:${httpcomponent.httpclient.version}</include>
-		    <include>org.apache.httpcomponents:httpcore:jar:${httpcomponent.httpcore.version}</include>
-		    <include>org.apache.httpcomponents:httpmime:jar:${httpcomponent.httpmime.version}</include>
-		    <include>org.noggit:noggit:jar:${noggit.version}</include>
                 </includes>
             </dependencySet>
             <dependencySet>