You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by br...@apache.org on 2014/09/11 04:12:24 UTC

git commit: SENTRY-428 - Sentry service should periodically renew the server kerberos ticket (Prasad via Brock)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master f7237c807 -> 70c47e803


SENTRY-428 - Sentry service should periodically renew the server kerberos ticket (Prasad via Brock)


Project: http://git-wip-us.apache.org/repos/asf/incubator-sentry/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-sentry/commit/70c47e80
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/70c47e80
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/70c47e80

Branch: refs/heads/master
Commit: 70c47e803512efbec51dc5ec5c5d725a5dfa04c3
Parents: f7237c8
Author: Brock Noland <br...@apache.org>
Authored: Wed Sep 10 19:12:17 2014 -0700
Committer: Brock Noland <br...@apache.org>
Committed: Wed Sep 10 19:12:17 2014 -0700

----------------------------------------------------------------------
 .../service/thrift/SentryKerberosContext.java   | 148 +++++++++++++++++++
 .../sentry/service/thrift/SentryService.java    |  29 +---
 .../service/thrift/SentryMiniKdcTestcase.java   |  70 +++++++++
 .../thrift/TestConnectionWithTicketTimeout.java |  45 ++++++
 .../thrift/TestSentryServiceWithKerberos.java   |   6 +-
 .../thrift/SentryServiceIntegrationBase.java    |  10 +-
 6 files changed, 285 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/70c47e80/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryKerberosContext.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryKerberosContext.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryKerberosContext.java
