You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by sy...@apache.org on 2022/03/01 14:15:41 UTC

[zookeeper] branch branch-3.5 updated: ZOOKEEPER-4477: Single Kerberos ticket renewal failure can prevent all future renewals since Java 9

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

symat pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
     new 73ea6aa  ZOOKEEPER-4477: Single Kerberos ticket renewal failure can prevent all future renewals since Java 9
73ea6aa is described below

commit 73ea6aaffc7a29a31df9007244f96d8d67fef953
Author: Mate Szalay-Beko <ms...@cloudera.com>
AuthorDate: Tue Mar 1 14:14:52 2022 +0000

    ZOOKEEPER-4477: Single Kerberos ticket renewal failure can prevent all future renewals since Java 9
    
    This is the ZOOKEEPER-4477 patch for branch-3.5. Similar to the same fix on newer branches
    but using junit 4 in the tests (the junit upgrade happened in ZooKeeper 3.6).
    
    This bug is similar to the one fixed in https://issues.apache.org/jira/browse/KAFKA-12730.
    
    Our Kerberos ticket refresh thread performs re-login by logging out and then logging in again. If
    login fails, we retry after some sleep. Every reLogin() operation performs loginContext.logout()
    and loginContext.login(). If login fails, we end up with two consecutive logouts. This used to
    work in older Java versions, but from Java 9 onwards, this results in a NullPointerException due
    to https://bugs.openjdk.java.net/browse/JDK-8173069. We should check if logout is required before
    attempting logout.
    
    I fixed the issue and added a new unit test to test some ticket renewal scenarios. I managed to
    reproduce the problem in KerberosTicketRenewalTest.shouldRecoverIfKerberosNotAvailableForSomeTime()
    which (before the fix) failed with Java13 but succeeded with Java8.
    
    Author: Mate Szalay-Beko <ms...@cloudera.com>
    
    Reviewers: Enrico Olivelli <eo...@apache.org>
    
    Closes #1830 from symat/ZOOKEEPER-4477-branch-3.5
---
 .../src/main/java/org/apache/zookeeper/Login.java  |  40 ++-
 .../zookeeper/KerberosTicketRenewalTest.java       | 285 +++++++++++++++++++++
 .../server/quorum/auth/KerberosTestUtils.java      |  64 ++++-
 .../zookeeper/server/quorum/auth/MiniKdc.java      |   8 +
 .../zookeeper/server/quorum/auth/MiniKdcTest.java  |  65 +----
 5 files changed, 390 insertions(+), 72 deletions(-)

diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/Login.java b/zookeeper-server/src/main/java/org/apache/zookeeper/Login.java
index d97d6c1..6c49287 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/Login.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/Login.java
@@ -49,6 +49,7 @@ import java.util.Set;
 public class Login {
     private static final String KINIT_COMMAND_DEFAULT = "/usr/bin/kinit";
     private static final Logger LOG = LoggerFactory.getLogger(Login.class);
+    public static final String SYSTEM_USER = System.getProperty("user.name", "<NA>");
     public CallbackHandler callbackHandler;
 
     // LoginThread will sleep until 80% of time from last refresh to
@@ -64,7 +65,10 @@ public class Login {
     // Regardless of TICKET_RENEW_WINDOW setting above and the ticket expiry time,
     // thread will not sleep between refresh attempts any less than 1 minute (60*1000 milliseconds = 1 minute).
     // Change the '1' to e.g. 5, to change this to 5 minutes.
-    private static final long MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L;
+    private static final long DEFAULT_MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L;
+    public static final String MIN_TIME_BEFORE_RELOGIN_CONFIG_KEY = "zookeeper.kerberos.minReLoginTimeMs";
+    private static final long MIN_TIME_BEFORE_RELOGIN = Long.getLong(
+      MIN_TIME_BEFORE_RELOGIN_CONFIG_KEY, DEFAULT_MIN_TIME_BEFORE_RELOGIN);
 
     private Subject subject = null;
     private Thread t = null;
@@ -219,7 +223,7 @@ public class Login {
                                     --retry;
                                     // sleep for 10 seconds
                                     try {
-                                        Thread.sleep(10 * 1000);
+                                        sleepBeforeRetryFailedRefresh();
                                     } catch (InterruptedException ie) {
                                         LOG.error("Interrupted while renewing TGT, exiting Login thread");
                                         return;
@@ -244,7 +248,7 @@ public class Login {
                                     --retry;
                                     // sleep for 10 seconds.
                                     try {
-                                        Thread.sleep(10 * 1000);
+                                        sleepBeforeRetryFailedRefresh();
                                     } catch (InterruptedException e) {
                                         LOG.error("Interrupted during login retry after LoginException:", le);
                                         throw le;
@@ -286,6 +290,13 @@ public class Login {
         return subject;
     }
 
+    public String getUserName() {
+        if (principal == null || principal.isEmpty()) {
+            return SYSTEM_USER;
+        }
+        return principal;
+    }
+
     public String getLoginContextName() {
         return loginContextName;
     }
@@ -381,10 +392,10 @@ public class Login {
     }
 
     /**
-     * Get the time of the last login.
-     * @return the number of milliseconds since the beginning of time.
+     * Get the time of the last login (ticket initialization or last ticket renewal).
+     * @return the number of milliseconds since epoch.
      */
-    private long getLastLogin() {
+    public long getLastLogin() {
         return lastLogin;
     }
 
@@ -410,7 +421,7 @@ public class Login {
             //clear up the kerberos state. But the tokens are not cleared! As per
             //the Java kerberos login module code, only the kerberos credentials
             //are cleared
-            login.logout();
+            logout();
             //login and also update the subject field of this instance to
             //have the new credentials (pass it to the LoginContext constructor)
             login = new LoginContext(loginContextName, getSubject());
@@ -419,4 +430,19 @@ public class Login {
             setLogin(login);
         }
     }
+
+    // this method also visible for unit tests, to make sure kerberos state cleaned up
+    protected synchronized void logout() throws LoginException {
+        // We need to make sure not to call LoginContext.logout() when we
+        // are not logged in. Since Java 9 this could result in an NPE.
+        // See ZOOKEEPER-4477 for more details.
+        if (subject != null && !subject.getPrincipals().isEmpty()) {
+            login.logout();
+        }
+    }
+
+    // this method is overwritten in unit tests to test concurrency
+    protected void sleepBeforeRetryFailedRefresh() throws InterruptedException {
+        Thread.sleep(10 * 1000);
+    }
 }
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/KerberosTicketRenewalTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/KerberosTicketRenewalTest.java
new file mode 100644
index 0000000..235a107
--- /dev/null
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/KerberosTicketRenewalTest.java
@@ -0,0 +1,285 @@
+/*
+ * 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.zookeeper;
+
+import static org.apache.zookeeper.server.quorum.auth.MiniKdc.MAX_TICKET_LIFETIME;
+import static org.apache.zookeeper.server.quorum.auth.MiniKdc.MIN_TICKET_LIFETIME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.security.Principal;
+import java.time.Duration;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Supplier;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.zookeeper.common.ZKConfig;
+import org.apache.zookeeper.server.quorum.auth.KerberosTestUtils;
+import org.apache.zookeeper.server.quorum.auth.MiniKdc;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This test class is mainly testing the TGT renewal logic implemented
+ * in the org.apache.zookeeper.Login class.
+ */
+public class KerberosTicketRenewalTest {
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(KerberosTicketRenewalTest.class);
+  private static final String JAAS_CONFIG_SECTION = "ClientUsingKerberos";
+  private static final String TICKET_LIFETIME = "5000";
+  private static File testTempDir;
+  private static MiniKdc kdc;
+  private static File kdcWorkDir;
+  private static String PRINCIPAL = KerberosTestUtils.getClientPrincipal();
+
+  TestableKerberosLogin login;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    // by default, we should wait at least 1 minute between subsequent TGT renewals.
+    // changing it to 500ms.
+    System.setProperty(Login.MIN_TIME_BEFORE_RELOGIN_CONFIG_KEY, "500");
+
+    testTempDir = ClientBase.createTmpDir();
+    startMiniKdcAndAddPrincipal();
+
+    String keytabFilePath = FilenameUtils.normalize(KerberosTestUtils.getKeytabFile(), true);
+
+    // note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM,
+    // making sure that we use the latest config even if other tests already have been executed
+    // and initialized the kerberos client configs before)
+    String jaasEntries = ""
+      + "ClientUsingKerberos {\n"
+      + "  com.sun.security.auth.module.Krb5LoginModule required\n"
+      + "  storeKey=\"false\"\n"
+      + "  useTicketCache=\"false\"\n"
+      + "  useKeyTab=\"true\"\n"
+      + "  doNotPrompt=\"true\"\n"
+      + "  debug=\"true\"\n"
+      + "  refreshKrb5Config=\"true\"\n"
+      + "  keyTab=\"" + keytabFilePath + "\"\n"
+      + "  principal=\"" + PRINCIPAL + "\";\n"
+      + "};\n";
+    setupJaasConfig(jaasEntries);
+  }
+
+  @AfterClass
+  public static void tearDownClass() {
+    System.clearProperty(Login.MIN_TIME_BEFORE_RELOGIN_CONFIG_KEY);
+    System.clearProperty("java.security.auth.login.config");
+    stopMiniKdc();
+    if (testTempDir != null) {
+      // the testTempDir contains the jaas config file and also the
+      // working folder of the currently running KDC server
+      FileUtils.deleteQuietly(testTempDir);
+    }
+  }
+
+  @After
+  public void tearDownTest() throws Exception {
+    if (login != null) {
+      login.shutdown();
+      login.logout();
+    }
+  }
+
+
+  /**
+   * We extend the regular Login class to be able to properly control the
+   * "sleeping" between the retry attempts of ticket refresh actions.
+   */
+  private static class TestableKerberosLogin extends Login {
+
+    private AtomicBoolean refreshFailed = new AtomicBoolean(false);
+    private CountDownLatch continueRefreshThread = new CountDownLatch(1);
+
+    public TestableKerberosLogin() throws LoginException {
+      super(JAAS_CONFIG_SECTION, (callbacks) -> {}, new ZKConfig());
+    }
+
+    @Override
+    protected void sleepBeforeRetryFailedRefresh() throws InterruptedException {
+      LOG.info("sleep started due to failed refresh");
+      refreshFailed.set(true);
+      continueRefreshThread.await(20, TimeUnit.SECONDS);
+      LOG.info("sleep due to failed refresh finished");
+    }
+
+    public void assertRefreshFailsEventually(Duration timeout) {
+      assertEventually(timeout, () -> refreshFailed.get());
+    }
+
+    public void continueWithRetryAfterFailedRefresh() {
+      LOG.info("continue refresh thread");
+      continueRefreshThread.countDown();
+    }
+  }
+
+
+  @Test
+  public void shouldLoginUsingKerberos() throws Exception {
+    login = new TestableKerberosLogin();
+    login.startThreadIfNeeded();
+
+    assertPrincipalLoggedIn();
+  }
+
+
+  @Test
+  public void shouldRenewTicketUsingKerberos() throws Exception {
+    login = new TestableKerberosLogin();
+    login.startThreadIfNeeded();
+
+    long initialLoginTime = login.getLastLogin();
+
+    // ticket lifetime is 5sec, so we will trigger ticket renewal in each ~2-3 sec
+    assertTicketRefreshHappenedUntil(Duration.ofSeconds(15));
+
+    assertPrincipalLoggedIn();
+    assertTrue(initialLoginTime < login.getLastLogin());
+  }
+
+
+  @Test
+  public void shouldRecoverIfKerberosNotAvailableForSomeTime() throws Exception {
+    login = new TestableKerberosLogin();
+    login.startThreadIfNeeded();
+
+    assertTicketRefreshHappenedUntil(Duration.ofSeconds(15));
+
+    stopMiniKdc();
+
+    // ticket lifetime is 5sec, so we will trigger ticket renewal in each ~2-3 sec
+    // the very next ticket renewal should fail (as KDC is offline)
+    login.assertRefreshFailsEventually(Duration.ofSeconds(15));
+
+    // now the ticket thread is "sleeping", it will retry the refresh later
+
+    // we restart KDC, then terminate the "sleeping" and expecting
+    // that the next retry should succeed
+    startMiniKdcAndAddPrincipal();
+    login.continueWithRetryAfterFailedRefresh();
+    assertTicketRefreshHappenedUntil(Duration.ofSeconds(15));
+
+    assertPrincipalLoggedIn();
+  }
+
+
+  private void assertPrincipalLoggedIn() {
+    assertEquals(PRINCIPAL, login.getUserName());
+    assertNotNull(login.getSubject());
+    assertEquals(1, login.getSubject().getPrincipals().size());
+    Principal actualPrincipal = login.getSubject().getPrincipals().iterator().next();
+    assertEquals(PRINCIPAL, actualPrincipal.getName());
+  }
+
+  private void assertTicketRefreshHappenedUntil(Duration timeout) {
+    long lastLoginTime = login.getLastLogin();
+    assertEventually(timeout, () -> login.getLastLogin() != lastLoginTime
+      && login.getSubject() != null && !login.getSubject().getPrincipals().isEmpty());
+  }
+
+  private static void assertEventually(Duration timeout, Supplier<Boolean> test) {
+    long until = System.currentTimeMillis() + timeout.toMillis();
+    while(System.currentTimeMillis() < until) {
+      if(test.get()) {
+        return;
+      }
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        fail("interrupted while waining for test condition to appear");
+      }
+    }
+    fail("timeout");
+  }
+
+  public static void startMiniKdcAndAddPrincipal() throws Exception {
+    kdcWorkDir = createTmpDirInside(testTempDir);
+
+    Properties conf = MiniKdc.createConf();
+    conf.setProperty(MAX_TICKET_LIFETIME, TICKET_LIFETIME);
+    conf.setProperty(MIN_TICKET_LIFETIME, TICKET_LIFETIME);
+
+    kdc = new MiniKdc(conf, kdcWorkDir);
+    kdc.start();
+
+    String principalName = PRINCIPAL.substring(0, PRINCIPAL.lastIndexOf("@"));
+    kdc.createPrincipal(new File(KerberosTestUtils.getKeytabFile()), principalName);
+  }
+
+  private static void stopMiniKdc() {
+    if (kdc != null) {
+      kdc.stop();
+      kdc = null;
+    }
+    if (kdcWorkDir != null) {
+      FileUtils.deleteQuietly(kdcWorkDir);
+      kdcWorkDir = null;
+    }
+  }
+
+  private static File createTmpDirInside(File parentDir) throws IOException {
+    File tmpFile = File.createTempFile("test", ".junit", parentDir);
+    // don't delete tmpFile - this ensures we don't attempt to create
+    // a tmpDir with a duplicate name
+    File tmpDir = new File(tmpFile + ".dir");
+    // never true if tmpfile does it's job
+    assertFalse(tmpDir.exists());
+    assertTrue(tmpDir.mkdirs());
+    return tmpDir;
+  }
+
+  private static void setupJaasConfig(String jaasEntries) {
+    try {
+      File saslConfFile = new File(testTempDir, "jaas.conf");
+      FileWriter fwriter = new FileWriter(saslConfFile);
+      fwriter.write(jaasEntries);
+      fwriter.close();
+      System.setProperty("java.security.auth.login.config", saslConfFile.getAbsolutePath());
+    } catch (IOException ioe) {
+      LOG.error("Failed to initialize JAAS conf file", ioe);
+    }
+
+    // refresh the SASL configuration in this JVM (making sure that we use the latest config
+    // even if other tests already have been executed and initialized the SASL configs before)
+    Configuration.getConfiguration().refresh();
+  }
+
+}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
index ab2a7b5..d7e0ad1 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
@@ -19,11 +19,17 @@
 package org.apache.zookeeper.server.quorum.auth;
 
 import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.UUID;
-
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
 import org.apache.zookeeper.util.SecurityUtils;
 
 public class KerberosTestUtils {
+
+    private static final boolean IBM_JAVA = System.getProperty("java.vendor").contains("IBM");
+
     private static String keytabFile = new File(System.getProperty("build.test.dir", "build"), UUID.randomUUID().toString())
             .getAbsolutePath();
 
@@ -81,4 +87,60 @@ public class KerberosTestUtils {
             return components[0] + "/" + hostname.toLowerCase();
         }
     }
+
+    public static class KerberosConfiguration extends Configuration {
+
+        private String principal;
+        private String keytab;
+        private boolean isInitiator;
+
+        private KerberosConfiguration(String principal, File keytab, boolean client) {
+            this.principal = principal;
+            this.keytab = keytab.getAbsolutePath();
+            this.isInitiator = client;
+        }
+
+        public static Configuration createClientConfig(String principal, File keytab) {
+            return new KerberosConfiguration(principal, keytab, true);
+        }
+
+        public static Configuration createServerConfig(String principal, File keytab) {
+            return new KerberosConfiguration(principal, keytab, false);
+        }
+
+        private static String getKrb5LoginModuleName() {
+            return System.getProperty("java.vendor").contains("IBM")
+              ? "com.ibm.security.auth.module.Krb5LoginModule"
+              : "com.sun.security.auth.module.Krb5LoginModule";
+        }
+
+        @Override
+        public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+            Map<String, String> options = new HashMap<String, String>();
+            options.put("principal", principal);
+            options.put("refreshKrb5Config", "true");
+            if (IBM_JAVA) {
+                options.put("useKeytab", keytab);
+                options.put("credsType", "both");
+            } else {
+                options.put("keyTab", keytab);
+                options.put("useKeyTab", "true");
+                options.put("storeKey", "true");
+                options.put("doNotPrompt", "true");
+                options.put("useTicketCache", "true");
+                options.put("renewTGT", "true");
+                options.put("isInitiator", Boolean.toString(isInitiator));
+            }
+            String ticketCache = System.getenv("KRB5CCNAME");
+            if (ticketCache != null) {
+                options.put("ticketCache", ticketCache);
+            }
+            options.put("debug", "true");
+
+            return new AppConfigurationEntry[]{new AppConfigurationEntry(getKrb5LoginModuleName(), AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, options)};
+        }
+
+    }
+
+
 }
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdc.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdc.java
index 1a3c88a..48c13ff 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdc.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdc.java
@@ -62,6 +62,7 @@ import java.util.Set;
  *   <li>kdc.port=0 (ephemeral port)</li>
  *   <li>instance=DefaultKrbServer</li>
  *   <li>max.ticket.lifetime=86400000 (1 day)</li>
+ *   <li>min.ticket.lifetime=3600000 (1 hour)</li>
  *   <li>max.renewable.lifetime=604800000 (7 days)</li>
  *   <li>transport=TCP</li>
  *   <li>debug=false</li>
@@ -154,6 +155,7 @@ public class MiniKdc {
     public static final String KDC_PORT = "kdc.port";
     public static final String INSTANCE = "instance";
     public static final String MAX_TICKET_LIFETIME = "max.ticket.lifetime";
+    public static final String MIN_TICKET_LIFETIME = "min.ticket.lifetime";
     public static final String MAX_RENEWABLE_LIFETIME = "max.renewable.lifetime";
     public static final String TRANSPORT = "transport";
     public static final String DEBUG = "debug";
@@ -170,6 +172,7 @@ public class MiniKdc {
         PROPERTIES.add(INSTANCE);
         PROPERTIES.add(TRANSPORT);
         PROPERTIES.add(MAX_TICKET_LIFETIME);
+        PROPERTIES.add(MIN_TICKET_LIFETIME);
         PROPERTIES.add(MAX_RENEWABLE_LIFETIME);
 
         DEFAULT_CONFIG.setProperty(KDC_BIND_ADDRESS, "localhost");
@@ -179,6 +182,7 @@ public class MiniKdc {
         DEFAULT_CONFIG.setProperty(ORG_DOMAIN, "COM");
         DEFAULT_CONFIG.setProperty(TRANSPORT, "TCP");
         DEFAULT_CONFIG.setProperty(MAX_TICKET_LIFETIME, "86400000");
+        DEFAULT_CONFIG.setProperty(MIN_TICKET_LIFETIME, "3600000");
         DEFAULT_CONFIG.setProperty(MAX_RENEWABLE_LIFETIME, "604800000");
         DEFAULT_CONFIG.setProperty(DEBUG, "false");
     }
@@ -325,6 +329,10 @@ public class MiniKdc {
         }
         simpleKdc.getKdcConfig().setString(KdcConfigKey.KDC_SERVICE_NAME,
                 conf.getProperty(INSTANCE));
+        long minTicketLifetimeConf = Long.parseLong(conf.getProperty(MIN_TICKET_LIFETIME)) / 1000;
+        simpleKdc.getKdcConfig().setLong(KdcConfigKey.MINIMUM_TICKET_LIFETIME, minTicketLifetimeConf);
+        long maxTicketLifetimeConf = Long.parseLong(conf.getProperty(MAX_TICKET_LIFETIME)) / 1000;
+        simpleKdc.getKdcConfig().setLong(KdcConfigKey.MAXIMUM_TICKET_LIFETIME, maxTicketLifetimeConf);
         if (conf.getProperty(DEBUG) != null) {
             krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG, conf.getProperty(DEBUG));
         }
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdcTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdcTest.java
index 69dbcd1..83fa467 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdcTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/MiniKdcTest.java
@@ -20,21 +20,18 @@ package org.apache.zookeeper.server.quorum.auth;
 
 import org.apache.kerby.kerberos.kerb.keytab.Keytab;
 import org.apache.kerby.kerberos.kerb.type.base.PrincipalName;
+import org.apache.zookeeper.server.quorum.auth.KerberosTestUtils.KerberosConfiguration;
 import org.junit.Assert;
 import org.junit.Test;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
 import java.io.File;
 import java.security.Principal;
 import java.util.List;
 import java.util.Set;
-import java.util.Map;
 import java.util.HashSet;
-import java.util.HashMap;
 import java.util.Arrays;
 
 /*
@@ -45,8 +42,6 @@ import java.util.Arrays;
  * Github Revision: 916140604ffef59466ba30832478311d3e6249bd
  */
 public class MiniKdcTest extends KerberosSecurityTestcase {
-    private static final boolean IBM_JAVA = System.getProperty("java.vendor")
-            .contains("IBM");
 
     @Test(timeout = 60000)
     public void testMiniKdcStart() {
@@ -73,64 +68,6 @@ public class MiniKdcTest extends KerberosSecurityTestcase {
                 principals);
       }
 
-    private static class KerberosConfiguration extends Configuration {
-        private String principal;
-        private String keytab;
-        private boolean isInitiator;
-
-        private KerberosConfiguration(String principal, File keytab,
-                boolean client) {
-            this.principal = principal;
-            this.keytab = keytab.getAbsolutePath();
-            this.isInitiator = client;
-        }
-
-        public static Configuration createClientConfig(String principal,
-                File keytab) {
-            return new KerberosConfiguration(principal, keytab, true);
-        }
-
-        public static Configuration createServerConfig(String principal,
-                File keytab) {
-            return new KerberosConfiguration(principal, keytab, false);
-        }
-
-        private static String getKrb5LoginModuleName() {
-            return System.getProperty("java.vendor").contains("IBM")
-                    ? "com.ibm.security.auth.module.Krb5LoginModule"
-                    : "com.sun.security.auth.module.Krb5LoginModule";
-        }
-
-        @Override
-        public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
-            Map<String, String> options = new HashMap<String, String>();
-            options.put("principal", principal);
-            options.put("refreshKrb5Config", "true");
-            if (IBM_JAVA) {
-                options.put("useKeytab", keytab);
-                options.put("credsType", "both");
-            } else {
-                options.put("keyTab", keytab);
-                options.put("useKeyTab", "true");
-                options.put("storeKey", "true");
-                options.put("doNotPrompt", "true");
-                options.put("useTicketCache", "true");
-                options.put("renewTGT", "true");
-                options.put("isInitiator", Boolean.toString(isInitiator));
-            }
-            String ticketCache = System.getenv("KRB5CCNAME");
-            if (ticketCache != null) {
-                options.put("ticketCache", ticketCache);
-            }
-            options.put("debug", "true");
-
-            return new AppConfigurationEntry[] {
-                    new AppConfigurationEntry(getKrb5LoginModuleName(),
-                            AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-                            options) };
-        }
-    }
-
     @Test(timeout = 60000)
     public void testKerberosLogin() throws Exception {
         MiniKdc kdc = getKdc();