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 2022/01/02 04:44:45 UTC

[ranger] branch master updated: RANGER-3565: updated RangerRESTClient with option to retry calls to Ranger admin

This is an automated email from the ASF dual-hosted git repository.

madhan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ranger.git


The following commit(s) were added to refs/heads/master by this push:
     new 8235a8e  RANGER-3565: updated RangerRESTClient with option to retry calls to Ranger admin
8235a8e is described below

commit 8235a8e17465dd63658ab370fd6f70ccaee5b736
Author: Madhan Neethiraj <ma...@apache.org>
AuthorDate: Wed Dec 29 16:22:09 2021 -0800

    RANGER-3565: updated RangerRESTClient with option to retry calls to Ranger admin
---
 .../ranger/admin/client/RangerAdminRESTClient.java |   9 +-
 .../ranger/plugin/util/RangerRESTClient.java       | 144 ++++++++++++++++++---
 .../admin/client/RangerAdminJersey2RESTClient.java |  47 ++++++-
 .../unixusersync/config/UserGroupSyncConfig.java   |  27 ++++
 .../process/RangerUgSyncRESTClient.java            |  89 +------------
 5 files changed, 205 insertions(+), 111 deletions(-)

diff --git a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java
index 3283487..cf7e9cf 100644
--- a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java
+++ b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java
@@ -107,6 +107,9 @@ public class RangerAdminRESTClient extends AbstractRangerAdminClient {
 		}
 		int	 restClientConnTimeOutMs	= config.getInt(propertyPrefix + ".policy.rest.client.connection.timeoutMs", 120 * 1000);
 		int	 restClientReadTimeOutMs	= config.getInt(propertyPrefix + ".policy.rest.client.read.timeoutMs", 30 * 1000);
+		int	 restClientMaxRetryAttempts	= config.getInt(propertyPrefix + ".policy.rest.client.max.retry.attempts", 3);
+		int	 restClientRetryIntervalMs	= config.getInt(propertyPrefix + ".policy.rest.client.retry.interval.ms", 1 * 1000);
+
 		supportsPolicyDeltas            = config.getBoolean(propertyPrefix + RangerCommonConstants.PLUGIN_CONFIG_SUFFIX_POLICY_DELTA, RangerCommonConstants.PLUGIN_CONFIG_SUFFIX_POLICY_DELTA_DEFAULT);
 		supportsTagDeltas               = config.getBoolean(propertyPrefix + RangerCommonConstants.PLUGIN_CONFIG_SUFFIX_TAG_DELTA, RangerCommonConstants.PLUGIN_CONFIG_SUFFIX_TAG_DELTA_DEFAULT);
 		isRangerCookieEnabled			= config.getBoolean(propertyPrefix + ".policy.rest.client.cookie.enabled", RangerCommonConstants.POLICY_REST_CLIENT_SESSION_COOKIE_ENABLED);
@@ -119,7 +122,7 @@ public class RangerAdminRESTClient extends AbstractRangerAdminClient {
             url = url.substring(0, url.length() - 1);
         }
 
-		init(url, sslConfigFileName, restClientConnTimeOutMs , restClientReadTimeOutMs, config);
+		init(url, sslConfigFileName, restClientConnTimeOutMs , restClientReadTimeOutMs, restClientMaxRetryAttempts, restClientRetryIntervalMs, config);
 
         try {
             this.serviceNameUrlParam = URLEncoderUtil.encodeURIParam(serviceName);
@@ -655,7 +658,7 @@ public class RangerAdminRESTClient extends AbstractRangerAdminClient {
 		}
 	}
 
