You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by zh...@apache.org on 2017/03/08 11:46:58 UTC

ranger git commit: RANGER-1318 - Consolidate XML configuration parsing

Repository: ranger
Updated Branches:
  refs/heads/master 20315eabb -> 5ea8062df


RANGER-1318 - Consolidate XML configuration parsing

Signed-off-by: zhangqiang2 <zh...@zte.com.cn>


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

Branch: refs/heads/master
Commit: 5ea8062df77769ddb02235c7e8f3b51185143b3d
Parents: 20315ea
Author: Colm O hEigeartaigh <co...@apache.org>
Authored: Tue Mar 7 17:48:34 2017 +0000
Committer: zhangqiang2 <zh...@zte.com.cn>
Committed: Wed Mar 8 06:34:15 2017 -0500

----------------------------------------------------------------------
 .../org/apache/ranger/plugin/util/XMLUtils.java | 128 +++++++++++++++++++
 .../ranger/server/tomcat/EmbeddedServer.java    |  90 +------------
 .../apache/ranger/common/RangerProperties.java  | 118 +----------------
 .../apache/ranger/common/XMLPropertiesUtil.java |  50 +-------
 .../config/UserGroupSyncConfig.java             | 115 +----------------
 .../unix/jaas/RemoteUnixLoginModule.java        |  66 +---------
 .../UnixAuthenticationService.java              |  67 +---------
 7 files changed, 148 insertions(+), 486 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ranger/blob/5ea8062d/agents-common/src/main/java/org/apache/ranger/plugin/util/XMLUtils.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/util/XMLUtils.java b/agents-common/src/main/java/org/apache/ranger/plugin/util/XMLUtils.java
