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 2014/12/12 02:30:14 UTC

[31/51] [partial] incubator-ranger git commit: RANGER-194: Rename packages from xasecure to apache ranger

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java
----------------------------------------------------------------------
diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java
new file mode 100644
index 0000000..d797b1a
--- /dev/null
+++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java
@@ -0,0 +1,849 @@
+/*
+ * 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.authorization.hive.authorizer;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilege;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivObjectActionType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.ranger.admin.client.RangerAdminRESTClient;
+import org.apache.ranger.admin.client.datatype.GrantRevokeData;
+import org.apache.ranger.audit.model.EnumRepositoryType;
+import org.apache.ranger.audit.model.HiveAuditEvent;
+import org.apache.ranger.audit.provider.AuditProviderFactory;
+import org.apache.ranger.authorization.hadoop.config.RangerConfiguration;
+import org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants;
+import org.apache.ranger.authorization.hive.RangerHiveAccessContext;
+import org.apache.ranger.authorization.hive.RangerHiveAccessVerifier;
+import org.apache.ranger.authorization.hive.RangerHiveAccessVerifierFactory;
+import org.apache.ranger.authorization.hive.RangerHiveObjectAccessInfo;
+import org.apache.ranger.authorization.hive.RangerHiveObjectAccessInfo.HiveAccessType;
+import org.apache.ranger.authorization.hive.RangerHiveObjectAccessInfo.HiveObjectType;
+import org.apache.ranger.authorization.utils.StringUtil;
+
+public class RangerHiveAuthorizer extends RangerHiveAuthorizerBase {
+	private static final Log LOG = LogFactory.getLog(RangerHiveAuthorizer.class) ; 
+
+	private static final String RangerModuleName =  RangerConfiguration.getInstance().get(RangerHadoopConstants.AUDITLOG_RANGER_MODULE_ACL_NAME_PROP , RangerHadoopConstants.DEFAULT_RANGER_MODULE_ACL_NAME) ;
+	private static final String repositoryName     = RangerConfiguration.getInstance().get(RangerHadoopConstants.AUDITLOG_REPOSITORY_NAME_PROP);
+	private static final boolean UpdateXaPoliciesOnGrantRevoke = RangerConfiguration.getInstance().getBoolean(RangerHadoopConstants.HIVE_UPDATE_RANGER_POLICIES_ON_GRANT_REVOKE_PROP, RangerHadoopConstants.HIVE_UPDATE_RANGER_POLICIES_ON_GRANT_REVOKE_DEFAULT_VALUE);
+
+	private RangerHiveAccessVerifier mHiveAccessVerifier = null ;
+
+
+	public RangerHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
+								  HiveConf                   hiveConf,
+								  HiveAuthenticationProvider hiveAuthenticator,
+								  HiveAuthzSessionContext    sessionContext) {
+		super(metastoreClientFactory, hiveConf, hiveAuthenticator, sessionContext);
+
+		LOG.debug("RangerHiveAuthorizer.RangerHiveAuthorizer()");
+
+		mHiveAccessVerifier = RangerHiveAccessVerifierFactory.getInstance() ;
+		
+		if(!RangerConfiguration.getInstance().isAuditInitDone()) {
+			if(sessionContext != null) {
+				AuditProviderFactory.ApplicationType appType = AuditProviderFactory.ApplicationType.Unknown;
+
+				switch(sessionContext.getClientType()) {
+					case HIVECLI:
+						appType = AuditProviderFactory.ApplicationType.HiveCLI;
+					break;
+
+					case HIVESERVER2:
+						appType = AuditProviderFactory.ApplicationType.HiveServer2;
+					break;
+				}
+
+				RangerConfiguration.getInstance().initAudit(appType);
+			}
+		}
+	}
+
+
+	/**
+	 * Grant privileges for principals on the object
+	 * @param hivePrincipals
+	 * @param hivePrivileges
+	 * @param hivePrivObject
+	 * @param grantorPrincipal
+	 * @param grantOption
+	 * @throws HiveAuthzPluginException
+	 * @throws HiveAccessControlException
+	 */
+	@Override
+	public void grantPrivileges(List<HivePrincipal> hivePrincipals,
+								List<HivePrivilege> hivePrivileges,
+								HivePrivilegeObject hivePrivObject,
+								HivePrincipal       grantorPrincipal,
+								boolean             grantOption)
+										throws HiveAuthzPluginException, HiveAccessControlException {
+		if(! UpdateXaPoliciesOnGrantRevoke) {
+			throw new HiveAuthzPluginException("GRANT/REVOKE not supported in Argus HiveAuthorizer. Please use Argus Security Admin to setup access control.");
+		}
+
+		boolean                isSuccess     = false;
+		RangerHiveObjectAccessInfo objAccessInfo = getObjectAccessInfo(HiveOperationType.GRANT_PRIVILEGE, hivePrivObject, new RangerHiveAccessContext(null, getHiveAuthzSessionContext()), true);
+
+		try {
+			GrantRevokeData grData = createGrantRevokeData(objAccessInfo, hivePrincipals, hivePrivileges, getGrantorUsername(grantorPrincipal), grantOption);
+
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("grantPrivileges(): " + grData.toJson());
+			}
+
+			RangerAdminRESTClient xaAdmin = new RangerAdminRESTClient();
+
+		    xaAdmin.grantPrivilege(grData);
+
+		    isSuccess = true;
+		} catch(Exception excp) {
+			throw new HiveAccessControlException(excp);
+		} finally {
+			if(mHiveAccessVerifier.isAudited(objAccessInfo)) {
+				UserGroupInformation ugi = this.getCurrentUserGroupInfo();
+
+				// Note: failed return from REST call will be logged as 'DENIED'
+				logAuditEvent(ugi, objAccessInfo, isSuccess);
+			}
+		}
+	}
+
+	/**
+	 * Revoke privileges for principals on the object
+	 * @param hivePrincipals
+	 * @param hivePrivileges
+	 * @param hivePrivObject
+	 * @param grantorPrincipal
+	 * @param grantOption
+	 * @throws HiveAuthzPluginException
+	 * @throws HiveAccessControlException
+	 */
+	@Override
+	public void revokePrivileges(List<HivePrincipal> hivePrincipals,
+								 List<HivePrivilege> hivePrivileges,
+								 HivePrivilegeObject hivePrivObject,
+								 HivePrincipal       grantorPrincipal,
+								 boolean             grantOption)
+										 throws HiveAuthzPluginException, HiveAccessControlException {
+		if(! UpdateXaPoliciesOnGrantRevoke) {
+			throw new HiveAuthzPluginException("GRANT/REVOKE not supported in Argus HiveAuthorizer. Please use Argus Security Admin to setup access control.");
+		}
+
+		boolean                isSuccess     = false;
+		RangerHiveObjectAccessInfo objAccessInfo = getObjectAccessInfo(HiveOperationType.REVOKE_PRIVILEGE, hivePrivObject, new RangerHiveAccessContext(null, getHiveAuthzSessionContext()), true);
+
+		try {
+			GrantRevokeData grData = createGrantRevokeData(objAccessInfo, hivePrincipals, hivePrivileges, getGrantorUsername(grantorPrincipal), grantOption);
+
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("revokePrivileges(): " + grData.toJson());
+			}
+
+			RangerAdminRESTClient xaAdmin = new RangerAdminRESTClient();
+
+		    xaAdmin.revokePrivilege(grData);
+
+		    isSuccess = true;
+		} catch(Exception excp) {
+			throw new HiveAccessControlException(excp);
+		} finally {
+			if(mHiveAccessVerifier.isAudited(objAccessInfo)) {
+				UserGroupInformation ugi = this.getCurrentUserGroupInfo();
+
+				// Note: failed return from REST call will be logged as 'DENIED'
+				logAuditEvent(ugi, objAccessInfo, isSuccess);
+			}
+		}
+	}
+
+	/**
+	 * Check if user has privileges to do this action on these objects
+	 * @param hiveOpType
+	 * @param inputsHObjs
+	 * @param outputHObjs
+	 * @param context
+	 * @throws HiveAuthzPluginException
+	 * @throws HiveAccessControlException
+	 */
+	@Override
+	public void checkPrivileges(HiveOperationType         hiveOpType,
+								List<HivePrivilegeObject> inputHObjs,
+							    List<HivePrivilegeObject> outputHObjs,
+							    HiveAuthzContext          context)
+		      throws HiveAuthzPluginException, HiveAccessControlException {
+
+		UserGroupInformation ugi =  this.getCurrentUserGroupInfo();
+
+		if(ugi == null) {
+			throw new HiveAccessControlException("Permission denied: user information not available");
+		}
+
+		RangerHiveAccessContext hiveContext = this.getAccessContext(context);
+
+		if(LOG.isDebugEnabled()) {
+			LOG.debug(toString(hiveOpType, inputHObjs, outputHObjs, hiveContext));
+		}
+		
+		if(hiveOpType == HiveOperationType.DFS) {
+			handleDfsCommand(hiveOpType, inputHObjs, outputHObjs, hiveContext);
+			
+			return;
+		}
+
+		List<RangerHiveObjectAccessInfo> objAccessList = getObjectAccessInfo(hiveOpType, inputHObjs, outputHObjs, hiveContext);
+
+		for(RangerHiveObjectAccessInfo objAccessInfo : objAccessList) {
+            boolean ret = false;
+
+            if(objAccessInfo.getObjectType() == HiveObjectType.URI) {
+                ret = isURIAccessAllowed(ugi, objAccessInfo.getAccessType(), objAccessInfo.getUri(), getHiveConf());
+            } else if(objAccessInfo.getAccessType() != HiveAccessType.ADMIN) {
+                ret = mHiveAccessVerifier.isAccessAllowed(ugi, objAccessInfo);
+            }
+
+			if(! ret) {
+				if(mHiveAccessVerifier.isAudited(objAccessInfo)) {
+					logAuditEvent(ugi, objAccessInfo, false);
+				}
+				
+				String deniedObjectName = objAccessInfo.getDeinedObjectName();
+				
+				if(StringUtil.isEmpty(deniedObjectName)) {
+					deniedObjectName = objAccessInfo.getObjectName();
+				}
+
+				throw new HiveAccessControlException(String.format("Permission denied: user [%s] does not have [%s] privilege on [%s]",
+													 ugi.getShortUserName(), objAccessInfo.getAccessType().name(), deniedObjectName));
+			}
+		}
+
+		// access is allowed; audit all accesses
+		for(RangerHiveObjectAccessInfo objAccessInfo : objAccessList) {
+			if(mHiveAccessVerifier.isAudited(objAccessInfo)) {
+				logAuditEvent(ugi, objAccessInfo, true);
+			}
+		}
+	}
+	
+	private List<RangerHiveObjectAccessInfo> getObjectAccessInfo(HiveOperationType       hiveOpType,
+														   List<HivePrivilegeObject> inputsHObjs,
+														   List<HivePrivilegeObject> outputHObjs,
+														   RangerHiveAccessContext       context) {
+		List<RangerHiveObjectAccessInfo> ret = new ArrayList<RangerHiveObjectAccessInfo>();
+
+		if(inputsHObjs != null) {
+			for(HivePrivilegeObject hiveObj : inputsHObjs) {
+				RangerHiveObjectAccessInfo hiveAccessObj = getObjectAccessInfo(hiveOpType, hiveObj, context, true);
+				
+				if(   hiveAccessObj != null
+				   && hiveAccessObj.getAccessType() != HiveAccessType.ADMIN // access check is performed at the Argus policy server, as a part of updating the permissions
+				   && !ret.contains(hiveAccessObj)) {
+					ret.add(hiveAccessObj);
+				}
+			}
+		}
+
+		if(outputHObjs != null) {
+			for(HivePrivilegeObject hiveObj : outputHObjs) {
+				RangerHiveObjectAccessInfo hiveAccessObj = getObjectAccessInfo(hiveOpType, hiveObj, context, false);
+				
+				if(   hiveAccessObj != null
+				   && hiveAccessObj.getAccessType() != HiveAccessType.ADMIN // access check is performed at the Argus policy server, as a part of updating the permissions
+				   && !ret.contains(hiveAccessObj)) {
+					ret.add(hiveAccessObj);
+				}
+			}
+		}
+
+		if(ret.size() == 0 && LOG.isDebugEnabled()) {
+			LOG.debug("getObjectAccessInfo(): no objects found for access check! " + toString(hiveOpType, inputsHObjs, outputHObjs, context));
+		}
+		
+		return ret;
+	}
+
+	private RangerHiveObjectAccessInfo getObjectAccessInfo(HiveOperationType hiveOpType, HivePrivilegeObject hiveObj, RangerHiveAccessContext context, boolean isInput) {
+		RangerHiveObjectAccessInfo ret = null;
+
+		HiveObjectType objectType = getObjectType(hiveObj, hiveOpType);
+		HiveAccessType accessType = getAccessType(hiveObj, hiveOpType, isInput);
+		String         operType   = hiveOpType.name();
+
+		switch(objectType) {
+			case DATABASE:
+				ret = new RangerHiveObjectAccessInfo(operType, context, accessType, hiveObj.getDbname());
+			break;
+	
+			case TABLE:
+				ret = new RangerHiveObjectAccessInfo(operType, context, accessType, hiveObj.getDbname(), HiveObjectType.TABLE, hiveObj.getObjectName());
+			break;
+	
+			case VIEW:
+				ret = new RangerHiveObjectAccessInfo(operType, context, accessType, hiveObj.getDbname(), HiveObjectType.VIEW, hiveObj.getObjectName());
+			break;
+	
+			case PARTITION:
+				ret = new RangerHiveObjectAccessInfo(operType, context, accessType, hiveObj.getDbname(), HiveObjectType.PARTITION, hiveObj.getObjectName());
+			break;
+	
+			case INDEX:
+				String indexName = "?"; // TODO:
+				ret = new RangerHiveObjectAccessInfo(operType, context, accessType, hiveObj.getDbname(), hiveObj.getObjectName(), HiveObjectType.INDEX, indexName);
+			break;
+	
+			case COLUMN:
+				ret = new RangerHiveObjectAccessInfo(operType, context, accessType, hiveObj.getDbname(), hiveObj.getObjectName(), hiveObj.getColumns());
+			break;
+
+			case FUNCTION:
+				ret = new RangerHiveObjectAccessInfo(operType, context, accessType, hiveObj.getDbname(), HiveObjectType.FUNCTION, hiveObj.getObjectName());
+			break;
+
+            case URI:
+                ret = new RangerHiveObjectAccessInfo(operType, context, accessType, HiveObjectType.URI, hiveObj.getObjectName());
+            break;
+
+			case NONE:
+			break;
+		}
+
+		return ret;
+	}
+
+	private HiveObjectType getObjectType(HivePrivilegeObject hiveObj, HiveOperationType hiveOpType) {
+		HiveObjectType objType = HiveObjectType.NONE;
+
+		switch(hiveObj.getType()) {
+			case DATABASE:
+				objType = HiveObjectType.DATABASE;
+			break;
+
+			case PARTITION:
+				objType = HiveObjectType.PARTITION;
+			break;
+
+			case TABLE_OR_VIEW:
+				String hiveOpTypeName = hiveOpType.name().toLowerCase();
+				if(hiveOpTypeName.contains("index")) {
+					objType = HiveObjectType.INDEX;
+				} else if(! StringUtil.isEmpty(hiveObj.getColumns())) {
+					objType = HiveObjectType.COLUMN;
+				} else if(hiveOpTypeName.contains("view")) {
+					objType = HiveObjectType.VIEW;
+				} else {
+					objType = HiveObjectType.TABLE;
+				}
+			break;
+
+			case FUNCTION:
+				objType = HiveObjectType.FUNCTION;
+			break;
+
+			case DFS_URI:
+			case LOCAL_URI:
+                objType = HiveObjectType.URI;
+            break;
+
+			case COMMAND_PARAMS:
+			case GLOBAL:
+			break;
+
+			case COLUMN:
+				// Thejas: this value is unused in Hive; the case should not be hit.
+			break;
+		}
+
+		return objType;
+	}
+	
+	private HiveAccessType getAccessType(HivePrivilegeObject hiveObj, HiveOperationType hiveOpType, boolean isInput) {
+		HiveAccessType           accessType       = HiveAccessType.NONE;
+		HivePrivObjectActionType objectActionType = hiveObj.getActionType();
+		
+		switch(objectActionType) {
+			case INSERT:
+			case INSERT_OVERWRITE:
+			case UPDATE:
+			case DELETE:
+				accessType = HiveAccessType.UPDATE;
+			break;
+			case OTHER:
+			switch(hiveOpType) {
+				case CREATEDATABASE:
+					if(hiveObj.getType() == HivePrivilegeObjectType.DATABASE) {
+						accessType = HiveAccessType.CREATE;
+					}
+				break;
+
+				case CREATEFUNCTION:
+					if(hiveObj.getType() == HivePrivilegeObjectType.FUNCTION) {
+						accessType = HiveAccessType.CREATE;
+					}
+				break;
+
+				case CREATETABLE:
+				case CREATEVIEW:
+				case CREATETABLE_AS_SELECT:
+					if(hiveObj.getType() == HivePrivilegeObjectType.TABLE_OR_VIEW) {
+						accessType = isInput ? HiveAccessType.SELECT : HiveAccessType.CREATE;
+					}
+				break;
+
+				case ALTERDATABASE:
+				case ALTERDATABASE_OWNER:
+				case ALTERINDEX_PROPS:
+				case ALTERINDEX_REBUILD:
+				case ALTERPARTITION_BUCKETNUM:
+				case ALTERPARTITION_FILEFORMAT:
+				case ALTERPARTITION_LOCATION:
+				case ALTERPARTITION_MERGEFILES:
+				case ALTERPARTITION_PROTECTMODE:
+				case ALTERPARTITION_SERDEPROPERTIES:
+				case ALTERPARTITION_SERIALIZER:
+				case ALTERTABLE_ADDCOLS:
+				case ALTERTABLE_ADDPARTS:
+				case ALTERTABLE_ARCHIVE:
+				case ALTERTABLE_BUCKETNUM:
+				case ALTERTABLE_CLUSTER_SORT:
+				case ALTERTABLE_COMPACT:
+				case ALTERTABLE_DROPPARTS:
+				case ALTERTABLE_FILEFORMAT:
+				case ALTERTABLE_LOCATION:
+				case ALTERTABLE_MERGEFILES:
+				case ALTERTABLE_PARTCOLTYPE:
+				case ALTERTABLE_PROPERTIES:
+				case ALTERTABLE_PROTECTMODE:
+				case ALTERTABLE_RENAME:
+				case ALTERTABLE_RENAMECOL:
+				case ALTERTABLE_RENAMEPART:
+				case ALTERTABLE_REPLACECOLS:
+				case ALTERTABLE_SERDEPROPERTIES:
+				case ALTERTABLE_SERIALIZER:
+				case ALTERTABLE_SKEWED:
+				case ALTERTABLE_TOUCH:
+				case ALTERTABLE_UNARCHIVE:
+				case ALTERTABLE_UPDATEPARTSTATS:
+				case ALTERTABLE_UPDATETABLESTATS:
+				case ALTERTBLPART_SKEWED_LOCATION:
+				case ALTERVIEW_AS:
+				case ALTERVIEW_PROPERTIES:
+				case ALTERVIEW_RENAME:
+				case DROPVIEW_PROPERTIES:
+					accessType = HiveAccessType.ALTER;
+				break;
+
+				case DROPFUNCTION:
+				case DROPINDEX:
+				case DROPTABLE:
+				case DROPVIEW:
+				case DROPDATABASE:
+					accessType = HiveAccessType.DROP;
+				break;
+
+				case CREATEINDEX:
+					accessType = HiveAccessType.INDEX;
+				break;
+
+				case IMPORT:
+				case EXPORT:
+				case LOAD:
+					accessType = isInput ? HiveAccessType.SELECT : HiveAccessType.UPDATE;
+				break;
+
+				case LOCKDB:
+				case LOCKTABLE:
+				case UNLOCKDB:
+				case UNLOCKTABLE:
+					accessType = HiveAccessType.LOCK;
+				break;
+
+				case QUERY:
+				case SHOW_TABLESTATUS:
+				case SHOW_CREATETABLE:
+				case SHOWCOLUMNS:
+				case SHOWINDEXES:
+				case SHOWPARTITIONS:
+				case SHOW_TBLPROPERTIES:
+				case DESCTABLE:
+				case ANALYZE_TABLE:
+					accessType = HiveAccessType.SELECT;
+				break;
+
+				case SWITCHDATABASE:
+				case DESCDATABASE:
+					accessType = HiveAccessType.USE;
+				break;
+
+				case TRUNCATETABLE:
+					accessType = HiveAccessType.UPDATE;
+				break;
+
+				case GRANT_PRIVILEGE:
+				case REVOKE_PRIVILEGE:
+					accessType = HiveAccessType.ADMIN;
+				break;
+
+				case ADD:
+				case DELETE:
+				case COMPILE:
+				case CREATEMACRO:
+				case CREATEROLE:
+				case DESCFUNCTION:
+				case DFS:
+				case DROPMACRO:
+				case DROPROLE:
+				case EXPLAIN:
+				case GRANT_ROLE:
+				case MSCK:
+				case REVOKE_ROLE:
+				case RESET:
+				case SET:
+				case SHOWCONF:
+				case SHOWDATABASES:
+				case SHOWFUNCTIONS:
+				case SHOWLOCKS:
+				case SHOWTABLES:
+				case SHOW_COMPACTIONS:
+				case SHOW_GRANT:
+				case SHOW_ROLES:
+				case SHOW_ROLE_GRANT:
+				case SHOW_ROLE_PRINCIPALS:
+				case SHOW_TRANSACTIONS:
+				break;
+			}
+			break;
+		}
+		
+		return accessType;
+	}
+
+    private boolean isURIAccessAllowed(UserGroupInformation ugi, HiveAccessType accessType, String uri, HiveConf conf) {
+        boolean ret = false;
+
+        FsAction action = FsAction.NONE;
+
+        switch(accessType) {
+            case ALTER:
+            case CREATE:
+            case UPDATE:
+            case DROP:
+            case INDEX:
+            case LOCK:
+            case ADMIN:
+    		case ALL:
+                action = FsAction.WRITE;
+            break;
+
+            case SELECT:
+            case USE:
+                action = FsAction.READ;
+            break;
+
+            case NONE:
+            break;
+        }
+
+        if(action == FsAction.NONE) {
+            ret = true;
+        } else {
+            try {
+                Path       filePath   = new Path(uri);
+                FileSystem fs         = FileSystem.get(filePath.toUri(), conf);
+                Path       path       = FileUtils.getPathOrParentThatExists(fs, filePath);
+                FileStatus fileStatus = fs.getFileStatus(path);
+                String     userName   = ugi.getShortUserName();
+
+                if (FileUtils.isOwnerOfFileHierarchy(fs, fileStatus, userName)) {
+                    ret = true;
+                } else {
+                    ret = FileUtils.isActionPermittedForFileHierarchy(fs, fileStatus, userName, action);
+                }
+            } catch(Exception excp) {
+                LOG.error("Error getting permissions for " + uri, excp);
+            }
+        }
+
+        return ret;
+    }
+
+	private void handleDfsCommand(HiveOperationType         hiveOpType,
+								  List<HivePrivilegeObject> inputHObjs,
+							      List<HivePrivilegeObject> outputHObjs,
+							      RangerHiveAccessContext       context)
+	      throws HiveAuthzPluginException, HiveAccessControlException {
+
+		String dfsCommandParams = null;
+
+		if(inputHObjs != null) {
+			for(HivePrivilegeObject hiveObj : inputHObjs) {
+				if(hiveObj.getType() == HivePrivilegeObjectType.COMMAND_PARAMS) {
+					dfsCommandParams = StringUtil.toString(hiveObj.getCommandParams());
+
+					if(! StringUtil.isEmpty(dfsCommandParams)) {
+						break;
+					}
+				}
+			}
+		}
+
+		UserGroupInformation ugi = this.getCurrentUserGroupInfo();
+
+		logAuditEventForDfs(ugi, dfsCommandParams, false);
+
+		throw new HiveAccessControlException(String.format("Permission denied: user [%s] does not have privilege for [%s] command",
+											 ugi.getShortUserName(), hiveOpType.name()));
+	}
+	
+	private String getGrantorUsername(HivePrincipal grantorPrincipal) {
+		String grantor = grantorPrincipal != null ? grantorPrincipal.getName() : null;
+
+		if(StringUtil.isEmpty(grantor)) {
+			UserGroupInformation ugi = this.getCurrentUserGroupInfo();
+
+			grantor = ugi != null ? ugi.getShortUserName() : null;
+		}
+
+		return grantor;
+	}
+
+	private GrantRevokeData createGrantRevokeData(RangerHiveObjectAccessInfo objAccessInfo,
+												  List<HivePrincipal>    hivePrincipals,
+												  List<HivePrivilege>    hivePrivileges,
+												  String                 grantor,
+												  boolean                grantOption)
+														  throws HiveAccessControlException {
+		if(objAccessInfo == null ||
+		  ! (   objAccessInfo.getObjectType() == HiveObjectType.DATABASE
+		     || objAccessInfo.getObjectType() == HiveObjectType.TABLE
+		     || objAccessInfo.getObjectType() == HiveObjectType.VIEW
+		     || objAccessInfo.getObjectType() == HiveObjectType.COLUMN
+		   )
+		  ) {
+			throw new HiveAccessControlException("grantPrivileges(): unexpected object type '" + objAccessInfo.getObjectType().name());
+		}
+
+		String database = objAccessInfo.getDatabase();
+		String table    = objAccessInfo.getObjectType() == HiveObjectType.VIEW ? objAccessInfo.getView() : objAccessInfo.getTable();
+		String columns  = StringUtil.toString(objAccessInfo.getColumns());
+
+		GrantRevokeData.PermMap permMap = new GrantRevokeData.PermMap ();
+
+		for(HivePrivilege privilege : hivePrivileges) {
+			String privName = privilege.getName();
+
+			if(StringUtil.equalsIgnoreCase(privName, HiveAccessType.ALL.name())) {
+				permMap.addPerm(HiveAccessType.ALL.name());
+			} else if(StringUtil.equalsIgnoreCase(privName, HiveAccessType.ALTER.name())) {
+				permMap.addPerm(HiveAccessType.ALTER.name());
+			} else if(StringUtil.equalsIgnoreCase(privName, HiveAccessType.CREATE.name())) {
+				permMap.addPerm(HiveAccessType.CREATE.name());
+			} else if(StringUtil.equalsIgnoreCase(privName, HiveAccessType.DROP.name())) {
+				permMap.addPerm(HiveAccessType.DROP.name());
+			} else if(StringUtil.equalsIgnoreCase(privName, HiveAccessType.INDEX.name())) {
+				permMap.addPerm(HiveAccessType.INDEX.name());
+			} else if(StringUtil.equalsIgnoreCase(privName, HiveAccessType.LOCK.name())) {
+				permMap.addPerm(HiveAccessType.LOCK.name());
+			} else if(StringUtil.equalsIgnoreCase(privName, HiveAccessType.SELECT.name())) {
+				permMap.addPerm(HiveAccessType.SELECT.name());
+			} else if(StringUtil.equalsIgnoreCase(privName, HiveAccessType.UPDATE.name())) {
+				permMap.addPerm(HiveAccessType.UPDATE.name());
+			}
+		}
+
+		if(grantOption) {
+			permMap.addPerm(HiveAccessType.ADMIN.name());
+		}
+
+		for(HivePrincipal principal : hivePrincipals) {
+			switch(principal.getType()) {
+				case USER:
+					permMap.addUser(principal.getName());
+				break;
+
+				case GROUP:
+				case ROLE:
+					permMap.addGroup(principal.getName());
+				break;
+
+				default:
+				break;
+			}
+		}
+
+		GrantRevokeData grData = new GrantRevokeData();
+
+		grData.setHiveData(grantor, repositoryName, database, table, columns, permMap);
+
+		return grData;
+	}
+
+    private void logAuditEventForDfs(UserGroupInformation ugi, String dfsCommand, boolean accessGranted) {
+		HiveAuditEvent auditEvent = new HiveAuditEvent();
+
+		try {
+			auditEvent.setAclEnforcer(RangerModuleName);
+			auditEvent.setResourceType("@dfs"); // to be consistent with earlier release
+			auditEvent.setAccessType("DFS");
+			auditEvent.setAction("DFS");
+			auditEvent.setUser(ugi.getShortUserName());
+			auditEvent.setAccessResult((short)(accessGranted ? 1 : 0));
+			auditEvent.setEventTime(StringUtil.getUTCDate());
+			auditEvent.setRepositoryType(EnumRepositoryType.HIVE);
+			auditEvent.setRepositoryName(repositoryName) ;
+			auditEvent.setRequestData(dfsCommand);
+
+			auditEvent.setResourcePath(dfsCommand);
+		
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("logAuditEvent [" + auditEvent + "] - START");
+			}
+
+			AuditProviderFactory.getAuditProvider().log(auditEvent);
+
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("logAuditEvent [" + auditEvent + "] - END");
+			}
+		}
+		catch(Throwable t) {
+			LOG.error("ERROR logEvent [" + auditEvent + "]", t);
+		}
+    }
+
+	private void logAuditEvent(UserGroupInformation ugi, RangerHiveObjectAccessInfo objAccessInfo, boolean accessGranted) {
+		HiveAuditEvent auditEvent = new HiveAuditEvent();
+
+		try {
+			auditEvent.setAclEnforcer(RangerModuleName);
+			auditEvent.setSessionId(objAccessInfo.getContext().getSessionString());
+			auditEvent.setResourceType("@" + StringUtil.toLower(objAccessInfo.getObjectType().name())); // to be consistent with earlier release
+			auditEvent.setAccessType(objAccessInfo.getAccessType().toString());
+			auditEvent.setAction(objAccessInfo.getOperType());
+			auditEvent.setUser(ugi.getShortUserName());
+			auditEvent.setAccessResult((short)(accessGranted ? 1 : 0));
+			auditEvent.setClientIP(objAccessInfo.getContext().getClientIpAddress());
+			auditEvent.setClientType(objAccessInfo.getContext().getClientType());
+			auditEvent.setEventTime(StringUtil.getUTCDate());
+			auditEvent.setRepositoryType(EnumRepositoryType.HIVE);
+			auditEvent.setRepositoryName(repositoryName) ;
+			auditEvent.setRequestData(objAccessInfo.getContext().getCommandString());
+
+			if(! accessGranted && !StringUtil.isEmpty(objAccessInfo.getDeinedObjectName())) {
+				auditEvent.setResourcePath(objAccessInfo.getDeinedObjectName());
+			} else {
+				auditEvent.setResourcePath(objAccessInfo.getObjectName());
+			}
+		
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("logAuditEvent [" + auditEvent + "] - START");
+			}
+
+			AuditProviderFactory.getAuditProvider().log(auditEvent);
+
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("logAuditEvent [" + auditEvent + "] - END");
+			}
+		}
+		catch(Throwable t) {
+			LOG.error("ERROR logEvent [" + auditEvent + "]", t);
+		}
+	}
+	
+	private String toString(HiveOperationType         hiveOpType,
+							List<HivePrivilegeObject> inputHObjs,
+							List<HivePrivilegeObject> outputHObjs,
+							RangerHiveAccessContext       context) {
+		StringBuilder sb = new StringBuilder();
+		
+		sb.append("'checkPrivileges':{");
+		sb.append("'hiveOpType':").append(hiveOpType);
+
+		sb.append(", 'inputHObjs':[");
+		toString(inputHObjs, sb);
+		sb.append("]");
+
+		sb.append(", 'outputHObjs':[");
+		toString(outputHObjs, sb);
+		sb.append("]");
+
+		sb.append(", 'context':{");
+		if(context != null) {
+			sb.append("'clientType':").append(context.getClientType());
+			sb.append(", 'commandString':").append(context.getCommandString());
+			sb.append(", 'ipAddress':").append(context.getClientIpAddress());
+			sb.append(", 'sessionString':").append(context.getSessionString());
+		}
+		sb.append("}");
+
+		sb.append(", 'user':").append(this.getCurrentUserGroupInfo().getUserName());
+		sb.append(", 'groups':[").append(StringUtil.toString(this.getCurrentUserGroupInfo().getGroupNames())).append("]");
+
+		sb.append("}");
+
+		return sb.toString();
+	}
+
+	private StringBuilder toString(List<HivePrivilegeObject> privObjs, StringBuilder sb) {
+		if(privObjs != null && privObjs.size() > 0) {
+			toString(privObjs.get(0), sb);
+			for(int i = 1; i < privObjs.size(); i++) {
+				sb.append(",");
+				toString(privObjs.get(i), sb);
+			}
+		}
+		
+		return sb;
+	}
+
+	private StringBuilder toString(HivePrivilegeObject privObj, StringBuilder sb) {
+		sb.append("'HivePrivilegeObject':{");
+		sb.append("'type':").append(privObj.getType().toString());
+		sb.append(", 'dbName':").append(privObj.getDbname());
+		sb.append(", 'objectType':").append(privObj.getType());
+		sb.append(", 'objectName':").append(privObj.getObjectName());
+		sb.append(", 'columns':[").append(StringUtil.toString(privObj.getColumns())).append("]");
+		sb.append(", 'partKeys':[").append(StringUtil.toString(privObj.getPartKeys())).append("]");
+		sb.append(", 'commandParams':[").append(StringUtil.toString(privObj.getCommandParams())).append("]");
+		sb.append(", 'actionType':").append(privObj.getActionType().toString());
+		sb.append("}");
+
+		return sb;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizerBase.java
----------------------------------------------------------------------
diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizerBase.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizerBase.java
new file mode 100644
index 0000000..776646d
--- /dev/null
+++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizerBase.java
@@ -0,0 +1,230 @@
+/*
+ * 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.authorization.hive.authorizer;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.DisallowTransformHook;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext.CLIENT_TYPE;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeInfo;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveRoleGrant;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.SettableConfigUpdater;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.ranger.authorization.hive.RangerHiveAccessContext;
+import org.apache.ranger.authorization.utils.StringUtil;
+
+public abstract class RangerHiveAuthorizerBase implements HiveAuthorizer {
+
+	private static final Log LOG = LogFactory.getLog(RangerHiveAuthorizerBase.class);
+
+	private HiveMetastoreClientFactory mMetastoreClientFactory;
+	private HiveConf                   mHiveConf;
+	private HiveAuthenticationProvider mHiveAuthenticator;
+	private HiveAuthzSessionContext    mSessionContext;
+	private UserGroupInformation       mUgi;
+	  
+	public RangerHiveAuthorizerBase(HiveMetastoreClientFactory metastoreClientFactory,
+									  HiveConf                   hiveConf,
+									  HiveAuthenticationProvider hiveAuthenticator,
+									  HiveAuthzSessionContext    context) {
+		mMetastoreClientFactory = metastoreClientFactory;
+		mHiveConf               = hiveConf;
+		mHiveAuthenticator      = hiveAuthenticator;
+		mSessionContext         = context;
+
+		String userName = mHiveAuthenticator == null ? null : mHiveAuthenticator.getUserName();
+
+		mUgi = userName == null ? null : UserGroupInformation.createRemoteUser(userName);
+
+		if(mHiveAuthenticator == null) {
+			LOG.warn("RangerHiveAuthorizerBase.RangerHiveAuthorizerBase(): hiveAuthenticator is null");
+		} else if(StringUtil.isEmpty(userName)) {
+			LOG.warn("RangerHiveAuthorizerBase.RangerHiveAuthorizerBase(): hiveAuthenticator.getUserName() returned null/empty");
+		} else if(mUgi == null) {
+			LOG.warn(String.format("RangerHiveAuthorizerBase.RangerHiveAuthorizerBase(): UserGroupInformation.createRemoteUser(%s) returned null", userName));
+		}
+	}
+
+	public HiveMetastoreClientFactory getMetastoreClientFactory() {
+		return mMetastoreClientFactory;
+	}
+
+	public HiveConf getHiveConf() {
+		return mHiveConf;
+	}
+
+	public HiveAuthenticationProvider getHiveAuthenticator() {
+		return mHiveAuthenticator;
+	}
+
+	public HiveAuthzSessionContext getHiveAuthzSessionContext() {
+		return mSessionContext;
+	}
+
+	public UserGroupInformation getCurrentUserGroupInfo() {
+		return mUgi;
+	}
+	
+	public RangerHiveAccessContext getAccessContext(HiveAuthzContext context) {
+		return new RangerHiveAccessContext(context, mSessionContext);
+	}
+
+	@Override
+	public void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPluginException {
+		LOG.debug("RangerHiveAuthorizerBase.applyAuthorizationConfigPolicy()");
+
+		// from SQLStdHiveAccessController.applyAuthorizationConfigPolicy()
+		if (mSessionContext != null && mSessionContext.getClientType() == CLIENT_TYPE.HIVESERVER2) {
+			// Configure PREEXECHOOKS with DisallowTransformHook to disallow transform queries
+			String hooks = hiveConf.getVar(ConfVars.PREEXECHOOKS).trim();
+			if (hooks.isEmpty()) {
+				hooks = DisallowTransformHook.class.getName();
+			} else {
+				hooks = hooks + "," + DisallowTransformHook.class.getName();
+			}
+
+			hiveConf.setVar(ConfVars.PREEXECHOOKS, hooks);
+
+			SettableConfigUpdater.setHiveConfWhiteList(hiveConf);
+		}
+	}
+
+	/**
+	 * Show privileges for given principal on given object
+	 * @param principal
+	 * @param privObj
+	 * @return
+	 * @throws HiveAuthzPluginException
+	 * @throws HiveAccessControlException
+	 */
+	@Override
+	public List<HivePrivilegeInfo> showPrivileges(HivePrincipal principal, HivePrivilegeObject privObj) 
+			throws HiveAuthzPluginException, HiveAccessControlException {
+		LOG.debug("RangerHiveAuthorizerBase.showPrivileges()");
+
+		throwNotImplementedException("showPrivileges");
+
+		return null;
+	}
+
+	@Override
+	public void createRole(String roleName, HivePrincipal adminGrantor)
+			throws HiveAuthzPluginException, HiveAccessControlException {
+		LOG.debug("RangerHiveAuthorizerBase.createRole()");
+
+		throwNotImplementedException("createRole");
+	}
+
+	@Override
+	public void dropRole(String roleName)
+			throws HiveAuthzPluginException, HiveAccessControlException {
+		LOG.debug("RangerHiveAuthorizerBase.dropRole()");
+
+		throwNotImplementedException("dropRole");
+	}
+
+	@Override
+	public List<String> getAllRoles()
+			throws HiveAuthzPluginException, HiveAccessControlException {
+		LOG.debug("RangerHiveAuthorizerBase.getAllRoles()");
+
+		throwNotImplementedException("getAllRoles");
+
+		return null;
+	}
+
+	@Override
+	public List<String> getCurrentRoleNames() throws HiveAuthzPluginException {
+		LOG.debug("RangerHiveAuthorizerBase.getCurrentRoleNames()");
+
+		throwNotImplementedException("getCurrentRoleNames");
+
+		return null;
+	}
+
+	@Override
+	public List<HiveRoleGrant> getPrincipalGrantInfoForRole(String roleName)
+			throws HiveAuthzPluginException, HiveAccessControlException {
+		LOG.debug("RangerHiveAuthorizerBase.getPrincipalGrantInfoForRole()");
+
+		throwNotImplementedException("getPrincipalGrantInfoForRole");
+
+		return null;
+	}
+
+	@Override
+	public List<HiveRoleGrant> getRoleGrantInfoForPrincipal(HivePrincipal principal)
+			throws HiveAuthzPluginException, HiveAccessControlException {
+		LOG.debug("RangerHiveAuthorizerBase.getRoleGrantInfoForPrincipal()");
+
+		throwNotImplementedException("getRoleGrantInfoForPrincipal");
+
+		return null;
+	}
+
+	@Override
+	public VERSION getVersion() {
+		return VERSION.V1;
+	}
+
+	@Override
+	public void grantRole(List<HivePrincipal> hivePrincipals, List<String> roles,
+			boolean grantOption, HivePrincipal grantorPrinc)
+					throws HiveAuthzPluginException, HiveAccessControlException {
+		LOG.debug("RangerHiveAuthorizerBase.grantRole()");
+
+		throwNotImplementedException("grantRole");
+	}
+
+	@Override
+	public void revokeRole(List<HivePrincipal> hivePrincipals, List<String> roles,
+			boolean grantOption, HivePrincipal grantorPrinc)
+					throws HiveAuthzPluginException, HiveAccessControlException {
+		LOG.debug("RangerHiveAuthorizerBase.revokeRole()");
+
+		throwNotImplementedException("revokeRole");
+	}
+
+	@Override
+	public void setCurrentRole(String roleName)
+			throws HiveAccessControlException, HiveAuthzPluginException {
+		LOG.debug("RangerHiveAuthorizerBase.setCurrentRole()");
+
+		throwNotImplementedException("setCurrentRole");
+	}
+
+	private void throwNotImplementedException(String method) throws HiveAuthzPluginException {
+		throw new HiveAuthzPluginException(method + "() not implemented in Argus HiveAuthorizer");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizerFactory.java
----------------------------------------------------------------------
diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizerFactory.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizerFactory.java
new file mode 100644
index 0000000..bd410b7
--- /dev/null
+++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizerFactory.java
@@ -0,0 +1,39 @@
+/*
+ * 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.authorization.hive.authorizer;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
+
+public class RangerHiveAuthorizerFactory implements HiveAuthorizerFactory {
+	@Override
+	public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
+											   HiveConf                   conf,
+											   HiveAuthenticationProvider hiveAuthenticator,
+											   HiveAuthzSessionContext    sessionContext)
+													   throws HiveAuthzPluginException {
+		return new RangerHiveAuthorizer(metastoreClientFactory, conf, hiveAuthenticator, sessionContext);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/hive-agent/src/main/java/org/apache/ranger/authorization/hive/constants/RangerHiveConstants.java
----------------------------------------------------------------------
diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/constants/RangerHiveConstants.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/constants/RangerHiveConstants.java
new file mode 100644
index 0000000..711683e
--- /dev/null
+++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/constants/RangerHiveConstants.java
@@ -0,0 +1,28 @@
+/*
+ * 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.authorization.hive.constants;
+
+public final class RangerHiveConstants {
+	public static final String WILDCARD_OBJECT = "*" ;
+	public static final String HAS_ANY_PERMISSION = "any" ;
+	public static final String SHOW_META_INFO_PERMISSION = "show" ;
+	public static final String PUBLIC_ACCESS_ROLE = "public" ;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/knox-agent/.gitignore
----------------------------------------------------------------------
diff --git a/knox-agent/.gitignore b/knox-agent/.gitignore
index 0f63015..de3a426 100644
--- a/knox-agent/.gitignore
+++ b/knox-agent/.gitignore
@@ -1,2 +1,3 @@
 /target/
 /bin/
+/bin/

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/knox-agent/conf/xasecure-knox-security-changes.cfg
----------------------------------------------------------------------
diff --git a/knox-agent/conf/xasecure-knox-security-changes.cfg b/knox-agent/conf/xasecure-knox-security-changes.cfg
index c48decc..8431962 100644
--- a/knox-agent/conf/xasecure-knox-security-changes.cfg
+++ b/knox-agent/conf/xasecure-knox-security-changes.cfg
@@ -16,7 +16,7 @@
 # Change the original policy parameter to work with policy manager based.
 # 
 #
-knox.authorization.verifier.classname					com.xasecure.pdp.knox.XASecureAuthorizer							mod	create-if-not-exists
+knox.authorization.verifier.classname					org.apache.ranger.pdp.knox.RangerAuthorizer							mod	create-if-not-exists
 xasecure.knox.policymgr.url							%POLICY_MGR_URL%/service/assets/policyList/%REPOSITORY_NAME% 	    mod create-if-not-exists
 xasecure.knox.policymgr.url.saveAsFile				   	/tmp/knox%REPOSITORY_NAME%_json  									mod create-if-not-exists
 xasecure.knox.policymgr.url.reloadIntervalInMillis 	30000 																mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/knox-agent/conf/xasecure-knox-security.xml
----------------------------------------------------------------------
diff --git a/knox-agent/conf/xasecure-knox-security.xml b/knox-agent/conf/xasecure-knox-security.xml
index 0307b42..61aa245 100644
--- a/knox-agent/conf/xasecure-knox-security.xml
+++ b/knox-agent/conf/xasecure-knox-security.xml
@@ -22,7 +22,7 @@
 	<!--  The following property is used to select appropriate XASecure Authorizer Module (filebased, policymanager based) -->
 	<property>
 		<name>knox.authorization.verifier.classname</name>
-		<value>com.xasecure.pdp.knox.XASecureAuthorizer</value>
+		<value>org.apache.ranger.pdp.knox.RangerAuthorizer</value>
 		<description>
 			Class Name of the authorization Module 
 		</description>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/knox-agent/scripts/install.sh
----------------------------------------------------------------------
diff --git a/knox-agent/scripts/install.sh b/knox-agent/scripts/install.sh
index 6150fba..903ef44 100644
--- a/knox-agent/scripts/install.sh
+++ b/knox-agent/scripts/install.sh
@@ -21,7 +21,7 @@ function create_jceks()
        alias=$1
        pass=$2
        jceksFile=$3
-       java -cp "${install_dir}/cred/lib/*" com.hortonworks.credentialapi.buildks create ${alias} -value ${pass} -provider jceks://file${jceksFile}
+       java -cp "${install_dir}/cred/lib/*" org.apache.ranger.credentialapi.buildks create ${alias} -value ${pass} -provider jceks://file${jceksFile}
 }
 
 #Update Properties to File
@@ -211,7 +211,7 @@ do
 		if [ $? -eq 0 ]
 		then
 			cp="${install_dir}/cred/lib/*:${install_dir}/installer/lib/*:/usr/lib/hadoop/*:/usr/lib/hadoop/lib/*"
-			java -cp "${cp}" com.xasecure.utils.install.XmlConfigChanger -i ${archivefn} -o ${newfn} -c ${f} ${PROP_ARGS}
+			java -cp "${cp}" org.apache.ranger.utils.install.XmlConfigChanger -i ${archivefn} -o ${newfn} -c ${f} ${PROP_ARGS}
 			if [ $? -eq 0 ]
 			then
 				diff -w ${newfn} ${fullpathorgfn} > /dev/null 2>&1

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/knox-agent/src/main/java/com/xasecure/authorization/knox/KnoxAccessVerifier.java
----------------------------------------------------------------------
diff --git a/knox-agent/src/main/java/com/xasecure/authorization/knox/KnoxAccessVerifier.java b/knox-agent/src/main/java/com/xasecure/authorization/knox/KnoxAccessVerifier.java
deleted file mode 100644
index 91f2b90..0000000
--- a/knox-agent/src/main/java/com/xasecure/authorization/knox/KnoxAccessVerifier.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * 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 com.xasecure.authorization.knox;
-
-import java.util.Set;
-
-public interface KnoxAccessVerifier {
-
-	
-	public boolean isAccessAllowed(String topologyName, String serviceName, String accessTypes, String userName, Set<String> groups, String requestIp) ;
-
-	public boolean isAuditEnabled(String topologyName, String serviceName) ;
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/knox-agent/src/main/java/com/xasecure/authorization/knox/KnoxAccessVerifierFactory.java
----------------------------------------------------------------------
diff --git a/knox-agent/src/main/java/com/xasecure/authorization/knox/KnoxAccessVerifierFactory.java b/knox-agent/src/main/java/com/xasecure/authorization/knox/KnoxAccessVerifierFactory.java
deleted file mode 100644
index 112d94b..0000000
--- a/knox-agent/src/main/java/com/xasecure/authorization/knox/KnoxAccessVerifierFactory.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * 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 com.xasecure.authorization.knox;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import com.xasecure.authorization.hadoop.config.XaSecureConfiguration;
-import com.xasecure.authorization.hadoop.constants.XaSecureHadoopConstants;
-
-public class KnoxAccessVerifierFactory {
-
-	private static final Log LOG = LogFactory.getLog(KnoxAccessVerifierFactory.class) ;
-
-	private static KnoxAccessVerifier knoxAccessVerififer = null ;
-	
-	public static KnoxAccessVerifier getInstance() {
-		if (knoxAccessVerififer == null) {
-			synchronized(KnoxAccessVerifierFactory.class) {
-				KnoxAccessVerifier temp = knoxAccessVerififer ;
-				if (temp == null) {
-					String knoxAccessVerifierClassName = XaSecureConfiguration.getInstance().get(XaSecureHadoopConstants.KNOX_ACCESS_VERIFIER_CLASS_NAME_PROP, XaSecureHadoopConstants.KNOX_ACCESS_VERIFIER_CLASS_NAME_DEFAULT_VALUE ) ;
-
-					if (knoxAccessVerifierClassName != null) {
-						LOG.info("Knox Access Verification class [" + knoxAccessVerifierClassName + "] - Being build");
-						try {
-							knoxAccessVerififer = (KnoxAccessVerifier) (Class.forName(knoxAccessVerifierClassName).newInstance()) ;
-							LOG.info("Created a new instance of class: [" + knoxAccessVerifierClassName + "] for Knox Access verification.");
-						} catch (InstantiationException e) {
-							LOG.error("Unable to create KnoxAccess Verifier: [" +  knoxAccessVerifierClassName + "]", e);
-						} catch (IllegalAccessException e) {
-							LOG.error("Unable to create KnoxAccess Verifier: [" +  knoxAccessVerifierClassName + "]", e);
-						} catch (ClassNotFoundException e) {
-							LOG.error("Unable to create KnoxAccess Verifier: [" +  knoxAccessVerifierClassName + "]", e);
-						} catch (Throwable t) {
-							LOG.error("Unable to create KnoxAccess Verifier: [" +  knoxAccessVerifierClassName + "]", t);
-						}
-						finally {
-							LOG.info("Created a new instance of class: [" + knoxAccessVerifierClassName + "] for Knox Access verification. (" + knoxAccessVerififer + ")");
-						}
-					}
-				}
-				else {
-					LOG.error("Unable to obtain knoxAccessVerifier [" +  XaSecureHadoopConstants.KNOX_ACCESS_VERIFIER_CLASS_NAME_PROP + "]");
-				}
-			}
-		}
-		return knoxAccessVerififer ;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/knox-agent/src/main/java/org/apache/ranger/authorization/knox/KnoxAccessVerifier.java
----------------------------------------------------------------------
diff --git a/knox-agent/src/main/java/org/apache/ranger/authorization/knox/KnoxAccessVerifier.java b/knox-agent/src/main/java/org/apache/ranger/authorization/knox/KnoxAccessVerifier.java
new file mode 100644
index 0000000..89265cb
--- /dev/null
+++ b/knox-agent/src/main/java/org/apache/ranger/authorization/knox/KnoxAccessVerifier.java
@@ -0,0 +1,30 @@
+/**
+ * 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.authorization.knox;
+
+import java.util.Set;
+
+public interface KnoxAccessVerifier {
+
+	
+	public boolean isAccessAllowed(String topologyName, String serviceName, String accessTypes, String userName, Set<String> groups, String requestIp) ;
+
+	public boolean isAuditEnabled(String topologyName, String serviceName) ;
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/knox-agent/src/main/java/org/apache/ranger/authorization/knox/KnoxAccessVerifierFactory.java
----------------------------------------------------------------------
diff --git a/knox-agent/src/main/java/org/apache/ranger/authorization/knox/KnoxAccessVerifierFactory.java b/knox-agent/src/main/java/org/apache/ranger/authorization/knox/KnoxAccessVerifierFactory.java
new file mode 100644
index 0000000..25e1e28
--- /dev/null
+++ b/knox-agent/src/main/java/org/apache/ranger/authorization/knox/KnoxAccessVerifierFactory.java
@@ -0,0 +1,65 @@
+/**
+ * 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.authorization.knox;
+
+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.hadoop.constants.RangerHadoopConstants;
+
+public class KnoxAccessVerifierFactory {
+
+	private static final Log LOG = LogFactory.getLog(KnoxAccessVerifierFactory.class) ;
+
+	private static KnoxAccessVerifier knoxAccessVerififer = null ;
+	
+	public static KnoxAccessVerifier getInstance() {
+		if (knoxAccessVerififer == null) {
+			synchronized(KnoxAccessVerifierFactory.class) {
+				KnoxAccessVerifier temp = knoxAccessVerififer ;
+				if (temp == null) {
+					String knoxAccessVerifierClassName = RangerConfiguration.getInstance().get(RangerHadoopConstants.KNOX_ACCESS_VERIFIER_CLASS_NAME_PROP, RangerHadoopConstants.KNOX_ACCESS_VERIFIER_CLASS_NAME_DEFAULT_VALUE ) ;
+
+					if (knoxAccessVerifierClassName != null) {
+						LOG.info("Knox Access Verification class [" + knoxAccessVerifierClassName + "] - Being build");
+						try {
+							knoxAccessVerififer = (KnoxAccessVerifier) (Class.forName(knoxAccessVerifierClassName).newInstance()) ;
+							LOG.info("Created a new instance of class: [" + knoxAccessVerifierClassName + "] for Knox Access verification.");
+						} catch (InstantiationException e) {
+							LOG.error("Unable to create KnoxAccess Verifier: [" +  knoxAccessVerifierClassName + "]", e);
+						} catch (IllegalAccessException e) {
+							LOG.error("Unable to create KnoxAccess Verifier: [" +  knoxAccessVerifierClassName + "]", e);
+						} catch (ClassNotFoundException e) {
+							LOG.error("Unable to create KnoxAccess Verifier: [" +  knoxAccessVerifierClassName + "]", e);
+						} catch (Throwable t) {
+							LOG.error("Unable to create KnoxAccess Verifier: [" +  knoxAccessVerifierClassName + "]", t);
+						}
+						finally {
+							LOG.info("Created a new instance of class: [" + knoxAccessVerifierClassName + "] for Knox Access verification. (" + knoxAccessVerififer + ")");
+						}
+					}
+				}
+				else {
+					LOG.error("Unable to obtain knoxAccessVerifier [" +  RangerHadoopConstants.KNOX_ACCESS_VERIFIER_CLASS_NAME_PROP + "]");
+				}
+			}
+		}
+		return knoxAccessVerififer ;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/lookup-client/src/main/java/com/xasecure/hadoop/client/HadoopFS.java
----------------------------------------------------------------------
diff --git a/lookup-client/src/main/java/com/xasecure/hadoop/client/HadoopFS.java b/lookup-client/src/main/java/com/xasecure/hadoop/client/HadoopFS.java
deleted file mode 100644
index 57de18a..0000000
--- a/lookup-client/src/main/java/com/xasecure/hadoop/client/HadoopFS.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * 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 com.xasecure.hadoop.client;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.UnknownHostException;
-import java.security.PrivilegedAction;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import javax.security.auth.Subject;
-
-import org.apache.commons.io.FilenameUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-
-import com.xasecure.hadoop.client.config.BaseClient;
-import com.xasecure.hadoop.client.exceptions.HadoopException;
-
-public class HadoopFS extends BaseClient {
-
-	private static final Log LOG = LogFactory.getLog(HadoopFS.class) ;
-
-	public HadoopFS(String dataSource) {
-		super(dataSource) ;
-	}
-	
-	public HadoopFS(String dataSource, HashMap<String,String> connectionProperties) {
-		super(dataSource,connectionProperties) ;
-	}
-	
-	private List<String> listFilesInternal(String baseDir, String fileMatching) {
-		List<String> fileList = new ArrayList<String>() ;
-		ClassLoader prevCl = Thread.currentThread().getContextClassLoader() ;
-		String errMsg = " You can still save the repository and start creating "
-				+ "policies, but you would not be able to use autocomplete for "
-				+ "resource names. Check xa_portal.log for more info.";
-		try {
-			Thread.currentThread().setContextClassLoader(getConfigHolder().getClassLoader());
-			String dirPrefix = (baseDir.endsWith("/") ? baseDir : (baseDir + "/")) ;
-			String filterRegEx = null;
-			if (fileMatching != null && fileMatching.trim().length() > 0) {
-				filterRegEx = fileMatching.trim() ;
-			}
-			
-			Configuration conf = new Configuration() ;
-			UserGroupInformation.setConfiguration(conf);
-			
-			FileSystem fs = null ;
-			try {
-				fs = FileSystem.get(conf) ;
-				
-				FileStatus[] fileStats = fs.listStatus(new Path(baseDir)) ;
-				if (fileStats != null) {
-					for(FileStatus stat : fileStats) {
-						Path path = stat.getPath() ;
-						String pathComponent = path.getName() ;
-						if (filterRegEx == null) {
-							fileList.add(dirPrefix + pathComponent) ;
-						}
-						else if (FilenameUtils.wildcardMatch(pathComponent, fileMatching)) {
-							fileList.add(dirPrefix + pathComponent) ;
-						}
-					}
-				}
-			} catch (UnknownHostException uhe) {
-				String msgDesc = "listFilesInternal: Unable to connect using given config parameters"
-						+ " of Hadoop environment [" + getDataSource() + "].";
-				HadoopException hdpException = new HadoopException(msgDesc, uhe);
-				hdpException.generateResponseDataMap(false, getMessage(uhe),
-						msgDesc + errMsg, null, null);
-				throw hdpException;
-			} catch (FileNotFoundException fne) {
-				String msgDesc = "listFilesInternal: Unable to locate files using given config parameters "
-						+ "of Hadoop environment [" + getDataSource() + "].";
-				HadoopException hdpException = new HadoopException(msgDesc, fne);
-				hdpException.generateResponseDataMap(false, getMessage(fne),
-						msgDesc + errMsg, null, null);
-				throw hdpException;
-			}
-			finally {
-			}
-		} catch (IOException ioe) {
-			String msgDesc = "listFilesInternal: Unable to get listing of files for directory "
-					+ baseDir
-					+ "] from Hadoop environment ["
-					+ getDataSource()
-					+ "].";
-			HadoopException hdpException = new HadoopException(msgDesc, ioe);
-			hdpException.generateResponseDataMap(false, getMessage(ioe),
-					msgDesc + errMsg, null, null);
-			throw hdpException;
-
-		} catch (IllegalArgumentException iae) {
-			String msgDesc = "Unable to get listing of files for directory ["
-					+ baseDir + "] from Hadoop environment [" + getDataSource()
-					+ "].";
-			HadoopException hdpException = new HadoopException(msgDesc, iae);
-			hdpException.generateResponseDataMap(false, getMessage(iae),
-					msgDesc + errMsg, null, null);
-			throw hdpException;
-		}
-		finally {
-			Thread.currentThread().setContextClassLoader(prevCl);
-		}
-		return fileList ;
-	}
-
-	
-	public List<String> listFiles(final String baseDir, final String fileMatching) {
-
-		PrivilegedAction<List<String>> action = new PrivilegedAction<List<String>>() {
-			@Override
-			public List<String> run() {
-				return listFilesInternal(baseDir, fileMatching) ;
-			}
-			
-		};
-		return Subject.doAs(getLoginSubject(),action) ;
-	}
-	
-	public static final void main(String[] args) {
-		
-		if (args.length < 2) {
-			System.err.println("USAGE: java " + HadoopFS.class.getName() + " repositoryName  basedirectory  [filenameToMatch]") ;
-			System.exit(1) ;
-		}
-		
-		String repositoryName = args[0] ;
-		String baseDir = args[1] ;
-		String fileNameToMatch = (args.length == 2 ? null : args[2]) ;
-		
-		HadoopFS fs = new HadoopFS(repositoryName) ;
-		List<String> fsList = fs.listFiles(baseDir, fileNameToMatch) ;
-		if (fsList != null && fsList.size() > 0) {
-			for(String s : fsList) {
-				System.out.println(s) ;
-			}
-		}
-		else {
-			System.err.println("Unable to get file listing for [" + baseDir + (baseDir.endsWith("/") ? "" : "/") + fileNameToMatch + "]  in repository [" + repositoryName + "]") ;
-		}
-	}
-
-	public static HashMap<String, Object> testConnection(String dataSource,
-			HashMap<String, String> connectionProperties) {
-
-		HashMap<String, Object> responseData = new HashMap<String, Object>();
-		boolean connectivityStatus = false;
-		HadoopFS connectionObj = new HadoopFS(dataSource, connectionProperties);
-		if (connectionObj != null) {
-			List<String> testResult = connectionObj.listFiles("/", null);
-			if (testResult != null && testResult.size() != 0) {
-				connectivityStatus = true;
-			}
-		}
-		if (connectivityStatus) {
-			String successMsg = "TestConnection Successful";
-			generateResponseDataMap(connectivityStatus, successMsg, successMsg,
-					null, null, responseData);
-		} else {
-			String failureMsg = "Unable to retrieve any files using given parameters, "
-					+ "You can still save the repository and start creating policies, "
-					+ "but you would not be able to use autocomplete for resource names. "
-					+ "Check xa_portal.log for more info.";
-			generateResponseDataMap(connectivityStatus, failureMsg, failureMsg,
-					null, null, responseData);
-		}
-		return responseData;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/lookup-client/src/main/java/com/xasecure/hadoop/client/HadoopFSTester.java
----------------------------------------------------------------------
diff --git a/lookup-client/src/main/java/com/xasecure/hadoop/client/HadoopFSTester.java b/lookup-client/src/main/java/com/xasecure/hadoop/client/HadoopFSTester.java
deleted file mode 100644
index 13ba146..0000000
--- a/lookup-client/src/main/java/com/xasecure/hadoop/client/HadoopFSTester.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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 com.xasecure.hadoop.client;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Properties;
-
-public class HadoopFSTester {
-
-	public static void main(String[] args) throws Throwable {
-		if (args.length < 3) {
-			System.err.println("USAGE: java " + HadoopFS.class.getName() + " repositoryName propertyFile basedirectory  [filenameToMatch]") ;
-			System.exit(1) ;
-		}
-		
-		String repositoryName = args[0] ;
-		String propFile = args[1] ;
-		String baseDir = args[2] ;
-		String fileNameToMatch = (args.length == 3 ? null : args[3]) ;
-
-		Properties conf = new Properties() ;
-		conf.load(HadoopFSTester.class.getClassLoader().getResourceAsStream(propFile));
-		
-		HashMap<String,String> prop = new HashMap<String,String>() ;
-		for(Object key : conf.keySet()) {
-			Object val = conf.get(key) ;
-			prop.put((String)key, (String)val) ;
-		}
-		
-		HadoopFS fs = new HadoopFS(repositoryName, prop) ;
-		List<String> fsList = fs.listFiles(baseDir, fileNameToMatch) ;
-		if (fsList != null && fsList.size() > 0) {
-			for(String s : fsList) {
-				System.out.println(s) ;
-			}
-		}
-		else {
-			System.err.println("Unable to get file listing for [" + baseDir + (baseDir.endsWith("/") ? "" : "/") + fileNameToMatch + "]  in repository [" + repositoryName + "]") ;
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/lookup-client/src/main/java/com/xasecure/hadoop/client/config/BaseClient.java
----------------------------------------------------------------------
diff --git a/lookup-client/src/main/java/com/xasecure/hadoop/client/config/BaseClient.java b/lookup-client/src/main/java/com/xasecure/hadoop/client/config/BaseClient.java
deleted file mode 100644
index 17acdf7..0000000
--- a/lookup-client/src/main/java/com/xasecure/hadoop/client/config/BaseClient.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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 com.xasecure.hadoop.client.config;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import javax.security.auth.Subject;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.security.SecureClientLogin;
-
-import com.xasecure.hadoop.client.HadoopFS;
-import com.xasecure.hadoop.client.exceptions.HadoopException;
-
-public abstract class BaseClient {
-	private static final Log LOG = LogFactory.getLog(HadoopFS.class) ;
-	
-	private String dataSource ;
-	private Subject loginSubject ;
-	private HadoopConfigHolder configHolder;
-	
-	protected HashMap<String,String> connectionProperties ;
-	
-	public BaseClient(String dataSource) {
-		this.dataSource = dataSource ;
-		init() ;
-		login() ;
-	}
-	
-	public BaseClient(String dataSource, HashMap<String,String> connectionProperties) {
-		this.dataSource = dataSource ;
-		this.connectionProperties = connectionProperties ;
-		init() ;
-		login() ;
-	}
-	
-	
-	private void init() {
-		if (connectionProperties == null) {
-			configHolder = HadoopConfigHolder.getInstance(dataSource) ;
-		}
-		else {
-			configHolder = HadoopConfigHolder.getInstance(dataSource,connectionProperties) ;
-		}
-	}
-	
-	
-	protected void login() {
-		ClassLoader prevCl = Thread.currentThread().getContextClassLoader() ;
-		String errMsg = " You can still save the repository and start creating "
-				+ "policies, but you would not be able to use autocomplete for "
-				+ "resource names. Check xa_portal.log for more info.";
-		try {
-			Thread.currentThread().setContextClassLoader(configHolder.getClassLoader());
-			String userName = configHolder.getUserName() ;
-			if (userName == null) {
-				String msgDesc = "Unable to find login username for hadoop environment, ["
-						+ dataSource + "]";
-				HadoopException hdpException = new HadoopException(msgDesc);
-				hdpException.generateResponseDataMap(false, msgDesc, msgDesc + errMsg,
-						null, null);
-
-				throw hdpException;
-			}
-			String keyTabFile = configHolder.getKeyTabFile() ;
-			if (keyTabFile != null) {
-				if ( configHolder.isKerberosAuthentication() ) {
-					LOG.info("Init Login: security enabled, using username/keytab");
-					loginSubject = SecureClientLogin.loginUserFromKeytab(userName, keyTabFile) ;
-				}
-				else {
-					LOG.info("Init Login: using username");
-					loginSubject = SecureClientLogin.login(userName) ;
-				}
-			}
-			else {
-				String password = configHolder.getPassword() ;
-				if ( configHolder.isKerberosAuthentication() ) {
-					LOG.info("Init Login: using username/password");
-					loginSubject = SecureClientLogin.loginUserWithPassword(userName, password) ;
-				}
-				else {
-					LOG.info("Init Login: security not enabled, using username");
-					loginSubject = SecureClientLogin.login(userName) ;
-				}
-			}
-		} catch (IOException ioe) {
-			String msgDesc = "Unable to login to Hadoop environment ["
-					+ dataSource + "]";
-
-			HadoopException hdpException = new HadoopException(msgDesc, ioe);
-			hdpException.generateResponseDataMap(false, getMessage(ioe),
-					msgDesc + errMsg, null, null);
-			throw hdpException;
-		} catch (SecurityException se) {
-			String msgDesc = "Unable to login to Hadoop environment ["
-					+ dataSource + "]";
-			HadoopException hdpException = new HadoopException(msgDesc, se);
-			hdpException.generateResponseDataMap(false, getMessage(se),
-					msgDesc + errMsg, null, null);
-			throw hdpException;
-		} finally {
-			Thread.currentThread().setContextClassLoader(prevCl);
-		}
-	}
-	
-	public String getDataSource() {
-		return dataSource ;
-	}
-
-	protected Subject getLoginSubject() {
-		return loginSubject;
-	}
-
-	protected HadoopConfigHolder getConfigHolder() {
-		return configHolder;
-	}
-	
-	public static void generateResponseDataMap(boolean connectivityStatus,
-			String message, String description, Long objectId,
-			String fieldName, HashMap<String, Object> responseData) {
-		responseData.put("connectivityStatus", connectivityStatus);
-		responseData.put("message", message);
-		responseData.put("description", description);
-		responseData.put("objectId", objectId);
-		responseData.put("fieldName", fieldName);
-	}
-
-	public static String getMessage(Throwable excp) {
-		List<String> errList = new ArrayList<String>();
-		while (excp != null) {
-			if (!errList.contains(excp.getMessage() + ". \n")) {
-				if (excp.getMessage() != null && !(excp.getMessage().equalsIgnoreCase(""))) {
-					errList.add(excp.getMessage() + ". \n");
-				}
-			}
-			excp = excp.getCause();
-		}
-		return StringUtils.join(errList, "");
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/413fcb68/lookup-client/src/main/java/com/xasecure/hadoop/client/config/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/lookup-client/src/main/java/com/xasecure/hadoop/client/config/HadoopClassLoader.java b/lookup-client/src/main/java/com/xasecure/hadoop/client/config/HadoopClassLoader.java
deleted file mode 100644
index 214ff4d..0000000
--- a/lookup-client/src/main/java/com/xasecure/hadoop/client/config/HadoopClassLoader.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * 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 com.xasecure.hadoop.client.config;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.net.URL;
-import java.util.Properties;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import com.xasecure.hadoop.client.exceptions.HadoopException;
-
-public class HadoopClassLoader extends ClassLoader {
-	
-	private static final Log LOG = LogFactory.getLog(HadoopClassLoader.class) ;
-	
-	private HadoopConfigHolder confHolder ;
-	
-	public HadoopClassLoader(HadoopConfigHolder confHolder) {
-		super(Thread.currentThread().getContextClassLoader()) ;
-		this.confHolder = confHolder;
-	}
-	
-	
-	@Override
-	protected URL findResource(String resourceName) {
-		LOG.debug("findResource(" + resourceName + ") is called.") ;
-		URL ret = null;
-	
-		if (confHolder.hasResourceExists(resourceName)) {
-			ret = buildResourceFile(resourceName) ;
-		}
-		else {
-			ret = super.findResource(resourceName);
-		}
-		LOG.debug("findResource(" + resourceName + ") is returning [" + ret + "]") ;
-		return ret ;
-	}
-	
-	
-	@SuppressWarnings("deprecation")
-	private URL buildResourceFile(String aResourceName) {
-		URL ret = null ;
-		String prefix = aResourceName ;
-		String suffix = ".txt" ;
-
-		Properties prop = confHolder.getProperties(aResourceName) ;
-		LOG.debug("Building XML for: " + prop.toString());
-		if (prop != null && prop.size() > 0) {
-			if (aResourceName.contains(".")) {
-				int lastDotFound = aResourceName.indexOf(".") ;
-				prefix = aResourceName.substring(0,lastDotFound) + "-" ;
-				suffix = aResourceName.substring(lastDotFound) ;
-			}
-			
-			try {
-				File tempFile = File.createTempFile(prefix, suffix) ;
-				tempFile.deleteOnExit();
-				PrintWriter out = new PrintWriter(new FileWriter(tempFile)) ;
-				out.println("<?xml version=\"1.0\"?>") ;
-				out.println("<?xml-stylesheet type=\"text/xsl\" href=\"configuration.xsl\"?>") ;
-				out.println("<configuration xmlns:xi=\"http://www.w3.org/2001/XInclude\">") ;
-				for(Object keyobj : prop.keySet()) {
-					String key = (String)keyobj;
-					String val = prop.getProperty(key) ;
-					if (HadoopConfigHolder.HADOOP_RPC_PROTECTION.equals(key) && (val == null || val.trim().isEmpty()))  {
-						continue;
-					}
-					out.println("<property><name>" + key.trim() + "</name><value>" + val + "</value></property>") ;
-				}
-				out.println("</configuration>") ;
-				out.close() ;
-				ret = tempFile.toURL() ;
-			} catch (IOException e) {
-				throw new HadoopException("Unable to load create hadoop configuration file [" + aResourceName + "]", e) ;
-			}
-			
-		}
-		
-		return ret ;
-
-	}
-	
-
-}