You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by ma...@apache.org on 2016/09/26 21:48:57 UTC

incubator-ranger git commit: RANGER-1177: updated tagsync to support Atlas notification for hdfs_path entity-type

Repository: incubator-ranger
Updated Branches:
  refs/heads/master 868c62b72 -> 74959da16


RANGER-1177: updated tagsync to support Atlas notification for hdfs_path entity-type


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

Branch: refs/heads/master
Commit: 74959da1617f65bdf74048ecd714dc3aa1d384e9
Parents: 868c62b
Author: Madhan Neethiraj <ma...@apache.org>
Authored: Fri Sep 23 17:51:03 2016 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Mon Sep 26 14:48:39 2016 -0700

----------------------------------------------------------------------
 tagsync/pom.xml                                 |   4 +
 .../source/atlas/AtlasHdfsResourceMapper.java   |  98 ++++++++++
 .../source/atlas/AtlasHiveResourceMapper.java   | 188 +++++--------------
 .../source/atlas/AtlasResourceMapper.java       |  92 +++++++--
 .../source/atlas/AtlasResourceMapperUtil.java   |  15 +-
 .../tagsync/process/TestHdfsResourceMapper.java | 167 ++++++++++++++++
 .../tagsync/process/TestHiveResourceMapper.java | 173 +++++++++++++++++
 .../tagsync/process/TestTagSynchronizer.java    |  31 ---
 8 files changed, 572 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/74959da1/tagsync/pom.xml
----------------------------------------------------------------------
diff --git a/tagsync/pom.xml b/tagsync/pom.xml
index 0e3fb39..2e63502 100644
--- a/tagsync/pom.xml
+++ b/tagsync/pom.xml
@@ -36,6 +36,10 @@
             <version>${junit.version}</version>
         </dependency>
         <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+        </dependency>