-	private void init(String url, String sslConfigFileName, int restClientConnTimeOutMs , int restClientReadTimeOutMs, Configuration config) {
+	private void init(String url, String sslConfigFileName, int restClientConnTimeOutMs , int restClientReadTimeOutMs, int restClientMaxRetryAttempts, int restClientRetryIntervalMs, Configuration config) {
 		if(LOG.isDebugEnabled()) {
 			LOG.debug("==> RangerAdminRESTClient.init(" + url + ", " + sslConfigFileName + ")");
 		}
@@ -663,6 +666,8 @@ public class RangerAdminRESTClient extends AbstractRangerAdminClient {
 		restClient = new RangerRESTClient(url, sslConfigFileName, config);
 		restClient.setRestClientConnTimeOutMs(restClientConnTimeOutMs);
 		restClient.setRestClientReadTimeOutMs(restClientReadTimeOutMs);
+		restClient.setMaxRetryAttempts(restClientMaxRetryAttempts);
+		restClient.setRetryIntervalMs(restClientRetryIntervalMs);
 
 		if(LOG.isDebugEnabled()) {
 			LOG.debug("<== RangerAdminRESTClient.init(" + url + ", " + sslConfigFileName + ")");
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTClient.java b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTClient.java
index 695212d..33f34f3 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTClient.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerRESTClient.java
@@ -105,6 +105,8 @@ public class RangerRESTClient {
 	private Gson   gsonBuilder;
 	private int    mRestClientConnTimeOutMs;
 	private int    mRestClientReadTimeOutMs;
+	private int    maxRetryAttempts;
+	private int    retryIntervalMs;
 	private int    lastKnownActiveUrlIndex;
 
 	private final List<String> configuredURLs;
@@ -154,6 +156,14 @@ public class RangerRESTClient {
 		this.mRestClientReadTimeOutMs = mRestClientReadTimeOutMs;
 	}
 
+	public int getMaxRetryAttempts() { return maxRetryAttempts; }
+
+	public void setMaxRetryAttempts(int maxRetryAttempts) { this.maxRetryAttempts = maxRetryAttempts; }
+
+	public int getRetryIntervalMs() { return retryIntervalMs; }
+
+	public void setRetryIntervalMs(int retryIntervalMs) { this.retryIntervalMs = retryIntervalMs; }
+
 	public void setBasicAuthInfo(String username, String password) {
 		mUsername = username;
 		mPassword = password;
@@ -450,6 +460,7 @@ public class RangerRESTClient {
 		ClientResponse finalResponse = null;
 		int startIndex = this.lastKnownActiveUrlIndex;
 		int currentIndex = 0;
+		int retryAttempt = 0;
 
 		for (int index = 0; index < configuredURLs.size(); index++) {
 			try {
@@ -465,8 +476,11 @@ public class RangerRESTClient {
 					break;
 				}
 			} catch (ClientHandlerException ex) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + configuredURLs.get(currentIndex));
-				processException(index, ex);
+				if (shouldRetry(configuredURLs.get(currentIndex), index, retryAttempt, ex)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
+				}
 			}
 		}
 		return finalResponse;
@@ -476,6 +490,7 @@ public class RangerRESTClient {
 		ClientResponse finalResponse = null;
 		int startIndex = this.lastKnownActiveUrlIndex;
 		int currentIndex = 0;
+		int retryAttempt = 0;
 
 		for (int index = 0; index < configuredURLs.size(); index++) {
 			try {
@@ -491,8 +506,11 @@ public class RangerRESTClient {
 					break;
 				}
 			} catch (ClientHandlerException ex) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : "+configuredURLs.get(currentIndex));
-				processException(index, ex);
+				if (shouldRetry(configuredURLs.get(currentIndex), index, retryAttempt, ex)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
+				}
 			}
 		}
 		return finalResponse;
@@ -502,6 +520,7 @@ public class RangerRESTClient {
 		ClientResponse finalResponse = null;
 		int startIndex = this.lastKnownActiveUrlIndex;
 		int currentIndex = 0;
+		int retryAttempt = 0;
 
 		for (int index = 0; index < configuredURLs.size(); index++) {
 			try {
@@ -515,17 +534,51 @@ public class RangerRESTClient {
 					break;
 				}
 			} catch (ClientHandlerException ex) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + configuredURLs.get(currentIndex));
-				processException(index, ex);
+				if (shouldRetry(configuredURLs.get(currentIndex), index, retryAttempt, ex)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
+				}
 			}
 		}
 		return finalResponse;
 	}
 
+	public ClientResponse post(String relativeURL, Map<String, String> params, Object obj, Cookie sessionId) throws Exception {
+		ClientResponse response = null;
+		int startIndex = this.lastKnownActiveUrlIndex;
+		int currentIndex = 0;
+		int retryAttempt = 0;
+
+		for (int index = 0; index < configuredURLs.size(); index++) {
+			try {
+				currentIndex = (startIndex + index) % configuredURLs.size();
+
+				WebResource webResource = createWebResourceForCookieAuth(currentIndex, relativeURL);
+				webResource = setQueryParams(webResource, params);
+				WebResource.Builder br = webResource.getRequestBuilder().cookie(sessionId);
+				response = br.accept(RangerRESTUtils.REST_EXPECTED_MIME_TYPE).type(RangerRESTUtils.REST_MIME_TYPE_JSON)
+						.post(ClientResponse.class, toJson(obj));
+				if (response != null) {
+					setLastKnownActiveUrlIndex(currentIndex);
+					break;
+				}
+			} catch (ClientHandlerException ex) {
+				if (shouldRetry(configuredURLs.get(currentIndex), index, retryAttempt, ex)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
+				}
+			}
+		}
+		return response;
+	}
+
 	public ClientResponse delete(String relativeUrl, Map<String, String> params) throws Exception {
 		ClientResponse finalResponse = null;
 		int startIndex = this.lastKnownActiveUrlIndex;
 		int currentIndex = 0;
+		int retryAttempt = 0;
 
 		for (int index = 0; index < configuredURLs.size(); index++) {
 			try {
@@ -540,17 +593,51 @@ public class RangerRESTClient {
 					break;
 				}
 			} catch (ClientHandlerException ex) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + configuredURLs.get(currentIndex));
-				processException(index, ex);
+				if (shouldRetry(configuredURLs.get(currentIndex), index, retryAttempt, ex)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
+				}
 			}
 		}
 		return finalResponse;
 	}
 
+	public ClientResponse delete(String relativeURL, Map<String, String> params, Cookie sessionId) throws Exception {
+		ClientResponse response = null;
+		int startIndex = this.lastKnownActiveUrlIndex;
+		int currentIndex = 0;
+		int retryAttempt = 0;
+
+		for (int index = 0; index < configuredURLs.size(); index++) {
+			try {
+				currentIndex = (startIndex + index) % configuredURLs.size();
+
+				WebResource webResource = createWebResourceForCookieAuth(currentIndex, relativeURL);
+				webResource = setQueryParams(webResource, params);
+				WebResource.Builder br = webResource.getRequestBuilder().cookie(sessionId);
+				response = br.delete(ClientResponse.class);
+				if (response != null) {
+					setLastKnownActiveUrlIndex(currentIndex);
+					break;
+				}
+			} catch (ClientHandlerException ex) {
+				if (shouldRetry(configuredURLs.get(currentIndex), index, retryAttempt, ex)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
+				}
+			}
+		}
+		return response;
+	}
+
 	public ClientResponse put(String relativeUrl, Map<String, String> params, Object obj) throws Exception {
 		ClientResponse finalResponse = null;
 		int startIndex = this.lastKnownActiveUrlIndex;
 		int currentIndex = 0;
+		int retryAttempt = 0;
+
 		for (int index = 0; index < configuredURLs.size(); index++) {
 			try {
 				currentIndex = (startIndex + index) % configuredURLs.size();
@@ -563,8 +650,11 @@ public class RangerRESTClient {
 					break;
 				}
 			} catch (ClientHandlerException ex) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + configuredURLs.get(currentIndex));
-				processException(index, ex);
+				if (shouldRetry(configuredURLs.get(currentIndex), index, retryAttempt, ex)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
+				}
 			}
 		}
 		return finalResponse;
