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

[ranger] branch master updated: RANGER-3194: Ranger Access Audits page not loading

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

mehul 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 6c5336d  RANGER-3194: Ranger Access Audits page not loading
6c5336d is described below

commit 6c5336da05ba64cd993005077034c872b73416a3
Author: Kishor Gollapalliwar <ki...@gmail.com>
AuthorDate: Tue Mar 2 09:07:32 2021 +0530

    RANGER-3194: Ranger Access Audits page not loading
    
    Signed-off-by: Mehul Parikh <me...@apache.org>
---
 .../main/java/org/apache/ranger/solr/SolrMgr.java  | 338 +++++++++--------
 .../main/java/org/apache/ranger/solr/SolrUtil.java |  37 +-
 .../ranger/solr/krb/AbstractKerberosUser.java      | 249 ++++++++++++
 .../ranger/solr/krb/InMemoryJAASConfiguration.java | 416 +++++++++++++++++++++
 .../org/apache/ranger/solr/krb/KerberosAction.java |  91 +++++
 .../ranger/solr/krb/KerberosJAASConfigUser.java    |  78 ++++
 .../org/apache/ranger/solr/krb/KerberosUser.java   |  87 +++++
 7 files changed, 1126 insertions(+), 170 deletions(-)

diff --git a/security-admin/src/main/java/org/apache/ranger/solr/SolrMgr.java b/security-admin/src/main/java/org/apache/ranger/solr/SolrMgr.java
index 2031317..399e5c1 100644
--- a/security-admin/src/main/java/org/apache/ranger/solr/SolrMgr.java
+++ b/security-admin/src/main/java/org/apache/ranger/solr/SolrMgr.java
@@ -19,12 +19,17 @@
 
 package org.apache.ranger.solr;
 
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.*;
 
 import org.apache.log4j.Logger;
-import org.apache.ranger.audit.utils.InMemoryJAASConfiguration;
 import org.apache.ranger.biz.RangerBizUtil;
 import org.apache.ranger.common.PropertiesUtil;