new file mode 100644
index 0000000..fc7bc05
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryKerberosContext.java
@@ -0,0 +1,148 @@
+/**
+ * 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.sentry.service.thrift;
+
+import java.io.File;
+import java.util.HashSet;
+import java.util.Set;
+
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Sets;
+
+public class SentryKerberosContext implements Runnable {
+  private static final float TICKET_RENEW_WINDOW = 0.80f;
+  private static final Logger LOGGER = LoggerFactory
+      .getLogger(SentryKerberosContext.class);
+  private LoginContext loginContext;
+  private Subject subject;
+  private final javax.security.auth.login.Configuration kerberosConfig;
+  private Thread renewerThread;
+  private boolean shutDownRenewer = false;
+
+  public SentryKerberosContext(String principal, String keyTab, boolean autoRenewTicket)
+      throws LoginException {
+    subject = new Subject(false, Sets.newHashSet(new KerberosPrincipal(principal)),
+          new HashSet<Object>(), new HashSet<Object>());
+    kerberosConfig = KerberosConfiguration.createClientConfig(principal, new File(keyTab));
+    loginWithNewContext();
+    if (autoRenewTicket) {
+      startRenewerThread();
+    }
+  }
+
+  public void loginWithNewContext() throws LoginException {
+    logoutSubject();
+    loginContext = new LoginContext("", subject, null, kerberosConfig);
+    loginContext.login();
+    subject = loginContext.getSubject();
+  }
+  
+  private void logoutSubject() {
+    if (loginContext != null) {
+      try {
+        loginContext.logout();
+      } catch (LoginException e) {
+        LOGGER.warn("Error logging out the subject", e);
+      }        
+    }
+    loginContext = null;
+  }
+  
+  public Subject getSubject() {
+    return subject;
+  }
+
+  /**
+   * Get the Kerberos TGT
+   * @return the user's TGT or null if none was found
+   */
+  private KerberosTicket getTGT() {
+    Set<KerberosTicket> tickets = subject.getPrivateCredentials(KerberosTicket.class);
+    for(KerberosTicket ticket: tickets) {
+      KerberosPrincipal server = ticket.getServer();
+      if (server.getName().equals("krbtgt/" + server.getRealm() +
+          "@" + server.getRealm())) {
+        return ticket;
+      }
+    }
+    return null;
+  }
+  
+  private long getRefreshTime(KerberosTicket tgt) {
+    long start = tgt.getStartTime().getTime();
+    long end = tgt.getEndTime().getTime();
+    return start + (long) ((end - start) * TICKET_RENEW_WINDOW);
+  }
+  
+  /***
+   * Ticket renewer thread
+   * wait till 80% time interval left on the ticket and then renew it
+   */
+  @Override
+  public void run() {
+    try {
+      LOGGER.info("Sentry Ticket renewer thread started");
+      while (!shutDownRenewer) {
+        KerberosTicket tgt = getTGT();
+        long nextRefresh = getRefreshTime(tgt);
+        if (tgt == null) {
+          LOGGER.warn("No ticket found in the cache");
+          return;
+        }
+        while (System.currentTimeMillis() < nextRefresh) {
+          Thread.sleep(1000);
+          if (shutDownRenewer) {
+            return;
+          }
+        }
+        loginWithNewContext();
+        LOGGER.debug("Renewed ticket");
+      }
+    } catch (InterruptedException e1) {
+      LOGGER.warn("Sentry Ticket renewer thread interrupted", e1);
+      return;
+    } catch (LoginException e) {
+      LOGGER.warn("Failed to renew ticket", e);
+    } finally {
+      logoutSubject();
+      LOGGER.info("Sentry Ticket renewer thread finished");
+    }
+  }
+
+  public void startRenewerThread() {
+    renewerThread = new Thread(this);
+    renewerThread.start();
+  }
+
+  public void shutDown() throws LoginException {
+    if (renewerThread != null) {
+      shutDownRenewer = true;
+    } else {
+      logoutSubject();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/70c47e80/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
index 33e51de..6843e80 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
@@ -33,9 +33,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.ThreadFactory;
 
 import javax.security.auth.Subject;
-import javax.security.auth.kerberos.KerberosPrincipal;
-import javax.security.auth.login.LoginContext;
-import javax.security.auth.login.LoginException;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -58,6 +55,7 @@ import org.apache.thrift.transport.TSaslServerTransport;
 import org.apache.thrift.transport.TServerSocket;
 import org.apache.thrift.transport.TServerTransport;
 import org.apache.thrift.transport.TTransportFactory;
+import org.mortbay.log.Log;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,7 +81,6 @@ public class SentryService implements Callable {
   private final String keytab;
   private final ExecutorService serviceExecutor;
   private Future future;
-
   private TServer thriftServer;
   private Status status;
 
@@ -141,17 +138,11 @@ public class SentryService implements Callable {
 
   @Override
   public String call() throws Exception {
-    LoginContext loginContext = null;
+    SentryKerberosContext kerberosContext = null;
     try {
       if (kerberos) {
-        Subject subject = new Subject(false,
-            Sets.newHashSet(new KerberosPrincipal(principal)),
-            new HashSet<Object>(), new HashSet<Object>());
-        loginContext = new LoginContext("", subject, null,
-            KerberosConfiguration.createClientConfig(principal, new File(keytab)));
-        loginContext.login();
-        subject = loginContext.getSubject();
-        Subject.doAs(subject, new PrivilegedExceptionAction<Void>() {
+        kerberosContext = new SentryKerberosContext(principal, keytab, true);
+        Subject.doAs(kerberosContext.getSubject(), new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
             runServer();
@@ -164,15 +155,11 @@ public class SentryService implements Callable {
     } catch (Exception t) {
       LOGGER.error("Error starting server", t);
       throw new Exception("Error starting server", t);
-    }finally {
-      status = Status.NOT_STARTED;
-      if (loginContext != null) {
-        try {
-          loginContext.logout();
-        } catch (LoginException e) {
-          LOGGER.error("Error logging out", e);
-        }
+    } finally {
+      if (kerberosContext != null) {
+        kerberosContext.shutDown();
       }
+      status = Status.NOT_STARTED;
     }
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/70c47e80/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/SentryMiniKdcTestcase.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/SentryMiniKdcTestcase.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/SentryMiniKdcTestcase.java
new file mode 100644
index 0000000..79acb58
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/SentryMiniKdcTestcase.java
@@ -0,0 +1,70 @@
+/**
+ * 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.sentry.provider.db.service.thrift;
+
+import java.io.File;
+import java.util.Properties;
+
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.junit.After;
+
+public class SentryMiniKdcTestcase {
+
+  private File workDir;
+  private Properties conf;
+  private MiniKdc kdc;
+
+  public void startMiniKdc(Properties confOverlay) throws Exception {
+    createTestDir();
+    createMiniKdcConf(confOverlay);
+    kdc = new MiniKdc(conf, workDir);
+    kdc.start();
+  }
+
+  private void createMiniKdcConf(Properties confOverlay) {
+    conf = MiniKdc.createConf();
+    for ( Object property : confOverlay.keySet()) {
+      conf.put(property, confOverlay.get(property));
+    }
+  }
+
+  private void createTestDir() {
+    workDir = new File(System.getProperty("test.dir", "target"));
+  }
+
+  @After
+  public void stopMiniKdc() {
+    if (kdc != null) {
+      kdc.stop();
+    }
+  }
+
+  public MiniKdc getKdc() {
+    return kdc;
+  }
+
+  public File getWorkDir() {
+    return workDir;
+  }
+
+  public Properties getConf() {
+    return conf;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/70c47e80/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestConnectionWithTicketTimeout.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestConnectionWithTicketTimeout.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestConnectionWithTicketTimeout.java
new file mode 100644
index 0000000..6c82304
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestConnectionWithTicketTimeout.java
@@ -0,0 +1,45 @@
+/**
+ * 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.sentry.provider.db.service.thrift;
+
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.junit.Test;
+
+public class TestConnectionWithTicketTimeout extends
+    org.apache.sentry.service.thrift.SentryServiceIntegrationBase {
+
+
+  @Override
+  public void beforeSetup() throws Exception {
+    kdcConfOverlay.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "300001");
+  }
+
+  /***
+   * Test is run only when sentry.hive.test.ticket.timeout is set to "true"
+   * @throws Exception
+   */
+  @Test
+  public void testConnectionAfterTicketTimeout() throws Exception {
+    if ("true".equalsIgnoreCase(System.getProperty("sentry.hive.test.ticket.timeout", "false"))) {
+      Thread.sleep(400000);
+      connectToSentryService();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/70c47e80/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithKerberos.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithKerberos.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithKerberos.java
index 3209ccf..ea666f1 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithKerberos.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceWithKerberos.java
@@ -17,8 +17,11 @@
  */
 package org.apache.sentry.provider.db.service.thrift;
 
-import org.apache.sentry.service.thrift.SentryServiceIntegrationBase;
+import java.util.Properties;
 
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.sentry.service.thrift.SentryServiceIntegrationBase;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -44,4 +47,5 @@ public class TestSentryServiceWithKerberos extends SentryServiceIntegrationBase
     // We just need to ensure that we are able to correct connect to the server
     connectToSentryService();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/70c47e80/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
index 838e8d3..cc12099 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
@@ -20,6 +20,7 @@ package org.apache.sentry.service.thrift;
 import java.io.File;
 import java.security.PrivilegedExceptionAction;
 import java.util.HashSet;
+import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
@@ -32,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.minikdc.KerberosSecurityTestcase;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.sentry.provider.db.service.thrift.SentryMiniKdcTestcase;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
 import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.service.thrift.ServiceConstants.ClientConfig;
@@ -46,7 +48,7 @@ import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
-public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestcase {
+public abstract class SentryServiceIntegrationBase extends SentryMiniKdcTestcase {
   private static final Logger LOGGER = LoggerFactory.getLogger(SentryServiceIntegrationBase.class);
 
   static {
@@ -79,6 +81,7 @@ public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestc
   protected final Configuration conf = new Configuration(false);
   protected PolicyFile policyFile;
   protected File policyFilePath;
+  protected Properties kdcConfOverlay = new Properties();
 
   @Before
   public void setup() throws Exception {
@@ -90,6 +93,10 @@ public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestc
     afterSetup();
   }
 
+  private void setupKdc() throws Exception {
+    startMiniKdc(kdcConfOverlay);
+  }
+
   public void startSentryService() throws Exception {
     server.start();
     final long start = System.currentTimeMillis();
@@ -103,6 +110,7 @@ public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestc
 
   public void setupConf() throws Exception {
     if (kerberos) {
+      setupKdc();
       kdc = getKdc();
       kdcWorkDir = getWorkDir();
       serverKeytab = new File(kdcWorkDir, "server.keytab");