@@ -574,6 +664,7 @@ public class RangerRESTClient {
 		ClientResponse response = null;
 		int startIndex = this.lastKnownActiveUrlIndex;
 		int currentIndex = 0;
+		int retryAttempt = 0;
 
 		for (int index = 0; index < configuredURLs.size(); index++) {
 			try {
@@ -587,9 +678,12 @@ public class RangerRESTClient {
 					setLastKnownActiveUrlIndex(currentIndex);
 					break;
 				}
-			} catch (ClientHandlerException e) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + configuredURLs.get(currentIndex));
-				processException(index, e);
+			} catch (ClientHandlerException ex) {
+				if (shouldRetry(configuredURLs.get(currentIndex), index, retryAttempt, ex)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
+				}
 			}
 		}
 		return response;
@@ -617,11 +711,29 @@ public class RangerRESTClient {
 		return ret;
 	}
 
-	protected void processException(int index, ClientHandlerException e) throws Exception {
-		if (index == configuredURLs.size() - 1) {
+	protected boolean shouldRetry(String currentUrl, int index, int retryAttemptCount, Exception ex) throws Exception {
+		LOG.warn("Failed to communicate with Ranger Admin. URL: " + currentUrl + ". Error: " + ex.getMessage());
+
+		boolean isLastUrl = index == (configuredURLs.size() - 1);
+
+		// attempt retry after failure on the last url
+		boolean ret = isLastUrl && (retryAttemptCount < maxRetryAttempts);
+
+		if (ret) {
+			LOG.warn("Waiting for " + retryIntervalMs + "ms before retry attempt #" + (retryAttemptCount + 1));
+
+			try {
+				Thread.sleep(retryIntervalMs);
+			} catch (InterruptedException excp) {
+				LOG.error("Failed while waiting to retry", excp);
+			}
+		} else if (isLastUrl) {
 			LOG.error("Failed to communicate with all Ranger Admin's URL's : [ " + configuredURLs + " ]");
-			throw e;
+
+			throw ex;
 		}
+
+		return ret;
 	}
 
 	public int getLastKnownActiveUrlIndex() {
diff --git a/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java b/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java
index 0316621..1df0d73 100644
--- a/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java
+++ b/knox-agent/src/main/java/org/apache/ranger/admin/client/RangerAdminJersey2RESTClient.java
@@ -79,6 +79,8 @@ public class RangerAdminJersey2RESTClient extends AbstractRangerAdminClient {
 	String _pluginId = null;
 	int	   _restClientConnTimeOutMs;
 	int	   _restClientReadTimeOutMs;
+	int	   _restClientMaxRetryAttempts;
+	int	   _restClientRetryIntervalMs;
 	private int lastKnownActiveUrlIndex;
 	private List<String> configURLs;
 	private boolean			 isRangerCookieEnabled;
@@ -107,6 +109,9 @@ public class RangerAdminJersey2RESTClient extends AbstractRangerAdminClient {
 		_sslConfigFileName 		 = config.get(configPropertyPrefix + ".policy.rest.ssl.config.file");
 		_restClientConnTimeOutMs = config.getInt(configPropertyPrefix + ".policy.rest.client.connection.timeoutMs", 120 * 1000);
 		_restClientReadTimeOutMs = config.getInt(configPropertyPrefix + ".policy.rest.client.read.timeoutMs", 30 * 1000);
+		_restClientMaxRetryAttempts	= config.getInt(configPropertyPrefix + ".policy.rest.client.max.retry.attempts", 3);
+		_restClientRetryIntervalMs	= config.getInt(configPropertyPrefix + ".policy.rest.client.retry.interval.ms", 1 * 1000);
+
 		_clusterName             = config.get(configPropertyPrefix + ".access.cluster.name", "");
 		if(StringUtil.isEmpty(_clusterName)){
 			_clusterName =config.get(configPropertyPrefix + ".ambari.cluster.name", "");
@@ -332,6 +337,7 @@ public class RangerAdminJersey2RESTClient extends AbstractRangerAdminClient {
 		Response response = null;
 		int startIndex = this.lastKnownActiveUrlIndex;
         int currentIndex = 0;
+        int retryAttempt = 0;
 
 		for (int index = 0; index < configURLs.size(); index++) {
 			try {
@@ -344,10 +350,10 @@ public class RangerAdminJersey2RESTClient extends AbstractRangerAdminClient {
 					break;
 				}
 			} catch (ProcessingException e) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + configURLs.get(currentIndex));
-				if (index == configURLs.size() - 1) {
-					throw new ClientHandlerException(
-							"Failed to communicate with all Ranger Admin's URL's : [ " + configURLs + " ]", e);
+				if (shouldRetry(configURLs.get(currentIndex), index, retryAttempt, e)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
 				}
 			}
 		}
@@ -358,6 +364,7 @@ public class RangerAdminJersey2RESTClient extends AbstractRangerAdminClient {
 		Response response = null;
 		int startIndex = this.lastKnownActiveUrlIndex;
 		int currentIndex = 0;
+		int retryAttempt = 0;
 
 		for (int index = 0; index < configURLs.size(); index++) {
 			try {
@@ -372,9 +379,10 @@ public class RangerAdminJersey2RESTClient extends AbstractRangerAdminClient {
 					break;
 				}
 			} catch (ProcessingException e) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : "+configURLs.get(currentIndex));
-				if(index == configURLs.size()-1) {
-					throw new ProcessingException("Failed to communicate with all Ranger Admin's URL : [ "+ configURLs+" ]", e);
+				if (shouldRetry(configURLs.get(currentIndex), index, retryAttempt, e)) {
+					retryAttempt++;
+
+					index = -1; // start from first url
 				}
 			}
 		}
@@ -1065,4 +1073,29 @@ public class RangerAdminJersey2RESTClient extends AbstractRangerAdminClient {
 			isValidRoleDownloadSessionCookie = (roleDownloadSessionId != null);
 		}
 	}
+
+	protected boolean shouldRetry(String currentUrl, int index, int retryAttemptCount, ProcessingException ex) {
+		LOG.warn("Failed to communicate with Ranger Admin. URL: " + currentUrl + ". Error: " + ex.getMessage());
+
+		boolean isLastUrl = index == (configURLs.size() - 1);
+
+		// attempt retry after failure on the last url
+		boolean ret = isLastUrl && (retryAttemptCount < _restClientMaxRetryAttempts);
+
+		if (ret) {
+			LOG.warn("Waiting for " + _restClientRetryIntervalMs + "ms before retry attempt #" + (retryAttemptCount + 1));
+
+			try {
+				Thread.sleep(_restClientRetryIntervalMs);
+			} catch (InterruptedException excp) {
+				LOG.error("Failed while waiting to retry", excp);
+			}
+		} else if (isLastUrl) {
+			LOG.error("Failed to communicate with all Ranger Admin's URL's : [ " + configURLs + " ]");
+
+			throw new ClientHandlerException("Failed to communicate with all Ranger Admin's URL : [ "+ configURLs+" ]", ex);
+		}
+
+		return ret;
+	}
 }
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 ce9d27f..ea6b47c 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
@@ -233,6 +233,9 @@ public class UserGroupSyncConfig  {
 
 	private static final String SYNC_POLICY_MGR_USERNAME = "ranger.usersync.policymgr.username";
 
+	private static final String SYNC_POLICY_MGR_MAX_RETRY_ATTEMPTS = "ranger.usersync.policymgr.max.retry.attempts";
+	private static final String SYNC_POLICY_MGR_RETRY_INTERVAL_MS  = "ranger.usersync.policymgr.retry.interval.ms";
+
 	private static final String DEFAULT_POLICYMGR_USERNAME = "rangerusersync";
 
 	private static final String SYNC_SOURCE = "ranger.usersync.sync.source";
@@ -955,6 +958,14 @@ public class UserGroupSyncConfig  {
 		return userName;
 	}
 
+	public int getPolicyMgrMaxRetryAttempts() {
+		return getIntProperty(prop, SYNC_POLICY_MGR_MAX_RETRY_ATTEMPTS, 0);
+	}
+
+	public int getPolicyMgrRetryIntervalMs() {
+		return getIntProperty(prop, SYNC_POLICY_MGR_RETRY_INTERVAL_MS, 1 * 1000);
+	}
+
 	public String getSyncSource() {
 		String syncSource=null;
 		if(prop!=null && prop.containsKey(SYNC_SOURCE)){
@@ -1283,4 +1294,20 @@ public class UserGroupSyncConfig  {
 		}
 		return isUserSyncNameValidationEnabled;
 	}
+
+
+	private int getIntProperty(Properties prop, String key, int defaultValue) {
+		int   ret  = defaultValue;
+		String val = prop.getProperty(key);
+
+		if (StringUtils.isNotBlank(val)) {
+			try {
+				ret = Integer.parseInt(val);
+			} catch (NumberFormatException excp) {
+				LOG.warn("Invalid value for property: " + key + "=" + val + ". Will use default value: " + defaultValue, excp);
+			}
+		}
+
+		return ret;
+	}
 }
diff --git a/ugsync/src/main/java/org/apache/ranger/unixusersync/process/RangerUgSyncRESTClient.java b/ugsync/src/main/java/org/apache/ranger/unixusersync/process/RangerUgSyncRESTClient.java
index 991d71f..22162b0 100644
--- a/ugsync/src/main/java/org/apache/ranger/unixusersync/process/RangerUgSyncRESTClient.java
+++ b/ugsync/src/main/java/org/apache/ranger/unixusersync/process/RangerUgSyncRESTClient.java
@@ -19,37 +19,25 @@
 
 package org.apache.ranger.unixusersync.process;
 
-import java.util.Map;
-
 import javax.net.ssl.HostnameVerifier;
 import javax.net.ssl.KeyManager;
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLSession;
 import javax.net.ssl.TrustManager;
-import javax.ws.rs.core.Cookie;
 
 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 org.apache.ranger.plugin.util.RangerRESTClient;
-import org.apache.ranger.plugin.util.RangerRESTUtils;
 import org.apache.ranger.unixusersync.config.UserGroupSyncConfig;
 import org.codehaus.jackson.jaxrs.JacksonJsonProvider;
 
 import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientHandlerException;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.client.config.ClientConfig;
 import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.api.client.filter.HTTPBasicAuthFilter;
 import com.sun.jersey.client.urlconnection.HTTPSProperties;
 
 public class RangerUgSyncRESTClient extends RangerRESTClient {
-
-	private static final Log LOG = LogFactory.getLog(RangerUgSyncRESTClient.class);
-
 	private String AUTH_KERBEROS = "kerberos";
 
 	public RangerUgSyncRESTClient(String policyMgrBaseUrls, String ugKeyStoreFile, String ugKeyStoreFilepwd,
@@ -83,81 +71,10 @@ public class RangerUgSyncRESTClient extends RangerRESTClient {
 				getClient().addFilter(new HTTPBasicAuthFilter(getUsername(), getPassword()));
 			}
 		}
-	}
-
-	public ClientResponse get(String relativeURL, Map<String, String> params, Cookie sessionId) throws Exception {
-		ClientResponse response = null;
-		int startIndex = getLastKnownActiveUrlIndex();
-		int currentIndex = 0;
-
-		for (int index = 0; index < getConfiguredURLs().size(); index++) {
-			try {
-				currentIndex = (startIndex + index) % getConfiguredURLs().size();
-
-				WebResource webResource = createWebResourceForCookieAuth(currentIndex, relativeURL);
-				webResource = setQueryParams(webResource, params);
-				WebResource.Builder br = webResource.getRequestBuilder().cookie(sessionId);
-				response = br.accept(RangerRESTUtils.REST_EXPECTED_MIME_TYPE).get(ClientResponse.class);
-				if (response != null) {
-					setLastKnownActiveUrlIndex(currentIndex);
-					break;
-				}
-			} catch (ClientHandlerException e) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + getConfiguredURLs().get(currentIndex));
-				processException(index, e);
-			}
-		}
-		return response;
-	}
 
-	public ClientResponse post(String relativeURL, Map<String, String> params, Object obj, Cookie sessionId)
-			throws Exception {
-		ClientResponse response = null;
-		int startIndex = getLastKnownActiveUrlIndex();
-		int currentIndex = 0;
+		UserGroupSyncConfig config = UserGroupSyncConfig.getInstance();
 
-		for (int index = 0; index < getConfiguredURLs().size(); index++) {
-			try {
-				currentIndex = (startIndex + index) % getConfiguredURLs().size();
-
-				WebResource webResource = createWebResourceForCookieAuth(currentIndex, relativeURL);
-				webResource = setQueryParams(webResource, params);
-				WebResource.Builder br = webResource.getRequestBuilder().cookie(sessionId);
-				response = br.accept(RangerRESTUtils.REST_EXPECTED_MIME_TYPE).type(RangerRESTUtils.REST_MIME_TYPE_JSON)
-						.post(ClientResponse.class, toJson(obj));
-				if (response != null) {
-					setLastKnownActiveUrlIndex(currentIndex);
-					break;
-				}
-			} catch (ClientHandlerException e) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + getConfiguredURLs().get(currentIndex));
-				processException(index, e);
-			}
-		}
-		return response;
-	}
-
-	public ClientResponse delete(String relativeURL, Map<String, String> params, Cookie sessionId) throws Exception {
-		ClientResponse response = null;
-		int startIndex = getLastKnownActiveUrlIndex();
-		int currentIndex = 0;
-		for (int index = 0; index < getConfiguredURLs().size(); index++) {
-			try {
-				currentIndex = (startIndex + index) % getConfiguredURLs().size();
-
-				WebResource webResource = createWebResourceForCookieAuth(currentIndex, relativeURL);
-				webResource = setQueryParams(webResource, params);
-				WebResource.Builder br = webResource.getRequestBuilder().cookie(sessionId);
-				response = br.delete(ClientResponse.class);
-				if (response != null) {
-					setLastKnownActiveUrlIndex(currentIndex);
-					break;
-				}
-			} catch (ClientHandlerException e) {
-				LOG.warn("Failed to communicate with Ranger Admin, URL : " + getConfiguredURLs().get(currentIndex));
-				processException(index, e);
-			}
-		}
-		return response;
+		super.setMaxRetryAttempts(config.getPolicyMgrMaxRetryAttempts());
+		super.setRetryIntervalMs(config.getPolicyMgrRetryIntervalMs());
 	}
 }