+import org.apache.ranger.solr.krb.InMemoryJAASConfiguration;
+import org.apache.ranger.solr.krb.KerberosAction;
+import org.apache.ranger.solr.krb.KerberosJAASConfigUser;
+import org.apache.ranger.solr.krb.KerberosUser;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.impl.BinaryRequestWriter;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -32,10 +37,13 @@ import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
-import javax.security.auth.login.Configuration;
+import javax.annotation.PreDestroy;
+import javax.security.auth.login.LoginException;
 
 /**
  * This class initializes Solr
@@ -44,155 +52,179 @@ import javax.security.auth.login.Configuration;
 @Component
 public class SolrMgr {
 
-	private static final Logger logger = Logger.getLogger(SolrMgr.class);
-
-	@Autowired
-	RangerBizUtil rangerBizUtil;
-
-	static final Object lock = new Object();
-
-	SolrClient solrClient = null;
-	Date lastConnectTime = null;
-	volatile boolean initDone = false;
-
-	final static String SOLR_URLS_PROP = "ranger.audit.solr.urls";
-	final static String SOLR_ZK_HOSTS = "ranger.audit.solr.zookeepers";
-	final static String SOLR_COLLECTION_NAME = "ranger.audit.solr.collection.name";
-	final static String PROP_JAVA_SECURITY_AUTH_LOGIN_CONFIG   = "java.security.auth.login.config";
-
-	public static final String DEFAULT_COLLECTION_NAME = "ranger_audits";
-
-	public SolrMgr() {
-		init();
-	}
-
-	void connect() {
-		if (!initDone) {
-			synchronized (lock) {
-				if (!initDone) {
-					if ("solr".equalsIgnoreCase(rangerBizUtil.getAuditDBType())) {
-						String zkHosts = PropertiesUtil
-								.getProperty(SOLR_ZK_HOSTS);
-						if (zkHosts == null) {
-							zkHosts = PropertiesUtil
-									.getProperty("ranger.audit.solr.zookeeper");
-						}
-						if (zkHosts == null) {
-							zkHosts = PropertiesUtil
-									.getProperty("ranger.solr.zookeeper");
-						}
-
-						String solrURL = PropertiesUtil
-								.getProperty(SOLR_URLS_PROP);
-
-						if (solrURL == null) {
-							// Try with url
-							solrURL = PropertiesUtil
-									.getProperty("ranger.audit.solr.url");
-						}
-						if (solrURL == null) {
-							// Let's try older property name
-							solrURL = PropertiesUtil
-									.getProperty("ranger.solr.url");
-						}
-
-						if (zkHosts != null && !"".equals(zkHosts.trim())
-								&& !"none".equalsIgnoreCase(zkHosts.trim())) {
-							zkHosts = zkHosts.trim();
-							String collectionName = PropertiesUtil
-									.getProperty(SOLR_COLLECTION_NAME);
-							if (collectionName == null
-									|| "none".equalsIgnoreCase(collectionName)) {
-								collectionName = DEFAULT_COLLECTION_NAME;
-							}
-
-							logger.info("Solr zkHosts=" + zkHosts
-									+ ", collectionName=" + collectionName);
-
-							try {
-								// Instantiate
-								Krb5HttpClientBuilder krbBuild = new Krb5HttpClientBuilder();
-								SolrHttpClientBuilder kb = krbBuild.getBuilder();
-								HttpClientUtil.setHttpClientBuilder(kb);
-								final List<String> zkhosts = new ArrayList<String>(Arrays.asList(zkHosts.split(",")));
-								CloudSolrClient solrCloudClient = new CloudSolrClient.Builder(zkhosts, Optional.empty()).build();
-								solrCloudClient
-										.setDefaultCollection(collectionName);
-								solrClient = solrCloudClient;
-							} catch (Throwable t) {
-								logger.fatal(
-										"Can't connect to Solr server. ZooKeepers="
-												+ zkHosts + ", collection="
-												+ collectionName, t);
-							}
-
-						} else {
-							if (solrURL == null || solrURL.isEmpty()
-									|| "none".equalsIgnoreCase(solrURL)) {
-								logger.fatal("Solr ZKHosts and URL for Audit are empty. Please set property "
-										+ SOLR_ZK_HOSTS
-										+ " or "
-										+ SOLR_URLS_PROP);
-							} else {
-								try {
-									Krb5HttpClientBuilder krbBuild = new Krb5HttpClientBuilder();
-									SolrHttpClientBuilder kb = krbBuild.getBuilder();
-									HttpClientUtil.setHttpClientBuilder(kb);
-									HttpSolrClient.Builder builder = new HttpSolrClient.Builder();
-									builder.withBaseSolrUrl(solrURL);
-									builder.allowCompression(true);
-									builder.withConnectionTimeout(1000);
-									HttpSolrClient httpSolrClient = builder.build();
-									httpSolrClient
-											.setRequestWriter(new BinaryRequestWriter());
-									solrClient = httpSolrClient;
-									initDone = true;
-
-								} catch (Throwable t) {
-									logger.fatal(
-											"Can't connect to Solr server. URL="
-													+ solrURL, t);
-								}
-							}
-						}
-					}
-
-				}
-			}
-		}
-	}
-
-	private void init() {
-		logger.info("==>SolrMgr.init()" );
-		Properties  props = PropertiesUtil.getProps();
-		try {
-			 // SolrJ requires "java.security.auth.login.config"  property to be set to identify itself that it is kerberized. So using a dummy property for it
-			 // Acutal solrclient JAAS configs are read from the ranger-admin-site.xml in ranger admin config folder and set by InMemoryJAASConfiguration
-			 // Refer InMemoryJAASConfiguration doc for JAAS Configuration
-			 if ( System.getProperty(PROP_JAVA_SECURITY_AUTH_LOGIN_CONFIG) == null ) {
-				 System.setProperty(PROP_JAVA_SECURITY_AUTH_LOGIN_CONFIG, "/dev/null");
-			 }
-			 logger.info("Loading SolrClient JAAS config from Ranger audit config if present...");
-			 Configuration conf = InMemoryJAASConfiguration.init(props);
-
-			 if (conf != null) {
-				 Configuration.setConfiguration(conf);
-			 }
-		} catch (Exception e) {
-			logger.error("ERROR: Unable to load SolrClient JAAS config from ranger admin config file. Audit to Kerberized Solr will fail...", e);
-		}
-		logger.info("<==SolrMgr.init()" );
-	}
-
-	public SolrClient getSolrClient() {
-		if(solrClient!=null){
-			return solrClient;
-		}else{
-			synchronized(this){
-				connect();
-			}
-		}
-		return solrClient;
-	}
-
+    private static final Logger logger = Logger.getLogger(SolrMgr.class);
+
+    @Autowired
+    RangerBizUtil rangerBizUtil;
+
+    static final Object lock = new Object();
+
+    SolrClient                    solrClient      = null;
+    Date                          lastConnectTime = null;
+    volatile boolean              initDone        = false;
+    private volatile KerberosUser kerberosUser    = null;
+
+    final static String SOLR_URLS_PROP                       = "ranger.audit.solr.urls";
+    final static String SOLR_ZK_HOSTS                        = "ranger.audit.solr.zookeepers";
+    final static String SOLR_COLLECTION_NAME                 = "ranger.audit.solr.collection.name";
+    final static String PROP_JAVA_SECURITY_AUTH_LOGIN_CONFIG = "java.security.auth.login.config";
+
+    public static final String DEFAULT_COLLECTION_NAME = "ranger_audits";
+
+    public SolrMgr() {
+        init();
+    }
+
+    void connect() {
+        if (!initDone) {
+            synchronized (lock) {
+                if (!initDone) {
+                    if ("solr".equalsIgnoreCase(rangerBizUtil.getAuditDBType())) {
+                        String zkHosts = PropertiesUtil.getProperty(SOLR_ZK_HOSTS);
+                        if (zkHosts == null) {
+                            zkHosts = PropertiesUtil.getProperty("ranger.audit.solr.zookeeper");
+                        }
+                        if (zkHosts == null) {
+                            zkHosts = PropertiesUtil.getProperty("ranger.solr.zookeeper");
+                        }
+
+                        String solrURL = PropertiesUtil.getProperty(SOLR_URLS_PROP);
+
+                        if (solrURL == null) {
+                            // Try with url
+                            solrURL = PropertiesUtil.getProperty("ranger.audit.solr.url");
+                        }
+                        if (solrURL == null) {
+                            // Let's try older property name
+                            solrURL = PropertiesUtil.getProperty("ranger.solr.url");
+                        }
+
+                        if (zkHosts != null && !"".equals(zkHosts.trim()) && !"none".equalsIgnoreCase(zkHosts.trim())) {
+                            zkHosts = zkHosts.trim();
+                            String collectionName = PropertiesUtil.getProperty(SOLR_COLLECTION_NAME);
+                            if (collectionName == null || "none".equalsIgnoreCase(collectionName)) {
+                                collectionName = DEFAULT_COLLECTION_NAME;
+                            }
+
+                            logger.info("Solr zkHosts=" + zkHosts + ", collectionName=" + collectionName);
+
+                            try (Krb5HttpClientBuilder krbBuild = new Krb5HttpClientBuilder()) {
+                                // Instantiate
+                                SolrHttpClientBuilder kb       = krbBuild.getBuilder();
+                                HttpClientUtil.setHttpClientBuilder(kb);
+                                final List<String> zkhosts         = new ArrayList<String>(Arrays.asList(zkHosts.split(",")));
+                                CloudSolrClient    solrCloudClient = new CloudSolrClient.Builder(zkhosts, Optional.empty()).build();
+                                solrCloudClient.setDefaultCollection(collectionName);
+                                solrClient = solrCloudClient;
+                            } catch (Throwable t) {
+                                logger.fatal("Can't connect to Solr server. ZooKeepers=" + zkHosts + ", collection=" + collectionName, t);
+                            }
+
+                        } else {
+                            if (solrURL == null || solrURL.isEmpty() || "none".equalsIgnoreCase(solrURL)) {
+                                logger.fatal("Solr ZKHosts and URL for Audit are empty. Please set property " + SOLR_ZK_HOSTS + " or " + SOLR_URLS_PROP);
+                            } else {
+                                try (Krb5HttpClientBuilder krbBuild = new Krb5HttpClientBuilder()) {
+                                    SolrHttpClientBuilder kb       = krbBuild.getBuilder();
+                                    HttpClientUtil.setHttpClientBuilder(kb);
+                                    HttpSolrClient.Builder builder = new HttpSolrClient.Builder();
+                                    builder.withBaseSolrUrl(solrURL);
+                                    builder.allowCompression(true);
+                                    builder.withConnectionTimeout(1000);
+                                    HttpSolrClient httpSolrClient = builder.build();
+                                    httpSolrClient.setRequestWriter(new BinaryRequestWriter());
+                                    solrClient = httpSolrClient;
+                                    initDone   = true;
+                                } catch (Throwable t) {
+                                    logger.fatal("Can't connect to Solr server. URL=" + solrURL, t);
+                                }
+                            }
+                        }
+                    }
+
+                }
+            }
+        }
+    }
+
+    private void init() {
+        logger.info("==>SolrMgr.init()");
+        Properties props = PropertiesUtil.getProps();
+        try {
+            // SolrJ requires "java.security.auth.login.config"  property to be set to identify itself that it is kerberized. So using a dummy property for it
+            // Acutal solrclient JAAS configs are read from the ranger-admin-site.xml in ranger admin config folder and set by InMemoryJAASConfiguration
+            // Refer InMemoryJAASConfiguration doc for JAAS Configuration
+             if (System.getProperty(PROP_JAVA_SECURITY_AUTH_LOGIN_CONFIG) == null) {
+                 System.setProperty(PROP_JAVA_SECURITY_AUTH_LOGIN_CONFIG, "/dev/null");
+             }
+             logger.info("Loading SolrClient JAAS config from Ranger audit config if present...");
+             InMemoryJAASConfiguration conf = InMemoryJAASConfiguration.init(props);
+
+             KerberosUser kerberosUser = new KerberosJAASConfigUser("Client", conf);
+
+             if (kerberosUser.getPrincipal() != null) {
+                this.kerberosUser = kerberosUser;
+             }
+         } catch (Exception e) {
+             logger.error("ERROR: Unable to load SolrClient JAAS config from ranger admin config file. Audit to Kerberized Solr will fail...", e);
+         }
+         logger.info("<==SolrMgr.init()");
+     }
+
+     public SolrClient getSolrClient() {
+         if (solrClient != null) {
+             return solrClient;
+         } else {
+             synchronized (this) {
+                 connect();
+             }
+         }
+         return solrClient;
+     }
+
+    @PreDestroy
+    public void stop() {
+        logger.info("SolrMgr.stop() called..");
+
+        if (solrClient != null) {
+            try {
+                solrClient.close();
+            } catch (IOException ioe) {
+                logger.error("Error while stopping solr!", ioe);
+            } finally {
+                solrClient = null;
+            }
+        }
+
+        if (kerberosUser != null) {
+            try {
+                kerberosUser.logout();
+            } catch (LoginException excp) {
+                logger.error("Error logging out keytab user", excp);
+            } finally {
+                kerberosUser = null;
+            }
+        }
+    }
+
+    public QueryResponse queryToSolr(final QueryRequest req) throws Exception {
+        final QueryResponse ret;
+
+        try {
+            final PrivilegedExceptionAction<QueryResponse> action = () -> req.process(solrClient);
+
+            if (kerberosUser != null) {
+                // execute the privileged action as the given keytab user
+                final KerberosAction<QueryResponse> kerberosAction = new KerberosAction<QueryResponse>(kerberosUser, action, logger);
+
+                ret = (QueryResponse) kerberosAction.execute();
+            } else {
+                ret = action.run();
+            }
+        } catch (Exception e) {
+            throw e;
+        }
+
+        return ret;
+    }
 }
diff --git a/security-admin/src/main/java/org/apache/ranger/solr/SolrUtil.java b/security-admin/src/main/java/org/apache/ranger/solr/SolrUtil.java
index 3ba8a3e..239698f 100644
--- a/security-admin/src/main/java/org/apache/ranger/solr/SolrUtil.java
+++ b/security-admin/src/main/java/org/apache/ranger/solr/SolrUtil.java
@@ -54,6 +54,9 @@ public class SolrUtil {
 	@Autowired
 	StringUtil stringUtil;
 
+	@Autowired
+	SolrMgr solrMgr;
+
 	SimpleDateFormat dateFormat = new SimpleDateFormat(
 			"yyyy-MM-dd'T'HH:mm:ss'Z'");
 
@@ -69,24 +72,24 @@ public class SolrUtil {
 		}
 	}
 
-	public QueryResponse runQuery(SolrClient solrClient, SolrQuery solrQuery) throws Throwable {
-	    if (solrQuery != null) {
-	        try {
-	            QueryRequest req = new QueryRequest(solrQuery, METHOD.POST);
-	            String username = PropertiesUtil.getProperty("ranger.solr.audit.user");
-	            String password = PropertiesUtil.getProperty("ranger.solr.audit.user.password");
-	            if (username != null && password != null) {
-	                req.setBasicAuthCredentials(username, password);
-	            }
+    public QueryResponse runQuery(SolrClient solrClient, SolrQuery solrQuery) throws Throwable {
+        if (solrQuery != null) {
+            try {
+                QueryRequest req      = new QueryRequest(solrQuery, METHOD.POST);
+                String       username = PropertiesUtil.getProperty("ranger.solr.audit.user");
+                String       password = PropertiesUtil.getProperty("ranger.solr.audit.user.password");
+                if (username != null && password != null) {
+                    req.setBasicAuthCredentials(username, password);
+                }
 
-	            return req.process(solrClient);
-	        } catch (Throwable e) {
-	            logger.error("Error from Solr server. ", e);
-	            throw e;
-	        }
-	    }
-	    return null;
-	}
+                return solrMgr.queryToSolr(req);
+            } catch (Throwable e) {
+                logger.error("Error from Solr server. ", e);
+                throw e;
+            }
+        }
+        return null;
+    }
 
 	public QueryResponse searchResources(SearchCriteria searchCriteria,
 			List<SearchField> searchFields, List<SortField> sortFieldList,
diff --git a/security-admin/src/main/java/org/apache/ranger/solr/krb/AbstractKerberosUser.java b/security-admin/src/main/java/org/apache/ranger/solr/krb/AbstractKerberosUser.java
new file mode 100644
index 0000000..4831f76
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/solr/krb/AbstractKerberosUser.java
@@ -0,0 +1,249 @@
+/*
+ * 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.solr.krb;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public abstract class AbstractKerberosUser implements KerberosUser {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractKerberosUser.class);
+
+    static final String DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss'Z'";
+
+    /**
+     * Percentage of the ticket window to use before we renew the TGT.
+     */
+    static final float TICKET_RENEW_WINDOW = 0.80f;
+
+    protected final AtomicBoolean loggedIn = new AtomicBoolean(false);
+
+    protected Subject subject;
+    protected LoginContext loginContext;
+
+    public AbstractKerberosUser() {
+    }
+
+    /**
+     * Performs a login using the specified principal and keytab.
+     *
+     * @throws LoginException if the login fails
+     */
+    @Override
+    public synchronized void login() throws LoginException {
+        if (isLoggedIn()) {
+            return;
+        }
+
+        try {
+            // If it's the first time ever calling login then we need to initialize a new context
+            if (loginContext == null) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Initializing new login context...");
+                }
+                if (this.subject == null) {
+                    // only create a new subject if a current one does not exist
+                    // other classes may be referencing an existing subject and replacing it may break functionality of those other classes after relogin
+                    this.subject = new Subject();
+                }
+                this.loginContext = createLoginContext(subject);
+            }
+
+            loginContext.login();
+            loggedIn.set(true);
+            LOG.info("Successful login for {}", new Object[]{getPrincipal()});
+        } catch (LoginException le) {
+            LoginException loginException = new LoginException("Unable to login with " + getPrincipal() + " due to: " + le.getMessage());
+            loginException.setStackTrace(le.getStackTrace());
+            throw loginException;
+        }
+    }
+
+    protected abstract LoginContext createLoginContext(final Subject subject) throws LoginException;
+
+    /**
+     * Performs a logout of the current user.
+     *
+     * @throws LoginException if the logout fails
+     */
+    @Override
+    public synchronized void logout() throws LoginException {
+        if (!isLoggedIn()) {
+            return;
+        }
+
+        try {
+            loginContext.logout();
+            loggedIn.set(false);
+            LOG.info("Successful logout for {}", new Object[]{getPrincipal()});
+
+            loginContext = null;
+        } catch (LoginException e) {
+            LOG.warn("Logout failed due to: " + e.getMessage());
+            throw e;
+        }
+    }
+
+    /**
+     * Executes the PrivilegedAction as this user.
+     *
+     * @param action the action to execute
+     * @param <T> the type of result
+     * @return the result of the action
+     * @throws IllegalStateException if this method is called while not logged in
+     */
+    @Override
+    public <T> T doAs(final PrivilegedAction<T> action) throws IllegalStateException {
+        if (!isLoggedIn()) {
+            throw new IllegalStateException("Must login before executing actions");
+        }
+
+        return Subject.doAs(subject, action);
+    }
+
+    /**
+     * Executes the PrivilegedAction as this user.
+     *
+     * @param action the action to execute
+     * @param <T> the type of result
+     * @return the result of the action
+     * @throws IllegalStateException if this method is called while not logged in
+     * @throws PrivilegedActionException if an exception is thrown from the action
+     */
+    @Override
+    public <T> T doAs(final PrivilegedExceptionAction<T> action)
+            throws IllegalStateException, PrivilegedActionException {
+        if (!isLoggedIn()) {
+            throw new IllegalStateException("Must login before executing actions");
+        }
+
+        return Subject.doAs(subject, action);
+    }
+
+    /**
+     * Re-login a user from keytab if TGT is expired or is close to expiry.
+     *
+     * @throws LoginException if an error happens performing the re-login
+     */
+    @Override
+    public synchronized boolean checkTGTAndRelogin() throws LoginException {
+        final KerberosTicket tgt = getTGT();
+        if (tgt == null) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("TGT was not found");
+            }
+        }
+
+        if (tgt != null && System.currentTimeMillis() < getRefreshTime(tgt)) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("TGT was found, but has not reached expiration window");
+            }
+            return false;
+        }
+
+        LOG.info("Performing relogin for {}", new Object[]{getPrincipal()});
+        logout();
+        login();
+        return true;
+    }
+
+    /**
+     * Get the Kerberos TGT.
+     *
+     * @return the user's TGT or null if none was found
+     */
+    private synchronized KerberosTicket getTGT() {
+        final Set<KerberosTicket> tickets = subject.getPrivateCredentials(KerberosTicket.class);
+
+        for (KerberosTicket ticket : tickets) {
+            if (isTGSPrincipal(ticket.getServer())) {
+                return ticket;
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * TGS must have the server principal of the form "krbtgt/FOO@FOO".
+     *
+     * @param principal the principal to check
+     * @return true if the principal is the TGS, false otherwise
+     */
+    private boolean isTGSPrincipal(final KerberosPrincipal principal) {
+        if (principal == null) {
+            return false;
+        }
+
+        if (principal.getName().equals("krbtgt/" + principal.getRealm() + "@" + principal.getRealm())) {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Found TGT principal: " + principal.getName());
+            }
+            return true;
+        }
+
+        return false;
+    }
+
+    private long getRefreshTime(final KerberosTicket tgt) {
+        long start = tgt.getStartTime().getTime();
+        long end = tgt.getEndTime().getTime();
+
+        if (LOG.isTraceEnabled()) {
+            final SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT);
+            final String startDate = dateFormat.format(new Date(start));
+            final String endDate = dateFormat.format(new Date(end));
+            LOG.trace("TGT valid starting at: " + startDate);
+            LOG.trace("TGT expires at: " + endDate);
+        }
+
+        return start + (long) ((end - start) * TICKET_RENEW_WINDOW);
+    }
+
+    /**
+     * @return true if this user is currently logged in, false otherwise
+     */
+    @Override
+    public boolean isLoggedIn() {
+        return loggedIn.get();
+    }
+
+    @Override
+    public String toString() {
+        return "KerberosUser{" +
+                "principal='" + getPrincipal() + '\'' +
+                ", loggedIn=" + loggedIn +
+                '}';
+    }
+}
+
diff --git a/security-admin/src/main/java/org/apache/ranger/solr/krb/InMemoryJAASConfiguration.java b/security-admin/src/main/java/org/apache/ranger/solr/krb/InMemoryJAASConfiguration.java
new file mode 100644
index 0000000..46bf6c1
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/solr/krb/InMemoryJAASConfiguration.java
@@ -0,0 +1,416 @@
+/**
+ * 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.solr.krb;
+
+import org.apache.commons.collections.MapUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.SortedSet;
+import java.util.StringTokenizer;
+import java.util.TreeSet;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+
+/**
+ * InMemoryJAASConfiguration
+ *
+ * An utility class - which has a static method init to load all JAAS configuration from Application properties file (eg: kafka.properties) and
+ * set it as part of the default lookup configuration for all JAAS configuration lookup.
+ *
+ * Example settings in application.properties:
+ *
+ * xasecure.audit.jaas.KafkaClient.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+ * xasecure.audit.jaas.KafkaClient.loginModuleControlFlag = required
+ * xasecure.audit.jaas.KafkaClient.option.useKeyTab = true
+ * xasecure.audit.jaas.KafkaClient.option.storeKey = true
+ * xasecure.audit.jaas.KafkaClient.option.serviceName = kafka
+ * xasecure.audit.jaas.KafkaClient.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+ * xasecure.audit.jaas.KafkaClient.option.principal = kafka-client-1@EXAMPLE.COM
+
+ * xasecure.audit.jaas.MyClient.0.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+ * xasecure.audit.jaas.MyClient.0.loginModuleControlFlag = required
+ * xasecure.audit.jaas.MyClient.0.option.useKeyTab = true
+ * xasecure.audit.jaas.MyClient.0.option.storeKey = true
+ * xasecure.audit.jaas.MyClient.0.option.serviceName = kafka
+ * xasecure.audit.jaas.MyClient.0.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+ * xasecure.audit.jaas.MyClient.0.option.principal = kafka-client-1@EXAMPLE.COM
+ *
+ * xasecure.audit.jaas.MyClient.1.loginModuleName = com.sun.security.auth.module.Krb5LoginModule
+ * xasecure.audit.jaas.MyClient.1.loginModuleControlFlag = optional
+ * xasecure.audit.jaas.MyClient.1.option.useKeyTab = true
+ * xasecure.audit.jaas.MyClient.1.option.storeKey = true
+ * xasecure.audit.jaas.MyClient.1.option.serviceName = kafka
+ * xasecure.audit.jaas.MyClient.1.option.keyTab = /etc/security/keytabs/kafka_client.keytab
+ * xasecure.audit.jaas.MyClient.1.option.principal = kafka-client-1@EXAMPLE.COM
+
+ * This will set the JAAS configuration - equivalent to the jaas.conf file entries:
+ *  KafkaClient {
+ *      com.sun.security.auth.module.Krb5LoginModule required
+ *          useKeyTab=true
+ *          storeKey=true
+ *          serviceName=kafka
+ *          keyTab="/etc/security/keytabs/kafka_client.keytab"
+ *          principal="kafka-client-1@EXAMPLE.COM";
+ *  };
+ *  MyClient {
+ *      com.sun.security.auth.module.Krb5LoginModule required
+ *          useKeyTab=true
+ *          storeKey=true
+ *          serviceName=kafka keyTab="/etc/security/keytabs/kafka_client.keytab"
+ *          principal="kafka-client-1@EXAMPLE.COM";
+ *  };
+ *  MyClient {
+ *      com.sun.security.auth.module.Krb5LoginModule optional
+ *          useKeyTab=true
+ *          storeKey=true
+ *          serviceName=kafka
+ *          keyTab="/etc/security/keytabs/kafka_client.keytab"
+ *          principal="kafka-client-1@EXAMPLE.COM";
+ *  };
+ *
+ *  Here is the syntax for atlas.properties to add JAAS configuration:
+ *
+ *  The property name has to begin with   'xasecure.audit.jaas.' +  clientId (in case of Kafka client,
+ *  it expects the clientId to be  KafkaClient).
+ *  The following property must be there to specify the JAAS loginModule name
+ *          'xasecure.audit.jaas.' +' +  clientId  + '.loginModuleName'
+ *  The following optional property should be set to specify the loginModuleControlFlag
+ *          'xasecure.audit.jaas.' +' + clientId + '.loginModuleControlFlag'
+ *          Default value :  required ,  Possible values:  required, optional, sufficient, requisite
+ *  Then you can add additional optional parameters as options for the configuration using the following
+ *  syntax:
+ *          'xasecure.audit.jaas.' +' + clientId + '.option.' + <optionName>  = <optionValue>
+ *
+ *  The current setup will lookup JAAS configration from the atlas-application.properties first, if not available,
+ *  it will delegate to the original configuration
+ *
+ */
+
+public final class InMemoryJAASConfiguration extends Configuration {
+
+    private static final Logger LOG = LoggerFactory.getLogger(InMemoryJAASConfiguration.class);
+
+    public static final String JAAS_CONFIG_PREFIX_PARAM                    = "xasecure.audit.jaas.";
+    public static final String JAAS_CONFIG_LOGIN_MODULE_NAME_PARAM         = "loginModuleName";
+    public static final String JAAS_CONFIG_LOGIN_MODULE_CONTROL_FLAG_PARAM = "loginModuleControlFlag";
+    public static final String JAAS_CONFIG_LOGIN_OPTIONS_PREFIX            = "option";
+    public static final String JAAS_PRINCIPAL_PROP                         = "principal";
+
+    private final Configuration                            parent;
+    private final Map<String, List<AppConfigurationEntry>> applicationConfigEntryMap = new HashMap<>();
+
+    public static InMemoryJAASConfiguration init(String propFile) throws Exception {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> InMemoryJAASConfiguration.init( {} ) ", propFile);
+        }
+
+        InMemoryJAASConfiguration ret = null;
+        InputStream               in  = null;
+
+        try {
+            Properties properties = new Properties();
+
+            in = ClassLoader.getSystemResourceAsStream(propFile);
+
+            if (in == null) {
+                if (!propFile.startsWith("/")) {
+                    in = ClassLoader.getSystemResourceAsStream("/" + propFile);
+                }
+                if (in == null) {
+                    in = new FileInputStream(new File(propFile));
+                }
+            }
+
+            properties.load(in);
+
+            ret = init(properties);
+        } catch (IOException e) {
+            throw new Exception("Failed to load JAAS application properties", e);
+        } finally {
+            if (in != null) {
+                try {
+                    in.close();
+                } catch (Exception e) {
+                    // Ignore
+                }
+            }
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== InMemoryJAASConfiguration.init( {} ) ", propFile);
+        }
+
+        return ret;
+    }
+
+    public static void init(org.apache.commons.configuration2.Configuration configuration) throws Exception {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> InMemoryJAASConfiguration.init()");
+        }
+
+        if (configuration != null && !configuration.isEmpty()) {
+            Properties properties = new Properties();
+            Iterator<String> iterator = configuration.getKeys();
+            while (iterator.hasNext()) {
+                String key = iterator.next();
+                properties.put(key, configuration.getProperty(key));
+            }
+            init(properties);
+        } else {
+            throw new Exception("Failed to load JAAS application properties: configuration NULL or empty!");
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== InMemoryJAASConfiguration.init()");
+        }
+    }
+
+    public static InMemoryJAASConfiguration init(Properties properties) throws Exception {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> InMemoryJAASConfiguration.init()");
+        }
+
+        InMemoryJAASConfiguration ret = null;
+
+        if (properties != null && MapUtils.isNotEmpty(properties)) {
+            ret = new InMemoryJAASConfiguration(properties);
+        } else {
+            throw new Exception("Failed to load JAAS application properties: properties NULL or empty!");
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== InMemoryJAASConfiguration.init()");
+        }
+
+        return ret;
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> InMemoryJAASConfiguration.getAppConfigurationEntry( {} )", name);
+        }
+
+        AppConfigurationEntry[] ret = null;
+
+        if (parent != null) {
+            ret = parent.getAppConfigurationEntry(name);
+        }
+
+        if (ret == null || ret.length == 0) {
+            List<AppConfigurationEntry> retList = applicationConfigEntryMap.get(name);
+
+            if (retList != null && retList.size() > 0) {
+                ret = retList.toArray(new AppConfigurationEntry[retList.size()]);
+            }
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== InMemoryJAASConfiguration.getAppConfigurationEntry( {} ) : {}", name, toString(ret));
+        }
+
+        return ret;
+    }
+
+    private InMemoryJAASConfiguration(Properties prop) {
+        parent = Configuration.getConfiguration();
+
+        initialize(prop);
+    }
+
+    private void initialize(Properties properties) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> InMemoryJAASConfiguration.initialize()");
+        }
+
+        int                             prefixLen   = JAAS_CONFIG_PREFIX_PARAM.length();
+        Map<String, SortedSet<Integer>> jaasClients = new HashMap<>();
+
+        for(String key : properties.stringPropertyNames()) {
+            if (key.startsWith(JAAS_CONFIG_PREFIX_PARAM)) {
+                String          jaasKey    = key.substring(prefixLen);
+                StringTokenizer tokenizer  = new StringTokenizer(jaasKey, ".");
+                int             tokenCount = tokenizer.countTokens();
+
+                if (tokenCount > 0) {
+                    String             clientId  = tokenizer.nextToken();
+                    SortedSet<Integer> indexList = jaasClients.get(clientId);
+
+                    if (indexList == null) {
+                        indexList = new TreeSet<>();
+
+                        jaasClients.put(clientId, indexList);
+                    }
+
+                    String  indexStr      = tokenizer.nextToken();
+                    int     indexId       = isNumeric(indexStr) ? Integer.parseInt(indexStr)  : -1;
+                    Integer clientIdIndex = Integer.valueOf(indexId);
+
+                    if (!indexList.contains(clientIdIndex)) {
+                        indexList.add(clientIdIndex);
+                    }
+                }
+            }
+        }
+
+        for(String jaasClient : jaasClients.keySet()) {
+            for(Integer index :  jaasClients.get(jaasClient)) {
+                String keyPrefix = JAAS_CONFIG_PREFIX_PARAM + jaasClient + ".";
+
+                if (index > -1) {
+                    keyPrefix = keyPrefix  + String.valueOf(index) + ".";
+                }
+
+                String keyParam        = keyPrefix + JAAS_CONFIG_LOGIN_MODULE_NAME_PARAM;
+                String loginModuleName = properties.getProperty(keyParam);
+
+                if (loginModuleName == null) {
+                    LOG.error("Unable to add JAAS configuration for "
+                            + "client [" + jaasClient + "] as it is missing param [" + keyParam + "]."
+                            + " Skipping JAAS config for [" + jaasClient + "]");
+                    continue;
+                } else {
+                    loginModuleName = loginModuleName.trim();
+                }
+
+                keyParam = keyPrefix + JAAS_CONFIG_LOGIN_MODULE_CONTROL_FLAG_PARAM;
+
+                String controlFlag = properties.getProperty(keyParam);
+
+                AppConfigurationEntry.LoginModuleControlFlag loginControlFlag = null;
+
+                if (controlFlag != null) {
+                    controlFlag = controlFlag.trim().toLowerCase();
+
+                    if (controlFlag.equals("optional")) {
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.OPTIONAL;
+                    } else if (controlFlag.equals("requisite")) {
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUISITE;
+                    } else if (controlFlag.equals("sufficient")) {
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.SUFFICIENT;
+                    } else if (controlFlag.equals("required")) {
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUIRED;
+                    } else {
+                        String validValues = "optional|requisite|sufficient|required";
+                        LOG.warn("Unknown JAAS configuration value for (" + keyParam
+                                + ") = [" + controlFlag + "], valid value are [" + validValues
+                                + "] using the default value, REQUIRED");
+                        loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUIRED;
+                    }
+                } else {
+                    LOG.warn("Unable to find JAAS configuration ("
+                            + keyParam + "); using the default value, REQUIRED");
+                    loginControlFlag = AppConfigurationEntry.LoginModuleControlFlag.REQUIRED;
+                }
+
+                Map<String, String> options         = new HashMap<>();
+                String              optionPrefix    = keyPrefix + JAAS_CONFIG_LOGIN_OPTIONS_PREFIX + ".";
+                int                 optionPrefixLen = optionPrefix.length();
+
+                for(String key : properties.stringPropertyNames()) {
+                    if (key.startsWith(optionPrefix)) {
+                        String optionKey = key.substring(optionPrefixLen);
+                        String optionVal = properties.getProperty(key);
+
+                        if (optionVal != null) {
+                            optionVal = optionVal.trim();
+
+                            try {
+                                if (optionKey.equalsIgnoreCase(JAAS_PRINCIPAL_PROP)) {
+                                    optionVal = SecurityUtil.getServerPrincipal(optionVal, (String) null);
+                                }
+                            } catch (IOException e) {
+                                LOG.warn("Failed to build serverPrincipal. Using provided value:["
+                                        + optionVal + "]");
+                            }
+                        }
+
+                        options.put(optionKey, optionVal);
+                    }
+                }
+
+                AppConfigurationEntry entry = new AppConfigurationEntry(loginModuleName, loginControlFlag, options);
+
+                if (LOG.isDebugEnabled()) {
+                    StringBuilder sb = new StringBuilder();
+
+                    sb.append("Adding client: [").append(jaasClient).append("{").append(index).append("}]\n");
+                    sb.append("\tloginModule: [").append(loginModuleName).append("]\n");
+                    sb.append("\tcontrolFlag: [").append(loginControlFlag).append("]\n");
+
+                    for (String key : options.keySet()) {
+                        String val = options.get(key);
+
+                        sb.append("\tOptions:  [").append(key).append("] => [").append(val).append("]\n");
+                    }
+
+                    LOG.debug(sb.toString());
+                }
+
+                List<AppConfigurationEntry> retList =  applicationConfigEntryMap.get(jaasClient);
+
+                if (retList == null) {
+                    retList = new ArrayList<>();
+
+                    applicationConfigEntryMap.put(jaasClient, retList);
+                }
+
+                retList.add(entry);
+            }
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== InMemoryJAASConfiguration.initialize()");
+        }
+    }
+
+    private static boolean isNumeric(String str) {
+        return str.matches("-?\\d+(\\.\\d+)?");  //match a number with optional '-' and decimal.
+    }
+
+    private String toString(AppConfigurationEntry[] entries) {
+        StringBuilder sb = new StringBuilder();
+
+        sb.append('[');
+        if (entries != null) {
+            for (AppConfigurationEntry entry : entries) {
+                sb.append("{ loginModuleName=").append(entry.getLoginModuleName())
+                        .append(", controlFlag=").append(entry.getControlFlag())
+                        .append(", options=").append(entry.getOptions())
+                        .append("}");
+            }
+        }
+        sb.append(']');
+
+        return sb.toString();
+    }
+}
diff --git a/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosAction.java b/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosAction.java
new file mode 100644
index 0000000..42d500d
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosAction.java
@@ -0,0 +1,91 @@
+/*
+ * 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.solr.krb;
+
+import org.apache.commons.lang3.Validate;
+import org.apache.log4j.Logger;
+
+import javax.security.auth.login.LoginException;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+
+/**
+ * Helper class for processors to perform an action as a KerberosUser.
+ */
+public class KerberosAction<T> {
+
+    private final KerberosUser kerberosUser;
+    private final PrivilegedExceptionAction<T> action;
+    private final Logger logger;
+
+    public KerberosAction(final KerberosUser kerberosUser,
+                          final PrivilegedExceptionAction<T> action,
+                          final Logger logger) {
+        this.kerberosUser = kerberosUser;
+        this.action = action;
+        this.logger = logger;
+        Validate.notNull(this.kerberosUser);
+        Validate.notNull(this.action);
+        Validate.notNull(this.logger);
+    }
+
+    public T execute() throws Exception {
+        T result;
+        // lazily login the first time the processor executes
+        if (!kerberosUser.isLoggedIn()) {
+            try {
+                kerberosUser.login();
+                if (logger != null) logger.info("Successful login for " + kerberosUser.getPrincipal());
+            } catch (LoginException e) {
+                throw new Exception("Login failed due to: " + e.getMessage(), e);
+            }
+        }
+
+        // check if we need to re-login, will only happen if re-login window is reached (80% of TGT life)
+        try {
+            kerberosUser.checkTGTAndRelogin();
+        } catch (LoginException e) {
+            throw new Exception("Relogin check failed due to: " + e.getMessage(), e);
+        }
+
+        // attempt to execute the action, if an exception is caught attempt to logout/login and retry
+        try {
+            result = kerberosUser.doAs(action);
+        } catch (SecurityException se) {
+            if (logger != null) {
+                logger.info("Privileged action failed, attempting relogin and retrying...");
+                logger.debug("", se);
+            }
+
+            try {
+                kerberosUser.logout();
+                kerberosUser.login();
+                result = kerberosUser.doAs(action);
+            } catch (Exception e) {
+                throw new Exception("Retrying privileged action failed due to: " + e.getMessage(), e);
+            }
+        } catch (PrivilegedActionException pae) {
+            final Exception cause = pae.getException();
+            throw new Exception("Privileged action failed due to: " + cause.getMessage(), cause);
+        }
+
+        return result;
+    }
+}
diff --git a/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosJAASConfigUser.java b/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosJAASConfigUser.java
new file mode 100644
index 0000000..115404e
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosJAASConfigUser.java
@@ -0,0 +1,78 @@
+/*
+ * 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.solr.krb;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+
+/**
+ * Used to authenticate and execute actions when Kerberos is enabled and a keytab is being used.
+ *
+ * */
+public class KerberosJAASConfigUser extends AbstractKerberosUser {
+    private static final Logger LOG = LoggerFactory.getLogger(KerberosJAASConfigUser.class);
+
+    private final String        configName;
+    private final Configuration config;
+
+    public KerberosJAASConfigUser(final String configName, final Configuration config) {
+        this.configName = configName;
+        this.config     = config;
+    }
+
+
+    @Override
+    public String getPrincipal() {
+        String                  ret     = null;
+        AppConfigurationEntry[] entries = config.getAppConfigurationEntry(configName);
+
+        if (entries != null) {
+           for (AppConfigurationEntry entry : entries) {
+               if (entry.getOptions().containsKey(InMemoryJAASConfiguration.JAAS_PRINCIPAL_PROP)) {
+                   ret = (String) entry.getOptions().get(InMemoryJAASConfiguration.JAAS_PRINCIPAL_PROP);
+
+                   break;
+               }
+           }
+        }
+
+        return ret;
+    }
+
+    @Override
+    protected LoginContext createLoginContext(Subject subject) throws LoginException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("==> KerberosJAASConfigUser.createLoginContext()");
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("<== KerberosJAASConfigUser.createLoginContext(), Subject: " + subject);
+        }
+
+        return new LoginContext(configName, subject, null, config);
+    }
+}
+
diff --git a/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosUser.java b/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosUser.java
new file mode 100644
index 0000000..4118471
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/solr/krb/KerberosUser.java
@@ -0,0 +1,87 @@
+/*
+ * 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.solr.krb;
+
+import javax.security.auth.login.LoginException;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+
+/**
+ * A keytab-based user that can login/logout and perform actions as the given user.
+ */
+public interface KerberosUser {
+
+    /**
+     * Performs a login for the given user.
+     *
+     * @throws LoginException if the login fails
+     */
+    void login() throws LoginException;
+
+    /**
+     * Performs a logout for the given user.
+     *
+     * @throws LoginException if the logout fails
+     */
+    void logout() throws LoginException;
+
+    /**
+     * Executes the given action as the given user.
+     *
+     * @param action the action to execute
+     * @param <T> the type of response
+     * @return the result of the action
+     * @throws IllegalStateException if attempting to execute an action before performing a login
+     */
+    <T> T doAs(PrivilegedAction<T> action) throws IllegalStateException;
+
+    /**
+     * Executes the given action as the given user.
+     *
+     * @param action the action to execute
+     * @param <T> the type of response
+     * @return the result of the action
+     * @throws IllegalStateException if attempting to execute an action before performing a login
+     * @throws PrivilegedActionException if the action itself threw an exception
+     */
+    <T> T doAs(PrivilegedExceptionAction<T> action)
+            throws IllegalStateException, PrivilegedActionException;
+
+    /**
+     * Performs a re-login if the TGT is close to expiration.
+     *
+     * @return true if a re-login was performed, false otherwise
+     * @throws LoginException if the re-login fails
+     */
+    boolean checkTGTAndRelogin() throws LoginException;
+
+    /**
+     * @return true if this user is currently logged in, false otherwise
+     */
+    boolean isLoggedIn();
+
+    /**
+     * @return the principal for this user
+     */
+    String getPrincipal();
+
+}
+