new file mode 100644
index 0000000..4647004
--- /dev/null
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/util/XMLUtils.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.ranger.plugin.util;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.util.Map;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+
+import org.apache.log4j.Logger;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class XMLUtils {
+
+	private static final Logger LOG = Logger.getLogger(XMLUtils.class);
+
+	private static final String XMLCONFIG_PROPERTY_TAGNAME = "property";
+	private static final String XMLCONFIG_NAME_TAGNAME = "name";
+	private static final String XMLCONFIG_VALUE_TAGNAME = "value";
+
+	public static void loadConfig(String configFileName, Map<Object, Object> properties) {
+		try (InputStream input = getFileInputStream(configFileName)) {
+			loadConfig(input, properties);
+		} catch (Exception e) {
+			LOG.error("Error loading : ", e);
+		}
+	}
+
+	public static void loadConfig(InputStream input, Map<Object, Object> properties) {
+		try {
+			DocumentBuilderFactory xmlDocumentBuilderFactory = DocumentBuilderFactory.newInstance();
+			xmlDocumentBuilderFactory.setIgnoringComments(true);
+			xmlDocumentBuilderFactory.setNamespaceAware(true);
+
+			DocumentBuilder xmlDocumentBuilder = xmlDocumentBuilderFactory.newDocumentBuilder();
+			Document xmlDocument = xmlDocumentBuilder.parse(input);
+			xmlDocument.getDocumentElement().normalize();
+
+			NodeList nList = xmlDocument.getElementsByTagName(XMLCONFIG_PROPERTY_TAGNAME);
+
+			for (int temp = 0; temp < nList.getLength(); temp++) {
+
+				Node nNode = nList.item(temp);
+
+				if (nNode.getNodeType() == Node.ELEMENT_NODE) {
+
+					Element eElement = (Element) nNode;
+
+					String propertyName = "";
+					String propertyValue = "";
+					if (eElement.getElementsByTagName(XMLCONFIG_NAME_TAGNAME).item(0) != null) {
+						propertyName = eElement.getElementsByTagName(XMLCONFIG_NAME_TAGNAME)
+								.item(0).getTextContent().trim();
+					}
+					if (eElement.getElementsByTagName(XMLCONFIG_VALUE_TAGNAME).item(0) != null) {
+						propertyValue = eElement.getElementsByTagName(XMLCONFIG_VALUE_TAGNAME)
+								.item(0).getTextContent().trim();
+					}
+
+					if (properties.get(propertyName) != null) {
+						properties.remove(propertyName);
+					}
+
+					properties.put(propertyName, propertyValue);
+
+				}
+			}
+
+		} catch (Exception e) {
+			LOG.error("Error loading : ", e);
+		}
+	}
+
+	private static InputStream getFileInputStream(String path) throws FileNotFoundException {
+
+		InputStream ret = null;
+
+		File f = new File(path);
+
+		if (f.exists()) {
+			ret = new FileInputStream(f);
+		} else {
+			ret = XMLUtils.class.getResourceAsStream(path);
+
+			if (ret == null) {
+				if (! path.startsWith("/")) {
+					ret = XMLUtils.class.getResourceAsStream("/" + path);
+				}
+			}
+
+			if (ret == null) {
+				ret = ClassLoader.getSystemClassLoader().getResourceAsStream(path);
+				if (ret == null) {
+					if (! path.startsWith("/")) {
+						ret = ClassLoader.getSystemResourceAsStream("/" + path);
+					}
+				}
+			}
+		}
+
+		return ret;
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ranger/blob/5ea8062d/embeddedwebserver/src/main/java/org/apache/ranger/server/tomcat/EmbeddedServer.java
----------------------------------------------------------------------
diff --git a/embeddedwebserver/src/main/java/org/apache/ranger/server/tomcat/EmbeddedServer.java b/embeddedwebserver/src/main/java/org/apache/ranger/server/tomcat/EmbeddedServer.java
index 635e63f..aef59bd 100644
--- a/embeddedwebserver/src/main/java/org/apache/ranger/server/tomcat/EmbeddedServer.java
+++ b/embeddedwebserver/src/main/java/org/apache/ranger/server/tomcat/EmbeddedServer.java
@@ -21,7 +21,6 @@ package org.apache.ranger.server.tomcat;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.URL;
 import java.security.PrivilegedAction;
 import java.util.Date;
 import java.util.Iterator;
@@ -29,8 +28,6 @@ import java.util.Properties;
 import java.util.logging.Logger;
 import java.util.List;
 import javax.servlet.ServletException;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.catalina.Context;
 import org.apache.catalina.LifecycleException;
@@ -42,13 +39,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.ranger.plugin.util.XMLUtils;
 import javax.security.auth.Subject;
 
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-
 public class EmbeddedServer {
 	
 	private static final Logger LOG = Logger.getLogger(EmbeddedServer.class
@@ -80,9 +73,9 @@ public class EmbeddedServer {
 		if (args.length > 0) {
 			configFile = args[0];
 		}
-		loadConfig(CORE_SITE_CONFIG_FILENAME);
-		loadConfig(DEFAULT_CONFIG_FILENAME);
-		loadConfig(configFile);		
+        XMLUtils.loadConfig(CORE_SITE_CONFIG_FILENAME, serverConfigProperties);
+        XMLUtils.loadConfig(DEFAULT_CONFIG_FILENAME, serverConfigProperties);
+        XMLUtils.loadConfig(configFile, serverConfigProperties);
 	}
 	
 	public static int DEFAULT_SHUTDOWN_PORT = 6185;
@@ -326,37 +319,6 @@ public class EmbeddedServer {
 		return ret;
 	}
 	
-	private String getResourceFileName(String aResourceName) {
-		
-		String ret = aResourceName;
-		
-		ClassLoader cl = getClass().getClassLoader();
-		
-		for (String path : new String[] { aResourceName, "/" + aResourceName }) {
-			
-			try {
-				URL lurl = cl.getResource(path);
-		
-				if (lurl != null) {
-					ret = lurl.getFile();
-				}
-			} catch (Throwable t) {
-				ret = null;
-			}
-			if (ret != null) {
-				break;
-			}
-
-		}
-		
-		if (ret == null) {
-			ret = aResourceName;
-		}
-		
-		return ret;
-		
-	}
-	
 	public void shutdownServer() {
 		int timeWaitForShutdownInSeconds = getIntConfig(
 				"service.waitTimeForForceShutdownInSeconds", 0);
@@ -387,50 +349,6 @@ public class EmbeddedServer {
 		System.exit(0);
 	}
 
-
-	public void loadConfig(String configFileName) {
-		String path = getResourceFileName(configFileName);
-		try {
-			DocumentBuilderFactory xmlDocumentBuilderFactory = DocumentBuilderFactory
-					.newInstance();
-			xmlDocumentBuilderFactory.setIgnoringComments(true);
-			xmlDocumentBuilderFactory.setNamespaceAware(true);
-			DocumentBuilder xmlDocumentBuilder = xmlDocumentBuilderFactory
-					.newDocumentBuilder();
-			Document xmlDocument = xmlDocumentBuilder.parse(new File(path));
-			xmlDocument.getDocumentElement().normalize();
-
-			NodeList nList = xmlDocument.getElementsByTagName("property");
-
-			for (int temp = 0; temp < nList.getLength(); temp++) {
-
-				Node nNode = nList.item(temp);
-
-				if (nNode.getNodeType() == Node.ELEMENT_NODE) {
-
-					Element eElement = (Element) nNode;
-
-					String propertyName = "";
-					String propertyValue = "";
-					if (eElement.getElementsByTagName("name").item(0) != null) {
-						propertyName = eElement.getElementsByTagName("name")
-								.item(0).getTextContent().trim();
-					}
-					if (eElement.getElementsByTagName("value").item(0) != null) {
-						propertyValue = eElement.getElementsByTagName("value")
-								.item(0).getTextContent().trim();
-					}
-
-					serverConfigProperties.put(propertyName, propertyValue);
-
-				}
-			}
-
-		} catch (Exception e) {
-			LOG.severe("Load configuration fail. Reason: " + e.toString());
-		}
-
-	}
 	protected long getLongConfig(String key, long defaultValue) {
 		long ret = defaultValue;
 		String retStr = getConfig(key);

http://git-wip-us.apache.org/repos/asf/ranger/blob/5ea8062d/security-admin/src/main/java/org/apache/ranger/common/RangerProperties.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/common/RangerProperties.java b/security-admin/src/main/java/org/apache/ranger/common/RangerProperties.java
index cb8b825..3a5d1c8 100644
--- a/security-admin/src/main/java/org/apache/ranger/common/RangerProperties.java
+++ b/security-admin/src/main/java/org/apache/ranger/common/RangerProperties.java
@@ -19,32 +19,15 @@
 
 package org.apache.ranger.common;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
 import java.util.HashMap;
 
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
+import org.apache.ranger.plugin.util.XMLUtils;
 
-import org.apache.log4j.Logger;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-
-public class RangerProperties extends  HashMap<String,String>  {
+public class RangerProperties extends HashMap<Object, Object> {
 	
 	private static final long serialVersionUID = -4094378755892810987L;
 
-	private final Logger LOG = Logger.getLogger(RangerProperties.class);
-
 	private final String XMLCONFIG_FILENAME_DELIMITOR = ",";
-	private final String XMLCONFIG_PROPERTY_TAGNAME = "property";
-	private final String XMLCONFIG_NAME_TAGNAME = "name";
-	private final String XMLCONFIG_VALUE_TAGNAME = "value";
 
 	private String xmlConfigFileNames = null;
 
@@ -55,106 +38,17 @@ public class RangerProperties extends  HashMap<String,String>  {
 
 	private void initProperties() {
 		
-		if (xmlConfigFileNames == null || xmlConfigFileNames.isEmpty())
+		if (xmlConfigFileNames == null || xmlConfigFileNames.isEmpty()) {
 			return;
+		}
 
-		String[] fnList = xmlConfigFileNames
-				.split(XMLCONFIG_FILENAME_DELIMITOR);
+		String[] fnList = xmlConfigFileNames.split(XMLCONFIG_FILENAME_DELIMITOR);
 
 		for (String fn : fnList) {
-			try {
-				loadXMLConfig(fn);
-			}
-			catch(IOException ioe) {
-				LOG.error("Unable to load configuration from file: [" + fn + "]", ioe);
-			}
+		    XMLUtils.loadConfig(fn, this);
 		}
 
 	}
 
-	private void loadXMLConfig(String fileName) throws IOException {
-
-		try {
-			InputStream in = getFileInputStream(fileName);
-
-			if (in == null) {
-				return;
-			}
-
-			DocumentBuilderFactory xmlDocumentBuilderFactory = DocumentBuilderFactory
-					.newInstance();
-			xmlDocumentBuilderFactory.setIgnoringComments(true);
-			xmlDocumentBuilderFactory.setNamespaceAware(true);
-			DocumentBuilder xmlDocumentBuilder = xmlDocumentBuilderFactory
-					.newDocumentBuilder();
-			Document xmlDocument = xmlDocumentBuilder.parse(in);
-			xmlDocument.getDocumentElement().normalize();
-
-			NodeList nList = xmlDocument.getElementsByTagName(XMLCONFIG_PROPERTY_TAGNAME);
-
-			for (int temp = 0; temp < nList.getLength(); temp++) {
-
-				Node nNode = nList.item(temp);
-
-				if (nNode.getNodeType() == Node.ELEMENT_NODE) {
-
-					Element eElement = (Element) nNode;
-
-					String propertyName = "";
-					String propertyValue = "";
-					
-					if (eElement.getElementsByTagName(XMLCONFIG_NAME_TAGNAME).item(0) != null) {
-						propertyName = eElement.getElementsByTagName(XMLCONFIG_NAME_TAGNAME).item(0).getTextContent().trim();
-					}
-					
-					if (eElement.getElementsByTagName(XMLCONFIG_VALUE_TAGNAME).item(0) != null) {
-						propertyValue = eElement.getElementsByTagName(XMLCONFIG_VALUE_TAGNAME).item(0).getTextContent().trim();
-					}
-					
-					if (get(propertyName) != null)
-						remove(propertyName);
-					
-					if (propertyValue != null)
-						put(propertyName, propertyValue);
-					
-				}
-			}
-		} catch (Throwable t) {
-			throw new IOException(t);
-		}
-	}
-
-	private InputStream getFileInputStream(String path)
-			throws FileNotFoundException {
-
-		InputStream ret = null;
-
-		File f = new File(path);
-
-		if (f.exists()) {
-			ret = new FileInputStream(f);
-		} else {
-			ret = getClass().getResourceAsStream(path);
-
-			if (ret == null) {
-				if (!path.startsWith("/")) {
-					ret = getClass().getResourceAsStream("/" + path);
-				}
-			}
-
-			if (ret == null) {
-				ret = ClassLoader.getSystemClassLoader().getResourceAsStream(
-						path);
-				if (ret == null) {
-					if (!path.startsWith("/")) {
-						ret = ClassLoader.getSystemResourceAsStream("/" + path);
-					}
-				}
-			}
-		}
-
-		return ret;
-	}
-	
 	
 }

http://git-wip-us.apache.org/repos/asf/ranger/blob/5ea8062d/security-admin/src/main/java/org/apache/ranger/common/XMLPropertiesUtil.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/common/XMLPropertiesUtil.java b/security-admin/src/main/java/org/apache/ranger/common/XMLPropertiesUtil.java
index 29fc78a..9c609db 100644
--- a/security-admin/src/main/java/org/apache/ranger/common/XMLPropertiesUtil.java
+++ b/security-admin/src/main/java/org/apache/ranger/common/XMLPropertiesUtil.java
@@ -23,18 +23,10 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.Properties;
 
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-
-import org.apache.log4j.Logger;
+import org.apache.ranger.plugin.util.XMLUtils;
 import org.springframework.util.DefaultPropertiesPersister;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 public class XMLPropertiesUtil extends DefaultPropertiesPersister {
-	private static final Logger logger = Logger.getLogger(XMLPropertiesUtil.class);
 
 	public XMLPropertiesUtil() {
 	}
@@ -42,45 +34,7 @@ public class XMLPropertiesUtil extends DefaultPropertiesPersister {
 	@Override
 	public void loadFromXml(Properties properties, InputStream inputStream)
 			throws IOException {
-		try {
-			DocumentBuilderFactory xmlDocumentBuilderFactory = DocumentBuilderFactory
-					.newInstance();
-			xmlDocumentBuilderFactory.setIgnoringComments(true);
-			xmlDocumentBuilderFactory.setNamespaceAware(true);
-			DocumentBuilder xmlDocumentBuilder = xmlDocumentBuilderFactory
-					.newDocumentBuilder();
-			Document xmlDocument = xmlDocumentBuilder.parse(inputStream);
-			xmlDocument.getDocumentElement().normalize();
-
-			NodeList nList = xmlDocument.getElementsByTagName("property");
-
-			for (int temp = 0; temp < nList.getLength(); temp++) {
-
-				Node nNode = nList.item(temp);
-
-				if (nNode.getNodeType() == Node.ELEMENT_NODE) {
-
-					Element eElement = (Element) nNode;
-
-					String propertyName = "";
-					String propertyValue = "";
-					if (eElement.getElementsByTagName("name").item(0) != null) {
-						propertyName = eElement.getElementsByTagName("name")
-								.item(0).getTextContent().trim();
-					}
-					if (eElement.getElementsByTagName("value").item(0) != null) {
-						propertyValue = eElement.getElementsByTagName("value")
-								.item(0).getTextContent().trim();
-					}
-
-					properties.put(propertyName, propertyValue);
-
-				}
-				//logger.info("ranger site properties loaded successfully.");
-			}
-		} catch (Exception e) {
-			logger.error("Error loading : ", e);
-		}
+        XMLUtils.loadConfig(inputStream, properties);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/ranger/blob/5ea8062d/ugsync/src/main/java/org/apache/ranger/unixusersync/config/UserGroupSyncConfig.java
----------------------------------------------------------------------
diff --git a/ugsync/src/main/java/org/apache/ranger/unixusersync/config/UserGroupSyncConfig.java b/ugsync/src/main/java/org/apache/ranger/unixusersync/config/UserGroupSyncConfig.java
index d839b39..90f9997 100644
--- a/ugsync/src/main/java/org/apache/ranger/unixusersync/config/UserGroupSyncConfig.java
+++ b/ugsync/src/main/java/org/apache/ranger/unixusersync/config/UserGroupSyncConfig.java
@@ -19,11 +19,6 @@
 
 package org.apache.ranger.unixusersync.config;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -31,21 +26,14 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.StringTokenizer;
 
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-
 import org.apache.ranger.credentialapi.CredentialReader;
+import org.apache.ranger.plugin.util.XMLUtils;
 import org.apache.ranger.usergroupsync.UserGroupSink;
 import org.apache.ranger.usergroupsync.UserGroupSource;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 import org.apache.log4j.Logger;
 
 
-
 public class UserGroupSyncConfig  {
 
 	public static final String CONFIG_FILE = "ranger-ugsync-site.xml";
@@ -251,104 +239,9 @@ public class UserGroupSyncConfig  {
 	}
 
 	private void init() {
-		readConfigFile(CORE_SITE_CONFIG_FILE);
-		readConfigFile(CONFIG_FILE);
-		readConfigFile(DEFAULT_CONFIG_FILE);		
-	}
-
-	private void readConfigFile(String fileName) {
-		try {
-			InputStream in = getFileInputStream(fileName);
-			if (in != null) {
-				try {
-					//					prop.load(in);
-					DocumentBuilderFactory xmlDocumentBuilderFactory = DocumentBuilderFactory
-							.newInstance();
-					xmlDocumentBuilderFactory.setIgnoringComments(true);
-					xmlDocumentBuilderFactory.setNamespaceAware(true);
-					DocumentBuilder xmlDocumentBuilder = xmlDocumentBuilderFactory
-							.newDocumentBuilder();
-					Document xmlDocument = xmlDocumentBuilder.parse(in);
-					xmlDocument.getDocumentElement().normalize();
-
-					NodeList nList = xmlDocument
-							.getElementsByTagName("property");
-
-					for (int temp = 0; temp < nList.getLength(); temp++) {
-
-						Node nNode = nList.item(temp);
-
-						if (nNode.getNodeType() == Node.ELEMENT_NODE) {
-
-							Element eElement = (Element) nNode;
-
-							String propertyName = "";
-							String propertyValue = "";
-							if (eElement.getElementsByTagName("name").item(
-									0) != null) {
-								propertyName = eElement
-										.getElementsByTagName("name")
-										.item(0).getTextContent().trim();
-							}
-							if (eElement.getElementsByTagName("value")
-									.item(0) != null) {
-								propertyValue = eElement
-										.getElementsByTagName("value")
-										.item(0).getTextContent().trim();
-							}
-
-							if (prop.get(propertyName) != null) {
-								prop.remove(propertyName);
-							}
-
-							prop.put(propertyName, propertyValue);
-
-						}
-					}
-				}
-				finally {
-					try {
-						in.close();
-					}
-					catch(IOException ioe) {
-						// Ignore IOE when closing stream
-					}
-				}
-			}
-		} catch (Throwable e) {
-			throw new RuntimeException("Unable to load configuration file [" + CONFIG_FILE + "]", e);
-		}
-	}
-
-
-	private InputStream getFileInputStream(String path) throws FileNotFoundException {
-
-		InputStream ret = null;
-
-		File f = new File(path);
-
-		if (f.exists()) {
-			ret = new FileInputStream(f);
-		} else {
-			ret = getClass().getResourceAsStream(path);
-
-			if (ret == null) {
-				if (! path.startsWith("/")) {
-					ret = getClass().getResourceAsStream("/" + path);
-				}
-			}
-
-			if (ret == null) {
-				ret = ClassLoader.getSystemClassLoader().getResourceAsStream(path);
-				if (ret == null) {
-					if (! path.startsWith("/")) {
-						ret = ClassLoader.getSystemResourceAsStream("/" + path);
-					}
-				}
-			}
-		}
-
-		return ret;
+		XMLUtils.loadConfig(CORE_SITE_CONFIG_FILE, prop);
+		XMLUtils.loadConfig(CONFIG_FILE, prop);
+		XMLUtils.loadConfig(DEFAULT_CONFIG_FILE, prop);
 	}
 
 	public String getUserSyncFileSource(){

http://git-wip-us.apache.org/repos/asf/ranger/blob/5ea8062d/unixauthclient/src/main/java/org/apache/ranger/authentication/unix/jaas/RemoteUnixLoginModule.java
----------------------------------------------------------------------
diff --git a/unixauthclient/src/main/java/org/apache/ranger/authentication/unix/jaas/RemoteUnixLoginModule.java b/unixauthclient/src/main/java/org/apache/ranger/authentication/unix/jaas/RemoteUnixLoginModule.java
index 178eb53..ff296b4 100644
--- a/unixauthclient/src/main/java/org/apache/ranger/authentication/unix/jaas/RemoteUnixLoginModule.java
+++ b/unixauthclient/src/main/java/org/apache/ranger/authentication/unix/jaas/RemoteUnixLoginModule.java
@@ -139,71 +139,7 @@ public class RemoteUnixLoginModule implements LoginModule {
 		String val = (String) options.get(REMOTE_UNIX_AUTHENICATION_CONFIG_FILE_PARAM);
 		log("Remote Unix Auth Configuration file [" + val + "]");
 		if (val != null) {
-			InputStream in = null;
-			try {
-				in = getFileInputStream(val);
-				if (in != null) {
-					try {
-						config = new Properties();
-						// config.load(in);
-						DocumentBuilderFactory xmlDocumentBuilderFactory = DocumentBuilderFactory
-								.newInstance();
-						xmlDocumentBuilderFactory.setIgnoringComments(true);
-						xmlDocumentBuilderFactory.setNamespaceAware(true);
-						DocumentBuilder xmlDocumentBuilder = xmlDocumentBuilderFactory
-								.newDocumentBuilder();
-						Document xmlDocument = xmlDocumentBuilder.parse(in);
-						xmlDocument.getDocumentElement().normalize();
-
-						NodeList nList = xmlDocument
-								.getElementsByTagName("property");
-
-						for (int temp = 0; temp < nList.getLength(); temp++) {
-
-							Node nNode = nList.item(temp);
-
-							if (nNode.getNodeType() == Node.ELEMENT_NODE) {
-
-								Element eElement = (Element) nNode;
-
-								String propertyName = "";
-								String propertyValue = "";
-								if (eElement.getElementsByTagName("name").item(
-										0) != null) {
-									propertyName = eElement
-											.getElementsByTagName("name")
-											.item(0).getTextContent().trim();
-								}
-								if (eElement.getElementsByTagName("value")
-										.item(0) != null) {
-									propertyValue = eElement
-											.getElementsByTagName("value")
-											.item(0).getTextContent().trim();
-								}
-
-								config.put(propertyName, propertyValue);
-
-							}
-							logError("ranger site properties loaded successfully.");
-						}
-					} catch (Exception e) {
-						logError("Error loading : " + e);
-
-					}
-					finally {
-						try {
-							in.close();
-						}
-						catch(IOException ioe) {
-							// Ignore IOException when closing streams
-						}
-					}
-				}
-				
-			}
-			catch(Throwable t) {
-				logError("Unable to load REMOTE_UNIX_AUTHENICATION_CONFIG_FILE_PARAM [" + val + "]");
-			}
+			XMLUtils.loadConfig(val, config);
 		}
 		
 		if (config == null) {

http://git-wip-us.apache.org/repos/asf/ranger/blob/5ea8062d/unixauthservice/src/main/java/org/apache/ranger/authentication/UnixAuthenticationService.java
----------------------------------------------------------------------
diff --git a/unixauthservice/src/main/java/org/apache/ranger/authentication/UnixAuthenticationService.java b/unixauthservice/src/main/java/org/apache/ranger/authentication/UnixAuthenticationService.java
index 11e0c92..6ed2174 100644
--- a/unixauthservice/src/main/java/org/apache/ranger/authentication/UnixAuthenticationService.java
+++ b/unixauthservice/src/main/java/org/apache/ranger/authentication/UnixAuthenticationService.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
- package org.apache.ranger.authentication;
+package org.apache.ranger.authentication;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -41,16 +41,11 @@ import javax.net.ssl.SSLServerSocket;
 import javax.net.ssl.SSLServerSocketFactory;
 import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.log4j.Logger;
 import org.apache.ranger.credentialapi.CredentialReader;
+import org.apache.ranger.plugin.util.XMLUtils;
 import org.apache.ranger.usergroupsync.UserGroupSync;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 public class UnixAuthenticationService {
 
@@ -88,9 +83,6 @@ public class UnixAuthenticationService {
 	static private boolean enableUnixAuth = false;
 	
 	private static final String[] UGSYNC_CONFIG_XML_FILES = { "ranger-ugsync-default.xml",  "ranger-ugsync-site.xml" };
-	private static final String    PROPERTY_ELEMENT_TAGNAME = "property";
-	private static final String    NAME_ELEMENT_TAGNAME = "name";
-	private static final String    VALUE_ELEMENT_TAGNAME = "value";
 
 	public static void main(String[] args) {
 		if (args.length > 0) {
@@ -147,60 +139,7 @@ public class UnixAuthenticationService {
 		Properties prop = new Properties();
 		
 		for (String fn : UGSYNC_CONFIG_XML_FILES ) {
-		
-			InputStream in = getFileInputStream(fn);
-	
-			if (in != null) {
-				try {
-					DocumentBuilderFactory xmlDocumentBuilderFactory = DocumentBuilderFactory.newInstance();
-					xmlDocumentBuilderFactory.setIgnoringComments(true);
-					xmlDocumentBuilderFactory.setNamespaceAware(true);
-					DocumentBuilder xmlDocumentBuilder = xmlDocumentBuilderFactory.newDocumentBuilder();
-					Document xmlDocument = xmlDocumentBuilder.parse(in);
-					xmlDocument.getDocumentElement().normalize();
-	
-					NodeList nList = xmlDocument.getElementsByTagName(PROPERTY_ELEMENT_TAGNAME);
-	
-					for (int temp = 0; temp < nList.getLength(); temp++) {
-	
-						Node nNode = nList.item(temp);
-	
-						if (nNode.getNodeType() == Node.ELEMENT_NODE) {
-	
-							Element eElement = (Element) nNode;
-	
-							String propertyName = "";
-							String propertyValue = "";
-							if (eElement.getElementsByTagName(NAME_ELEMENT_TAGNAME).item(
-									0) != null) {
-								propertyName = eElement
-										.getElementsByTagName(NAME_ELEMENT_TAGNAME)
-										.item(0).getTextContent().trim();
-							}
-							if (eElement.getElementsByTagName(VALUE_ELEMENT_TAGNAME)
-									.item(0) != null) {
-								propertyValue = eElement
-										.getElementsByTagName(VALUE_ELEMENT_TAGNAME)
-										.item(0).getTextContent().trim();
-							}
-	
-							//LOG.info("Adding Property:[" + propertyName + "] Value:["+ propertyValue + "]");
-							if (prop.get(propertyName) != null ) {
-								prop.remove(propertyName);
-	 						}
-							prop.put(propertyName, propertyValue);
-						}
-					}
-				}
-				finally {
-					try {
-						in.close();
-					}
-					catch(IOException ioe) {
-						LOG.debug("Close streams failure. Detail: \n", ioe);
-					}
-				}
-			}
+            XMLUtils.loadConfig(fn, prop);
 		}
 		
 		String credStoreFileName = prop.getProperty(CREDSTORE_FILENAME_PARAM);