+        <dependency>
             <groupId>log4j</groupId>
             <artifactId>log4j</artifactId>
             <version>${log4j.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/74959da1/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHdfsResourceMapper.java
----------------------------------------------------------------------
diff --git a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHdfsResourceMapper.java b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHdfsResourceMapper.java
new file mode 100644
index 0000000..06bff90
--- /dev/null
+++ b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHdfsResourceMapper.java
@@ -0,0 +1,98 @@
+/*
+ * 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.tagsync.source.atlas;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.atlas.typesystem.IReferenceableInstance;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.ranger.plugin.model.RangerPolicy;
+import org.apache.ranger.plugin.model.RangerPolicy.RangerPolicyResource;
+import org.apache.ranger.plugin.model.RangerServiceResource;
+
+public class AtlasHdfsResourceMapper extends AtlasResourceMapper {
+	public static final String ENTITY_TYPE_HDFS_PATH = "hdfs_path";
+	public static final String RANGER_TYPE_HDFS_PATH = "path";
+
+	public static final String ENTITY_ATTRIBUTE_PATH           = "path";
+	public static final String ENTITY_ATTRIBUTE_CLUSTER_NAME   = "clusterName";
+	public static final String ENTITY_ATTRIBUTE_QUALIFIED_NAME = "qualifiedName";
+
+	public static final String[] SUPPORTED_ENTITY_TYPES = { ENTITY_TYPE_HDFS_PATH };
+
+
+	public AtlasHdfsResourceMapper() {
+		super("hdfs", SUPPORTED_ENTITY_TYPES);
+	}
+
+	@Override
+	public String getRangerServiceName(String clusterName) {
+		String ret = getCustomRangerServiceName(clusterName);
+
+		if (StringUtils.isBlank(ret)) {
+			ret = clusterName + TAGSYNC_DEFAULT_CLUSTERNAME_AND_COMPONENTNAME_SEPARATOR + "hadoop";
+		}
+
+		return ret;
+	}
+
+	@Override
+	public RangerServiceResource buildResource(final IReferenceableInstance entity) throws Exception {
+		String path          = getEntityAttribute(entity, ENTITY_ATTRIBUTE_PATH, String.class);
+		String clusterName   = getEntityAttribute(entity, ENTITY_ATTRIBUTE_CLUSTER_NAME, String.class);
+		String qualifiedName = getEntityAttribute(entity, ENTITY_ATTRIBUTE_QUALIFIED_NAME, String.class);
+
+		if(StringUtils.isEmpty(path)) {
+			path = getResourceNameFromQualifiedName(qualifiedName);
+
+			if(StringUtils.isEmpty(path)) {
+				throwExceptionWithMessage("path not found in attribute '" + ENTITY_ATTRIBUTE_PATH + "' or '" + ENTITY_ATTRIBUTE_QUALIFIED_NAME +  "'");
+			}
+		}
+
+		if(StringUtils.isEmpty(clusterName)) {
+			clusterName = getClusterNameFromQualifiedName(qualifiedName);
+
+			if(StringUtils.isEmpty(clusterName)) {
+				clusterName = defaultClusterName;
+			}
+
+			if(StringUtils.isEmpty(clusterName)) {
+				throwExceptionWithMessage("attributes " + ENTITY_ATTRIBUTE_CLUSTER_NAME + ", " + ENTITY_ATTRIBUTE_QUALIFIED_NAME +  "' not found in entity");
+			}
+		}
+
+		String  entityGuid  = entity.getId() != null ? entity.getId()._getId() : null;
+		String  serviceName = getRangerServiceName(clusterName);
+		Boolean isExcludes  = Boolean.FALSE;
+		Boolean isRecursive = Boolean.TRUE;
+
+		Path pathObj = new Path(path);
+
+		Map<String, RangerPolicyResource> elements = new HashMap<String, RangerPolicy.RangerPolicyResource>();
+		elements.put(RANGER_TYPE_HDFS_PATH, new RangerPolicyResource(pathObj.toUri().getPath(), isExcludes, isRecursive));
+
+		RangerServiceResource ret = new RangerServiceResource(entityGuid, serviceName, elements);
+
+		return ret;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/74959da1/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHiveResourceMapper.java
----------------------------------------------------------------------
diff --git a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHiveResourceMapper.java b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHiveResourceMapper.java
index ca7b063..a359622 100644
--- a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHiveResourceMapper.java
+++ b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasHiveResourceMapper.java
@@ -19,177 +19,83 @@
 
 package org.apache.ranger.tagsync.source.atlas;
 
+import java.util.Map;
+import java.util.HashMap;
+
 import org.apache.atlas.typesystem.IReferenceableInstance;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.ranger.plugin.model.RangerPolicy;
+import org.apache.ranger.plugin.model.RangerPolicy.RangerPolicyResource;
 import org.apache.ranger.plugin.model.RangerServiceResource;
 
-import java.util.Map;
-import java.util.HashMap;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Arrays;
-
 public class AtlasHiveResourceMapper extends AtlasResourceMapper {
-	private static final Log LOG = LogFactory.getLog(AtlasHiveResourceMapper.class);
-
-	public static final String COMPONENT_NAME = "hive";
-
-	public static final String ENTITY_TYPE_HIVE_DB = "hive_db";
-	public static final String ENTITY_TYPE_HIVE_TABLE = "hive_table";
+	public static final String ENTITY_TYPE_HIVE_DB     = "hive_db";
+	public static final String ENTITY_TYPE_HIVE_TABLE  = "hive_table";
 	public static final String ENTITY_TYPE_HIVE_COLUMN = "hive_column";
 
-	public static final String RANGER_TYPE_HIVE_DB = "database";
-	public static final String RANGER_TYPE_HIVE_TABLE = "table";
+	public static final String RANGER_TYPE_HIVE_DB     = "database";
+	public static final String RANGER_TYPE_HIVE_TABLE  = "table";
 	public static final String RANGER_TYPE_HIVE_COLUMN = "column";
 
 	public static final String ENTITY_ATTRIBUTE_QUALIFIED_NAME = "qualifiedName";
+	public static final String QUALIFIED_NAME_DELIMITER        = "\\.";
 
-	public static final String TAGSYNC_DEFAULT_CLUSTERNAME_AND_COMPONENTNAME_SEPARATOR = "_";
-
-	public static final String clusterDelimiter = "@";
-
-	public static final String qualifiedNameDelimiter = "\\.";
-
-	public static final String[] supportedEntityTypes = { ENTITY_TYPE_HIVE_DB, ENTITY_TYPE_HIVE_TABLE, ENTITY_TYPE_HIVE_COLUMN };
+	public static final String[] SUPPORTED_ENTITY_TYPES = { ENTITY_TYPE_HIVE_DB, ENTITY_TYPE_HIVE_TABLE, ENTITY_TYPE_HIVE_COLUMN };
 
 	public AtlasHiveResourceMapper() {
-		super();
-	}
-
-	@Override
-	public List<String> getSupportedEntityTypes() {
-		return Arrays.asList(supportedEntityTypes);
+		super("hive", SUPPORTED_ENTITY_TYPES);
 	}
 
 	@Override
 	public RangerServiceResource buildResource(final IReferenceableInstance entity) throws Exception {
-
-		Map<String, RangerPolicy.RangerPolicyResource> elements = new HashMap<String, RangerPolicy.RangerPolicyResource>();
-
-		String serviceName = null;
-
-		List<String> components = getQualifiedNameComponents(entity);
-		// components should contain qualifiedName, clusterName, dbName, tableName, columnName in that order
-
-		String entityTypeName = entity.getTypeName();
-
-		String qualifiedName = components.get(0);
-
-		String clusterName, dbName, tableName, columnName;
-
-		if (components.size() > 1) {
-			clusterName = components.get(1);
-			serviceName = getRangerServiceName(clusterName);
-		}
-
-		if (StringUtils.equals(entityTypeName, ENTITY_TYPE_HIVE_DB)) {
-			if (components.size() > 2) {
-				dbName = components.get(2);
-				RangerPolicy.RangerPolicyResource dbPolicyResource = new RangerPolicy.RangerPolicyResource(dbName);
-				elements.put(RANGER_TYPE_HIVE_DB, dbPolicyResource);
-
-			} else {
-				LOG.error("invalid qualifiedName for HIVE_DB, qualifiedName=" + qualifiedName);
-				throw new Exception("invalid qualifiedName for HIVE_DB, qualifiedName=" + qualifiedName);
-			}
-		} else if (StringUtils.equals(entityTypeName, ENTITY_TYPE_HIVE_TABLE)) {
-			if (components.size() > 3) {
-				dbName = components.get(2);
-				tableName = components.get(3);
-				RangerPolicy.RangerPolicyResource dbPolicyResource = new RangerPolicy.RangerPolicyResource(dbName);
-				elements.put(RANGER_TYPE_HIVE_DB, dbPolicyResource);
-				RangerPolicy.RangerPolicyResource tablePolicyResource = new RangerPolicy.RangerPolicyResource(tableName);
-				elements.put(RANGER_TYPE_HIVE_TABLE, tablePolicyResource);
-			} else {
-				LOG.error("invalid qualifiedName for HIVE_TABLE, qualifiedName=" + qualifiedName);
-				throw new Exception("invalid qualifiedName for HIVE_TABLE, qualifiedName=" + qualifiedName);
-			}
-		} else if (StringUtils.equals(entityTypeName, ENTITY_TYPE_HIVE_COLUMN)) {
-			if (components.size() > 4) {
-				dbName = components.get(2);
-				tableName = components.get(3);
-				columnName = components.get(4);
-				RangerPolicy.RangerPolicyResource dbPolicyResource = new RangerPolicy.RangerPolicyResource(dbName);
-				elements.put(RANGER_TYPE_HIVE_DB, dbPolicyResource);
-				RangerPolicy.RangerPolicyResource tablePolicyResource = new RangerPolicy.RangerPolicyResource(tableName);
-				elements.put(RANGER_TYPE_HIVE_TABLE, tablePolicyResource);
-				RangerPolicy.RangerPolicyResource columnPolicyResource = new RangerPolicy.RangerPolicyResource(columnName);
-				elements.put(RANGER_TYPE_HIVE_COLUMN, columnPolicyResource);
-			} else {
-				LOG.error("invalid qualifiedName for HIVE_COLUMN, qualifiedName=" + qualifiedName);
-				throw new Exception("invalid qualifiedName for HIVE_COLUMN, qualifiedName=" + qualifiedName);
-			}
-
+		String qualifiedName = getEntityAttribute(entity, ENTITY_ATTRIBUTE_QUALIFIED_NAME, String.class);
+		if (StringUtils.isEmpty(qualifiedName)) {
+			throw new Exception("attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "' not found in entity");
 		}
 
-		RangerServiceResource ret = new RangerServiceResource();
-
-		ret.setGuid(entity.getId()._getId());
-		ret.setServiceName(serviceName);
-		ret.setResourceElements(elements);
-
-		return ret;
-	}
-
-	public String getRangerServiceName(String clusterName) {
-		String ret = getRangerServiceName(COMPONENT_NAME, clusterName);
-
-		if (StringUtils.isBlank(ret)) {
-			ret = clusterName + TAGSYNC_DEFAULT_CLUSTERNAME_AND_COMPONENTNAME_SEPARATOR + COMPONENT_NAME;
+		String resourceStr = getResourceNameFromQualifiedName(qualifiedName);
+		if (StringUtils.isEmpty(resourceStr)) {
+			throwExceptionWithMessage("resource not found in attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "': " + qualifiedName);
 		}
-		return ret;
-	}
-
-	public final List<String> getQualifiedNameComponents(IReferenceableInstance entity) throws Exception {
-
-		String qualifiedNameAttributeName = ENTITY_ATTRIBUTE_QUALIFIED_NAME;
-		String qualifiedName = getEntityAttribute(entity, qualifiedNameAttributeName, String.class);
 
-		if (StringUtils.isBlank(qualifiedName)) {
-			throw new Exception("Could not get a valid value for " + qualifiedNameAttributeName + " attribute from entity.");
+		String clusterName = getClusterNameFromQualifiedName(qualifiedName);
+		if (StringUtils.isEmpty(clusterName)) {
+			throwExceptionWithMessage("cluster-name not found in attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "': " + qualifiedName);
 		}
 
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Received .... " + qualifiedNameAttributeName + "=" + qualifiedName + " for entity type " + entity.getTypeName());
-		}
+		String   entityType  = entity.getTypeName();
+		String   entityGuid  = entity.getId() != null ? entity.getId()._getId() : null;
+		String   serviceName = getRangerServiceName(clusterName);
+		String[] resources   = resourceStr.split(QUALIFIED_NAME_DELIMITER);
+		String   dbName      = resources.length > 0 ? resources[0] : null;
+		String   tblName     = resources.length > 1 ? resources[1] : null;
+		String   colName     = resources.length > 2 ? resources[2] : null;
 
-		List<String> ret = getQualifiedNameComponents(entity.getTypeName(), qualifiedName);
+		Map<String, RangerPolicyResource> elements = new HashMap<String, RangerPolicyResource>();
 
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("----- Entity-Id:" + entity.getId()._getId());
-			LOG.debug("----- Entity-Type-Name:" + entity.getTypeName());
-			LOG.debug("----- 	Entity-Components -----");
-			int i = 0;
-			for (String value : ret) {
-				LOG.debug("-----		Index:" + i++ + "	Value:" + value);
+		if (StringUtils.equals(entityType, ENTITY_TYPE_HIVE_DB)) {
+			if (StringUtils.isNotEmpty(dbName)) {
+				elements.put(RANGER_TYPE_HIVE_DB, new RangerPolicyResource(dbName));
 			}
-		}
-		return ret;
-	}
-
-	public final List<String> getQualifiedNameComponents(String entityTypeName, String qualifiedName) throws Exception {
-
-		String components[] = qualifiedName.split(clusterDelimiter);
-
-		if (components.length != 2) {
-			throw new Exception("Qualified Name does not contain cluster-name, qualifiedName=" + qualifiedName);
+		} else if (StringUtils.equals(entityType, ENTITY_TYPE_HIVE_TABLE)) {
+			if (StringUtils.isNotEmpty(dbName) && StringUtils.isNotEmpty(tblName)) {
+				elements.put(RANGER_TYPE_HIVE_DB, new RangerPolicyResource(dbName));
+				elements.put(RANGER_TYPE_HIVE_TABLE, new RangerPolicyResource(tblName));
+			}
+		} else if (StringUtils.equals(entityType, ENTITY_TYPE_HIVE_COLUMN)) {
+			if (StringUtils.isNotEmpty(dbName) && StringUtils.isNotEmpty(tblName) && StringUtils.isNotEmpty(colName)) {
+				elements.put(RANGER_TYPE_HIVE_DB, new RangerPolicyResource(dbName));
+				elements.put(RANGER_TYPE_HIVE_TABLE, new RangerPolicyResource(tblName));
+				elements.put(RANGER_TYPE_HIVE_COLUMN, new RangerPolicyResource(colName));
+			}
+		} else {
+			throwExceptionWithMessage("unrecognized entity-type: " + entityType);
 		}
 
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("name-hierarchy=" + components[0] + ", cluster-name=" + components[1]);
+		if(elements.isEmpty()) {
+			throwExceptionWithMessage("invalid qualifiedName for entity-type '" + entityType + "': " + qualifiedName);
 		}
 
-		String nameHierarchy[] = components[0].split(qualifiedNameDelimiter);
-
-		List<String> ret = new ArrayList<String>();
-
-		ret.add(qualifiedName);
-		ret.add(components[1]);
-
-		ret.addAll(Arrays.asList(nameHierarchy));
+		RangerServiceResource ret = new RangerServiceResource(entityGuid, serviceName, elements);
 
 		return ret;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/74959da1/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapper.java
----------------------------------------------------------------------
diff --git a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapper.java b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapper.java
index d47d052..8ececdf 100644
--- a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapper.java
+++ b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapper.java
@@ -19,44 +19,104 @@
 
 package org.apache.ranger.tagsync.source.atlas;
 
+import java.util.Properties;
+import java.util.Map;
+
 import org.apache.atlas.AtlasException;
 import org.apache.atlas.typesystem.IReferenceableInstance;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ranger.plugin.model.RangerServiceResource;
 
-import java.util.Properties;
-import java.util.List;
-import java.util.Map;
-
 public abstract class AtlasResourceMapper {
 	private static final Log LOG = LogFactory.getLog(AtlasResourceMapper.class);
 
-	protected static final String TAGSYNC_SERVICENAME_MAPPER_PROP_PREFIX = "ranger.tagsync.atlas.";
+	public static final String TAGSYNC_DEFAULT_CLUSTER_NAME = "ranger.tagsync.atlas.default.cluster.name";
 
-	protected static final String TAGSYNC_SERVICENAME_MAPPER_PROP_SUFFIX = ".ranger.service";
+	protected static final String TAGSYNC_SERVICENAME_MAPPER_PROP_PREFIX                  = "ranger.tagsync.atlas.";
+	protected static final String TAGSYNC_SERVICENAME_MAPPER_PROP_SUFFIX                  = ".ranger.service";
+	protected static final String TAGSYNC_ATLAS_CLUSTER_IDENTIFIER                        = ".instance.";
+	protected static final String TAGSYNC_DEFAULT_CLUSTERNAME_AND_COMPONENTNAME_SEPARATOR = "_";
+	protected static final String CLUSTER_DELIMITER                                       = "@";
+
+	protected final String   componentName;
+	protected final String[] supportedEntityTypes;
 
-	protected static final String TAGSYNC_ATLAS_CLUSTER_IDENTIFIER = ".instance.";
 	protected Properties properties;
+	protected String     defaultClusterName;
 
-	public AtlasResourceMapper() {
+	public AtlasResourceMapper(String componentName, String[] supportedEntityTypes) {
+		this.componentName        = componentName;
+		this.supportedEntityTypes = supportedEntityTypes;
 	}
 
-	public void initialize(Properties properties) {
-		this.properties = properties;
+	public final String getComponentName() {
+		return componentName;
+	}
+
+	public final String[] getSupportedEntityTypes() {
+		return supportedEntityTypes;
+	}
+
+	public String getRangerServiceName(String clusterName) {
+		String ret = getCustomRangerServiceName(clusterName);
+
+		if (StringUtils.isBlank(ret)) {
+			ret = clusterName + TAGSYNC_DEFAULT_CLUSTERNAME_AND_COMPONENTNAME_SEPARATOR + componentName;
+		}
+		return ret;
 	}
 
-	abstract public List<String> getSupportedEntityTypes();
+	public void initialize(Properties properties) {
+		this.properties         = properties;
+		this.defaultClusterName = properties != null ? properties.getProperty(TAGSYNC_DEFAULT_CLUSTER_NAME) : null;
+	}
 
 	abstract public RangerServiceResource buildResource(final IReferenceableInstance entity) throws Exception;
 
+	protected String getCustomRangerServiceName(String atlasInstanceName) {
+		if(properties != null) {
+			String propName = TAGSYNC_SERVICENAME_MAPPER_PROP_PREFIX + componentName
+					+ TAGSYNC_ATLAS_CLUSTER_IDENTIFIER + atlasInstanceName
+					+ TAGSYNC_SERVICENAME_MAPPER_PROP_SUFFIX;
+
+			return properties.getProperty(propName);
+		} else {
+			return null;
+		}
+	}
+
+	protected  String getResourceNameFromQualifiedName(String qualifiedName) {
+		if(StringUtils.isNotBlank(qualifiedName)) {
+			int idx = qualifiedName.lastIndexOf(CLUSTER_DELIMITER);
+
+			if(idx != -1) {
+				return qualifiedName.substring(0, idx);
+			} else {
+				return qualifiedName;
+			}
+		}
+
+		return null;
+	}
+
+	protected  String getClusterNameFromQualifiedName(String qualifiedName) {
+		if(StringUtils.isNotBlank(qualifiedName)) {
+			int idx = qualifiedName.lastIndexOf(CLUSTER_DELIMITER);
+
+			if(idx != -1 && qualifiedName.length() > idx) {
+				return qualifiedName.substring(idx + 1);
+			}
+		}
+
+		return null;
+	}
 
-	protected String getRangerServiceName(String componentName, String atlasInstanceName) {
-		String propName = TAGSYNC_SERVICENAME_MAPPER_PROP_PREFIX + componentName
-				+ TAGSYNC_ATLAS_CLUSTER_IDENTIFIER + atlasInstanceName
-				+ TAGSYNC_SERVICENAME_MAPPER_PROP_SUFFIX;
+	protected void throwExceptionWithMessage(String msg) throws Exception {
+		LOG.error(msg);
 
-		return properties.getProperty(propName);
+		throw new Exception(msg);
 	}
 
 	static protected <T> T getEntityAttribute(IReferenceableInstance entity, String name, Class<T> type) {

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/74959da1/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java
----------------------------------------------------------------------
diff --git a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java
index f05d814..14b2001 100644
--- a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java
+++ b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java
@@ -84,18 +84,17 @@ public class AtlasResourceMapperUtil {
 		}
 		boolean ret = true;
 
-		String allResourceMappers = "org.apache.ranger.tagsync.source.atlas.AtlasHiveResourceMapper";
+		List<String> mapperNames = new ArrayList<String>();
+		mapperNames.add("org.apache.ranger.tagsync.source.atlas.AtlasHiveResourceMapper");
+		mapperNames.add("org.apache.ranger.tagsync.source.atlas.AtlasHdfsResourceMapper");
 
 		if (StringUtils.isNotBlank(customMapperNames)) {
-			allResourceMappers = allResourceMappers + MAPPER_NAME_DELIMIER + customMapperNames;
+			for(String customMapperName : customMapperNames.split(MAPPER_NAME_DELIMIER)) {
+				mapperNames.add(customMapperName.trim());
+			}
 		}
 
-		String[] mapperNamesArray = allResourceMappers.split(MAPPER_NAME_DELIMIER);
-
-		List<String> mapperNames = Arrays.asList(mapperNamesArray);
-
 		for (String mapperName : mapperNames) {
-			mapperName = mapperName.trim();
 			try {
 				Class clazz = Class.forName(mapperName);
 				AtlasResourceMapper resourceMapper = (AtlasResourceMapper) clazz.newInstance();
@@ -113,7 +112,7 @@ public class AtlasResourceMapperUtil {
 		}
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("<== initializeAtlasResourceMappers.initializeAtlasResourceMappers(" + allResourceMappers + "): " + ret);
+			LOG.debug("<== initializeAtlasResourceMappers.initializeAtlasResourceMappers(" + mapperNames + "): " + ret);
 		}
 		return ret;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/74959da1/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestHdfsResourceMapper.java
----------------------------------------------------------------------
diff --git a/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestHdfsResourceMapper.java b/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestHdfsResourceMapper.java
new file mode 100644
index 0000000..0986331
--- /dev/null
+++ b/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestHdfsResourceMapper.java
@@ -0,0 +1,167 @@
+/*
+ * 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.tagsync.process;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.atlas.typesystem.IReferenceableInstance;
+import org.apache.ranger.plugin.model.RangerServiceResource;
+import org.apache.ranger.tagsync.source.atlas.AtlasHdfsResourceMapper;
+import org.apache.ranger.tagsync.source.atlas.AtlasResourceMapper;
+import org.junit.Test;
+
+import org.mockito.Mockito;
+import org.junit.Assert;
+
+
+
+public class TestHdfsResourceMapper {
+	private static final String CLUSTER_NAME   = "cl1";
+	private static final String PATH           = "hdfs://localhost:8020/user/testuser/finance";
+	private static final String QUALIFIED_NAME = "hdfs://localhost:8020/user/testuser/finance@cl1";
+
+	private static final String SERVICE_NAME = "cl1_hadoop";
+	private static final String RANGER_PATH  = "/user/testuser/finance";
+
+	AtlasHdfsResourceMapper resourceMapper = new AtlasHdfsResourceMapper();
+	AtlasHdfsResourceMapper resourceMapperWithDefaultClusterName = new AtlasHdfsResourceMapper();
+
+	{
+		Properties properties = new Properties();
+
+		properties.setProperty(AtlasResourceMapper.TAGSYNC_DEFAULT_CLUSTER_NAME, CLUSTER_NAME);
+
+		resourceMapperWithDefaultClusterName.initialize(properties);
+	}
+
+	@Test
+	public void testHdfsResourceFromPathAndClusterName() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_PATH, PATH);
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_CLUSTER_NAME, CLUSTER_NAME);
+
+		IReferenceableInstance entity   = getHdfsPathEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapper.buildResource(entity);
+
+		assertServiceResource(resource);
+	}
+
+	@Test
+	public void testHdfsResourceFromPathAndQualifiedName() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_PATH, PATH);
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME, QUALIFIED_NAME);
+
+		IReferenceableInstance entity   = getHdfsPathEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapper.buildResource(entity);
+
+		assertServiceResource(resource);
+	}
+
+	@Test
+	public void testHdfsResourceFromClusterNameAndQualifiedName() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_CLUSTER_NAME, CLUSTER_NAME);
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME, QUALIFIED_NAME);
+
+		IReferenceableInstance entity   = getHdfsPathEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapper.buildResource(entity);
+
+		assertServiceResource(resource);
+	}
+
+	@Test
+	public void testHdfsResourceFromPathAndClusterNameAndQualifiedName() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_PATH, PATH);
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_CLUSTER_NAME, CLUSTER_NAME);
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME, QUALIFIED_NAME);
+
+		IReferenceableInstance entity   = getHdfsPathEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapper.buildResource(entity);
+
+		assertServiceResource(resource);
+	}
+
+	@Test
+	public void testHdfsResourceFromQualifiedNameAndClusterNameFromDefault() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME, PATH);
+
+		IReferenceableInstance entity   = getHdfsPathEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapperWithDefaultClusterName.buildResource(entity);
+
+		assertServiceResource(resource);
+	}
+
+	@Test
+	public void testHdfsResourceFromPathAndClusterNameFromDefault() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHdfsResourceMapper.ENTITY_ATTRIBUTE_PATH, PATH);
+
+		IReferenceableInstance entity   = getHdfsPathEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapperWithDefaultClusterName.buildResource(entity);
+
+		assertServiceResource(resource);
+	}
+
+	@Test
+	public void testHdfsResourceFromMissingAttribs() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		IReferenceableInstance entity  = getHdfsPathEntity(entAttribs);
+
+		try {
+			RangerServiceResource resource = resourceMapper.buildResource(entity);
+
+			Assert.fail("expected exception. Found " + resource);
+		} catch(Exception excp) {
+			// ignore
+		}
+	}
+
+	private IReferenceableInstance getHdfsPathEntity(Map<String, Object> entAttribs) throws Exception {
+		IReferenceableInstance entity = Mockito.mock(IReferenceableInstance.class);
+
+		Mockito.when(entity.getTypeName()).thenReturn(AtlasHdfsResourceMapper.ENTITY_TYPE_HDFS_PATH);
+		Mockito.when(entity.getValuesMap()).thenReturn(entAttribs);
+
+		return entity;
+	}
+
+	private void assertServiceResource(RangerServiceResource resource) {
+		Assert.assertNotNull(resource);
+		Assert.assertEquals(SERVICE_NAME, resource.getServiceName());
+		Assert.assertNotNull(resource.getResourceElements());
+		Assert.assertEquals(1, resource.getResourceElements().size());
+		Assert.assertTrue(resource.getResourceElements().containsKey(AtlasHdfsResourceMapper.RANGER_TYPE_HDFS_PATH));
+		Assert.assertNotNull(resource.getResourceElements().get(AtlasHdfsResourceMapper.RANGER_TYPE_HDFS_PATH).getValues());
+		Assert.assertEquals(1, resource.getResourceElements().get(AtlasHdfsResourceMapper.RANGER_TYPE_HDFS_PATH).getValues().size());
+		Assert.assertEquals(RANGER_PATH, resource.getResourceElements().get(AtlasHdfsResourceMapper.RANGER_TYPE_HDFS_PATH).getValues().get(0));
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/74959da1/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestHiveResourceMapper.java
----------------------------------------------------------------------
diff --git a/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestHiveResourceMapper.java b/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestHiveResourceMapper.java
new file mode 100644
index 0000000..aaf9590
--- /dev/null
+++ b/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestHiveResourceMapper.java
@@ -0,0 +1,173 @@
+/*
+ * 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.tagsync.process;
+
+import org.apache.atlas.typesystem.IReferenceableInstance;
+import org.apache.ranger.plugin.model.RangerServiceResource;
+import org.apache.ranger.tagsync.source.atlas.AtlasHiveResourceMapper;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+public class TestHiveResourceMapper {
+	private static final String DB_QUALIFIED_NAME     = "default@cl1";
+	private static final String TABLE_QUALIFIED_NAME  = "default.testTable@cl1";
+	private static final String COLUMN_QUALIFIED_NAME = "default.testTable.col1@cl1";
+
+	private static final String SERVICE_NAME    = "cl1_hive";
+	private static final String RANGER_DATABASE = "default";
+	private static final String RANGER_TABLE    = "testTable";
+	private static final String RANGER_COLUMN   = "col1";
+
+	AtlasHiveResourceMapper resourceMapper = new AtlasHiveResourceMapper();
+
+	@Test
+	public void testHiveDb() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHiveResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME, DB_QUALIFIED_NAME);
+
+		IReferenceableInstance entity   = getHiveDbEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapper.buildResource(entity);
+
+		assertDbResource(resource);
+	}
+
+	@Test
+	public void testHiveTable() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHiveResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME, TABLE_QUALIFIED_NAME);
+
+		IReferenceableInstance entity   = getHiveTableEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapper.buildResource(entity);
+
+		assertTableResource(resource);
+	}
+
+	@Test
+	public void testHiveColumn() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		entAttribs.put(AtlasHiveResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME, COLUMN_QUALIFIED_NAME);
+
+		IReferenceableInstance entity   = getHiveColumnEntity(entAttribs);
+		RangerServiceResource  resource = resourceMapper.buildResource(entity);
+
+		assertColumnResource(resource);
+	}
+
+	@Test
+	public void testHiveResourceFromMissingAttribs() throws Exception {
+		Map<String, Object> entAttribs = new HashMap<String, Object>();
+
+		IReferenceableInstance entity = getHiveDbEntity(entAttribs);
+
+		try {
+			RangerServiceResource resource = resourceMapper.buildResource(entity);
+
+			Assert.fail("expected exception. Found " + resource);
+		} catch(Exception excp) {
+			// ignore
+		}
+	}
+
+	private IReferenceableInstance getHiveDbEntity(Map<String, Object> entAttribs) throws Exception {
+		IReferenceableInstance entity = Mockito.mock(IReferenceableInstance.class);
+
+		Mockito.when(entity.getTypeName()).thenReturn(AtlasHiveResourceMapper.ENTITY_TYPE_HIVE_DB);
+		Mockito.when(entity.getValuesMap()).thenReturn(entAttribs);
+
+		return entity;
+	}
+
+	private IReferenceableInstance getHiveTableEntity(Map<String, Object> entAttribs) throws Exception {
+		IReferenceableInstance entity = Mockito.mock(IReferenceableInstance.class);
+
+		Mockito.when(entity.getTypeName()).thenReturn(AtlasHiveResourceMapper.ENTITY_TYPE_HIVE_TABLE);
+		Mockito.when(entity.getValuesMap()).thenReturn(entAttribs);
+
+		return entity;
+	}
+
+	private IReferenceableInstance getHiveColumnEntity(Map<String, Object> entAttribs) throws Exception {
+		IReferenceableInstance entity = Mockito.mock(IReferenceableInstance.class);
+
+		Mockito.when(entity.getTypeName()).thenReturn(AtlasHiveResourceMapper.ENTITY_TYPE_HIVE_COLUMN);
+		Mockito.when(entity.getValuesMap()).thenReturn(entAttribs);
+
+		return entity;
+	}
+
+	private void assertServiceResource(RangerServiceResource resource) {
+		Assert.assertNotNull(resource);
+		Assert.assertEquals(SERVICE_NAME, resource.getServiceName());
+		Assert.assertNotNull(resource.getResourceElements());
+	}
+
+	private void assertDbResource(RangerServiceResource resource) {
+		assertServiceResource(resource);
+
+		Assert.assertEquals(1, resource.getResourceElements().size());
+		Assert.assertTrue(resource.getResourceElements().containsKey(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB));
+		Assert.assertNotNull(resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues());
+		Assert.assertEquals(1, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues().size());
+		Assert.assertEquals(RANGER_DATABASE, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues().get(0));
+	}
+
+	private void assertTableResource(RangerServiceResource resource) {
+		assertServiceResource(resource);
+
+		Assert.assertEquals(2, resource.getResourceElements().size());
+		Assert.assertTrue(resource.getResourceElements().containsKey(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB));
+		Assert.assertNotNull(resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues());
+		Assert.assertEquals(1, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues().size());
+		Assert.assertEquals(RANGER_DATABASE, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues().get(0));
+
+		Assert.assertTrue(resource.getResourceElements().containsKey(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_TABLE));
+		Assert.assertNotNull(resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_TABLE).getValues());
+		Assert.assertEquals(1, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_TABLE).getValues().size());
+		Assert.assertEquals(RANGER_TABLE, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_TABLE).getValues().get(0));
+	}
+
+	private void assertColumnResource(RangerServiceResource resource) {
+		assertServiceResource(resource);
+
+		Assert.assertEquals(3, resource.getResourceElements().size());
+		Assert.assertTrue(resource.getResourceElements().containsKey(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB));
+		Assert.assertNotNull(resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues());
+		Assert.assertEquals(1, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues().size());
+		Assert.assertEquals(RANGER_DATABASE, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_DB).getValues().get(0));
+
+		Assert.assertTrue(resource.getResourceElements().containsKey(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_TABLE));
+		Assert.assertNotNull(resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_TABLE).getValues());
+		Assert.assertEquals(1, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_TABLE).getValues().size());
+		Assert.assertEquals(RANGER_TABLE, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_TABLE).getValues().get(0));
+
+		Assert.assertTrue(resource.getResourceElements().containsKey(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_COLUMN));
+		Assert.assertNotNull(resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_COLUMN).getValues());
+		Assert.assertEquals(1, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_COLUMN).getValues().size());
+		Assert.assertEquals(RANGER_COLUMN, resource.getResourceElements().get(AtlasHiveResourceMapper.RANGER_TYPE_HIVE_COLUMN).getValues().get(0));
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/74959da1/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestTagSynchronizer.java
----------------------------------------------------------------------
diff --git a/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestTagSynchronizer.java b/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestTagSynchronizer.java
index c644e0d..13af2cb 100644
--- a/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestTagSynchronizer.java
+++ b/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestTagSynchronizer.java
@@ -20,13 +20,11 @@
 package org.apache.ranger.tagsync.process;
 
 
-import org.apache.ranger.tagsync.source.atlas.AtlasHiveResourceMapper;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.*;
-import java.util.List;
 import java.util.Properties;
 
 
@@ -73,33 +71,4 @@ public class TestTagSynchronizer {
 		assert(initDone);
 
 	}
-
-	@Test
-	public void testQualifiedNames() {
-
-		List<String> components;
-		AtlasHiveResourceMapper hiveResourceBuilder = new AtlasHiveResourceMapper();
-		try {
-			components = hiveResourceBuilder.getQualifiedNameComponents("hive_db", "database@cluster");
-			printComponents(components);
-
-			components = hiveResourceBuilder.getQualifiedNameComponents("hive_table", "database.table@cluster");
-			printComponents(components);
-
-			components = hiveResourceBuilder.getQualifiedNameComponents("hive_column", "database.table.column@cluster");
-			printComponents(components);
-
-			assert(true);
-		} catch (Exception e) {
-			System.out.println("Failed...");
-			assert(false);
-		}
-
-	}
-	private void printComponents(List<String> components) {
-		int i = 0;
-		for (String value : components) {
-			System.out.println("-----		Index:" + i++ + "	Value:" + value);
-		}
-	}
 }