Преглед изворни кода

HADOOP-9747. Reduce unnecessary UGI synchronization. Contributed by Daryn Sharp.

Kihwal Lee пре 7 година
родитељ
комит
59cf758877

+ 0 - 11
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -355,17 +355,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS =
     "hadoop.user.group.metrics.percentiles.intervals";
 
-  /* When creating UGI with UserGroupInformation(Subject), treat the passed
-   * subject external if set to true, and assume the owner of the subject
-   * should do the credential renewal.
-   *
-   * This is a temporary config to solve the compatibility issue with
-   * HADOOP-13558 and HADOOP-13805 fix, see the jiras for discussions.
-   */
-  public static final String HADOOP_TREAT_SUBJECT_EXTERNAL_KEY =
-      "hadoop.treat.subject.external";
-  public static final boolean HADOOP_TREAT_SUBJECT_EXTERNAL_DEFAULT = false;
-
   public static final String RPC_METRICS_QUANTILE_ENABLE =
       "rpc.metrics.quantile.enable";
   public static final boolean RPC_METRICS_QUANTILE_ENABLE_DEFAULT = false;

Разлика између датотеке није приказан због своје велике величине
+ 429 - 462
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java


+ 0 - 13
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -2946,19 +2946,6 @@
         the ZK CLI).
     </description>
   </property>
-  <property>
-    <name>hadoop.treat.subject.external</name>
-    <value>false</value>
-    <description>
-      When creating UGI with UserGroupInformation(Subject), treat the passed
-      subject external if set to true, and assume the owner of the subject
-      should do the credential renewal.
-
-      When true this property will introduce an incompatible change which
-      may require changes in client code. For more details, see the jiras:
-      HADOOP-13805,HADOOP-13558.
-    </description>
-  </property>
   <property>
     <name>hadoop.system.tags</name>
     <value>YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT

+ 402 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java

@@ -21,14 +21,41 @@ package org.apache.hadoop.security;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.LoginContext;
 
 /**
  * Verify UGI login from keytab. Check that the UGI is
@@ -39,6 +66,7 @@ public class TestUGILoginFromKeytab {
 
   private MiniKdc kdc;
   private File workDir;
+  private ExecutorService executor;
 
   @Rule
   public final TemporaryFolder folder = new TemporaryFolder();
@@ -51,9 +79,12 @@ public class TestUGILoginFromKeytab {
     conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
         "kerberos");
     UserGroupInformation.setConfiguration(conf);
+    UserGroupInformation.setShouldRenewImmediatelyForTests(true);
     workDir = folder.getRoot();
     kdc = new MiniKdc(MiniKdc.createConf(), workDir);
     kdc.start();
+    executor = Executors.newCachedThreadPool();
+
   }
 
   @After
@@ -61,6 +92,9 @@ public class TestUGILoginFromKeytab {
     if (kdc != null) {
       kdc.stop();
     }
+    if (executor != null) {
+      executor.shutdownNow();
+    }
   }
 
   /**
@@ -69,7 +103,6 @@ public class TestUGILoginFromKeytab {
    */
   @Test
   public void testUGILoginFromKeytab() throws Exception {
-    UserGroupInformation.setShouldRenewImmediatelyForTests(true);
     String principal = "foo";
     File keytab = new File(workDir, "foo.keytab");
     kdc.createPrincipal(keytab, principal);
@@ -80,12 +113,379 @@ public class TestUGILoginFromKeytab {
         ugi.isFromKeytab());
 
     // Verify relogin from keytab.
-    User user = ugi.getSubject().getPrincipals(User.class).iterator().next();
+    User user = getUser(ugi.getSubject());
     final long firstLogin = user.getLastLogin();
+    final LoginContext login1 = user.getLogin();
+    Assert.assertNotNull(login1);
+
     ugi.reloginFromKeytab();
     final long secondLogin = user.getLastLogin();
+    final LoginContext login2 = user.getLogin();
     Assert.assertTrue("User should have been able to relogin from keytab",
         secondLogin > firstLogin);
+    Assert.assertNotNull(login2);
+    Assert.assertNotSame(login1, login2);
+  }
+
+  @Test
+  public void testGetUGIFromKnownSubject() throws Exception {
+    KerberosPrincipal principal = new KerberosPrincipal("user");
+    File keytab = new File(workDir, "user.keytab");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    UserGroupInformation ugi1 =
+      UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        principal.getName(), keytab.getPath());
+    Subject subject = ugi1.getSubject();
+    User user = getUser(subject);
+    Assert.assertNotNull(user);
+    LoginContext login = user.getLogin();
+    Assert.assertNotNull(login);
+
+    // User instance and/or login context should not change.
+    UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject);
+    Assert.assertSame(user, getUser(ugi2.getSubject()));
+    Assert.assertSame(login, user.getLogin());
+  }
+
+  @Test
+  public void testGetUGIFromExternalSubject() throws Exception {
+    KerberosPrincipal principal = new KerberosPrincipal("user");
+    File keytab = new File(workDir, "user.keytab");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    UserGroupInformation ugi =
+      UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        principal.getName(), keytab.getPath());
+    Subject subject = ugi.getSubject();
+    removeUser(subject);
+
+    // first call to get the ugi should add the User instance w/o a login
+    // context.
+    UserGroupInformation ugi1 = UserGroupInformation.getUGIFromSubject(subject);
+    Assert.assertSame(subject, ugi1.getSubject());
+    User user = getUser(subject);
+    Assert.assertNotNull(user);
+    Assert.assertEquals(principal.getName(), user.getName());
+    Assert.assertNull(user.getLogin());
+
+    // subsequent call should not change the existing User instance.
+    UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject);
+    Assert.assertSame(subject, ugi2.getSubject());
+    Assert.assertSame(user, getUser(ugi2.getSubject()));
+    Assert.assertNull(user.getLogin());
+  }
+
+  @Test
+  public void testGetUGIFromExternalSubjectWithLogin() throws Exception {
+    KerberosPrincipal principal = new KerberosPrincipal("user");
+    File keytab = new File(workDir, "user.keytab");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    // alter the User's login context to appear to be a foreign and
+    // unmanagable context.
+    UserGroupInformation ugi =
+      UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        principal.getName(), keytab.getPath());
+    Subject subject = ugi.getSubject();
+    User user = getUser(subject);
+    final LoginContext dummyLogin = Mockito.mock(LoginContext.class);
+    user.setLogin(dummyLogin);
+
+    // nothing should change.
+    UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject);
+    Assert.assertSame(subject, ugi2.getSubject());
+    Assert.assertSame(user, getUser(ugi2.getSubject()));
+    Assert.assertSame(dummyLogin, user.getLogin());
+  }
+
+
+  private static KerberosTicket getTicket(UserGroupInformation ugi) {
+    Set<KerberosTicket> tickets =
+        ugi.getSubject().getPrivateCredentials(KerberosTicket.class);
+    return tickets.isEmpty() ? null : tickets.iterator().next();
   }
 
+  // verify ugi has expected principal, a keytab, and has a ticket for
+  // the expected principal.
+  private static KerberosTicket checkTicketAndKeytab(UserGroupInformation ugi,
+      KerberosPrincipal principal, boolean expectIsKeytab) {
+    Assert.assertEquals("wrong principal",
+      principal.getName(), ugi.getUserName());
+    Assert.assertEquals("is not keytab",
+      expectIsKeytab, ugi.isFromKeytab());
+    KerberosTicket ticket = getTicket(ugi);
+    Assert.assertNotNull("no ticket", ticket);
+    Assert.assertEquals("wrong principal", principal, ticket.getClient());
+    return ticket;
+  }
+
+  @Test
+  public void testReloginForUGIFromSubject() throws Exception {
+    KerberosPrincipal principal1 = new KerberosPrincipal("user1");
+    File keytab1 = new File(workDir, "user1.keytab");
+    kdc.createPrincipal(keytab1, principal1.getName());
+
+    KerberosPrincipal principal2 = new KerberosPrincipal("user2");
+    File keytab2 = new File(workDir, "user2.keytab");
+    kdc.createPrincipal(keytab2, principal2.getName());
+
+    // Login a user and remove the User instance so it looks like an
+    // "external" subject.
+    final Subject extSubject =
+      UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        principal2.getName(), keytab2.getPath()).getSubject();
+    removeUser(extSubject);
+
+    // Login another user.
+    UserGroupInformation.loginUserFromKeytab(
+        principal1.getName(), keytab1.getPath());
+    final UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
+
+    loginUser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws IOException {
+        KerberosTicket loginTicket =
+            checkTicketAndKeytab(loginUser, principal1, true);
+
+        // get the ugi for the previously logged in subject.
+        UserGroupInformation extSubjectUser =
+            UserGroupInformation.getUGIFromSubject(extSubject);
+        KerberosTicket ticket =
+          checkTicketAndKeytab(extSubjectUser, principal2, false);
+
+        // verify login user got a new ticket.
+        loginUser.reloginFromKeytab();
+        KerberosTicket newLoginTicket =
+            checkTicketAndKeytab(loginUser, principal1, true);
+        Assert.assertNotEquals(loginTicket.getAuthTime(),
+            newLoginTicket.getAuthTime());
+
+        // verify an "external" subject ticket does not change.
+        extSubjectUser.reloginFromKeytab();
+        Assert.assertSame(ticket,
+            checkTicketAndKeytab(extSubjectUser, principal2, false));
+
+        // verify subject ugi relogin did not affect the login user.
+        Assert.assertSame(newLoginTicket,
+            checkTicketAndKeytab(loginUser, principal1, true));
+
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testReloginForLoginFromSubject() throws Exception {
+    KerberosPrincipal principal1 = new KerberosPrincipal("user1");
+    File keytab1 = new File(workDir, "user1.keytab");
+    kdc.createPrincipal(keytab1, principal1.getName());
+
+    KerberosPrincipal principal2 = new KerberosPrincipal("user2");
+    File keytab2 = new File(workDir, "user2.keytab");
+    kdc.createPrincipal(keytab2, principal2.getName());
+
+    // login principal1 with a keytab.
+    UserGroupInformation.loginUserFromKeytab(
+        principal1.getName(), keytab1.getPath());
+    final UserGroupInformation originalLoginUser =
+        UserGroupInformation.getLoginUser();
+    Assert.assertNotNull(getUser(originalLoginUser.getSubject()).getLogin());
+
+    originalLoginUser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws IOException {
+        KerberosTicket originalLoginUserTicket =
+            checkTicketAndKeytab(originalLoginUser, principal1, true);
+
+        // login principal2 from a subject with keytab.  it's external so
+        // no login context should be attached to the user.
+        final Subject subject =
+          UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+            principal2.getName(), keytab2.getPath()).getSubject();
+        removeUser(subject);
+
+        // verify the new login user is external.
+        UserGroupInformation.loginUserFromSubject(subject);
+        Assert.assertNull(getUser(subject).getLogin());
+        UserGroupInformation extLoginUser =
+          UserGroupInformation.getLoginUser();
+        KerberosTicket extLoginUserTicket =
+            checkTicketAndKeytab(extLoginUser, principal2, false);
+
+        // verify subject-based login user does not get a new ticket, and
+        // original login user not affected.
+        extLoginUser.reloginFromKeytab();
+        Assert.assertSame(extLoginUserTicket,
+          checkTicketAndKeytab(extLoginUser, principal2, false));
+        Assert.assertSame(originalLoginUserTicket,
+          checkTicketAndKeytab(originalLoginUser, principal1, true));
+
+        // verify original login user gets a new ticket, new login user
+        // not affected.
+        originalLoginUser.reloginFromKeytab();
+        Assert.assertNotSame(originalLoginUserTicket,
+            checkTicketAndKeytab(originalLoginUser, principal1, true));
+        Assert.assertSame(extLoginUserTicket,
+            checkTicketAndKeytab(extLoginUser, principal2, false));
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testReloginAfterFailedRelogin() throws Exception {
+    KerberosPrincipal principal = new KerberosPrincipal("user1");
+    File keytab = new File(workDir, "user1.keytab");
+    File keytabBackup = new File(keytab + ".backup");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    UserGroupInformation.loginUserFromKeytab(
+        principal.getName(), keytab.getPath());
+    final UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
+    checkTicketAndKeytab(loginUser, principal, true);
+
+    // move the keytab to induce a relogin failure.
+    Assert.assertTrue(keytab.renameTo(keytabBackup));
+    try {
+      loginUser.reloginFromKeytab();
+      Assert.fail("relogin should fail");
+    } catch (KerberosAuthException kae) {
+      // expected.
+    }
+
+    // even though no KeyTab object, ugi should know it's keytab based.
+    Assert.assertTrue(loginUser.isFromKeytab());
+    Assert.assertNull(getTicket(loginUser));
+
+    // move keytab back to enable relogin to succeed.
+    Assert.assertTrue(keytabBackup.renameTo(keytab));
+    loginUser.reloginFromKeytab();
+    checkTicketAndKeytab(loginUser, principal, true);
+  }
+
+  // verify getting concurrent relogins blocks to avoid indeterminate
+  // credentials corruption, but getting a ugi for the subject does not block.
+  @Test(timeout=180000)
+  public void testConcurrentRelogin() throws Exception {
+    final CyclicBarrier barrier = new CyclicBarrier(2);
+    final CountDownLatch latch = new CountDownLatch(1);
+    assertTrue(UserGroupInformation.isSecurityEnabled());
+
+    KerberosPrincipal principal = new KerberosPrincipal("testUser");
+    File keytab = new File(workDir, "user1.keytab");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    // create a keytab ugi.
+    final UserGroupInformation loginUgi =
+        UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+          principal.getName(), keytab.getPath());
+    assertEquals(AuthenticationMethod.KERBEROS,
+        loginUgi.getAuthenticationMethod());
+    assertTrue(loginUgi.isFromKeytab());
+
+    // create a new ugi instance based on subject from the logged in user.
+    final UserGroupInformation clonedUgi =
+        UserGroupInformation.getUGIFromSubject(loginUgi.getSubject());
+    assertEquals(AuthenticationMethod.KERBEROS,
+        clonedUgi.getAuthenticationMethod());
+    assertTrue(clonedUgi.isFromKeytab());
+
+    // cause first relogin to block on a barrier in logout to verify relogins
+    // are atomic.
+    User user = getUser(loginUgi.getSubject());
+    final LoginContext spyLogin = Mockito.spy(user.getLogin());
+    user.setLogin(spyLogin);
+    Mockito.doAnswer(new Answer<Void>(){
+      @Override
+      public Void answer(InvocationOnMock invocation)
+          throws Throwable {
+        invocation.callRealMethod();
+        latch.countDown();
+        barrier.await();
+        return null;
+      }
+    }).when(spyLogin).logout();
+
+    Future<Void> relogin = executor.submit(
+        new Callable<Void>(){
+          @Override
+          public Void call() throws Exception {
+            Thread.currentThread().setName("relogin");
+            loginUgi.reloginFromKeytab();
+            return null;
+          }
+        });
+    // wait for the thread to block on the barrier in the logout of the
+    // relogin.
+    assertTrue("first relogin didn't block",
+      latch.await(2, TimeUnit.SECONDS));
+
+    // although the logout removed the keytab instance, verify the ugi
+    // knows from its login params that it is supposed to be from a keytab.
+    assertTrue(clonedUgi.isFromKeytab());
+
+    // another concurrent re-login should block.
+    Mockito.doNothing().when(spyLogin).logout();
+    Mockito.doNothing().when(spyLogin).login();
+    Future<UserGroupInformation> clonedRelogin = executor.submit(
+        new Callable<UserGroupInformation>(){
+          @Override
+          public UserGroupInformation call() throws Exception {
+            Thread.currentThread().setName("clonedRelogin");
+            clonedUgi.reloginFromKeytab();
+            return clonedUgi;
+          }
+        });
+
+    try {
+      clonedRelogin.get(2, TimeUnit.SECONDS);
+      fail("second relogin didn't block!");
+    } catch (TimeoutException te) {
+      // expected
+    }
+
+    // concurrent UGI instantiation should not block and again should
+    // know it's supposed to be from a keytab.
+    loginUgi.doAs(new PrivilegedExceptionAction<Void>(){
+      @Override
+      public Void run() throws Exception {
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        assertEquals(principal.getName(), ugi.getUserName());
+        assertTrue(ugi.isFromKeytab());
+        return null;
+      }
+    });
+    clonedUgi.doAs(new PrivilegedExceptionAction<Void>(){
+      @Override
+      public Void run() throws Exception {
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        assertEquals(principal.getName(), ugi.getUserName());
+        assertTrue(ugi.isFromKeytab());
+        return null;
+      }
+    });
+
+    // second relogin should still be blocked until the original relogin
+    // is blocked.
+    assertFalse(clonedRelogin.isDone());
+    barrier.await();
+    relogin.get();
+    clonedRelogin.get();
+  }
+
+  private User getUser(Subject subject) {
+    Iterator<User> iter = subject.getPrincipals(User.class).iterator();
+    return iter.hasNext() ? iter.next() : null;
+  }
+
+  private void removeUser(Subject subject) {
+    // remove User instance so it appears to not be logged in.
+    for (Iterator<Principal> iter = subject.getPrincipals().iterator();
+         iter.hasNext(); ) {
+      if (iter.next() instanceof User) {
+        iter.remove();
+      }
+    }
+  }
 }

+ 12 - 42
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java

@@ -58,9 +58,11 @@ public class TestUGIWithMiniKdc {
 
   private void setupKdc() throws Exception {
     Properties kdcConf = MiniKdc.createConf();
-    // tgt expire time = 30 seconds
-    kdcConf.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "30");
-    kdcConf.setProperty(MiniKdc.MIN_TICKET_LIFETIME, "30");
+    // tgt expire time = 2 seconds.  just testing that renewal thread retries
+    // for expiring tickets, so no need to waste time waiting for expiry to
+    // arrive.
+    kdcConf.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "2");
+    kdcConf.setProperty(MiniKdc.MIN_TICKET_LIFETIME, "2");
     File kdcDir = new File(System.getProperty("test.dir", "target"));
     kdc = new MiniKdc(kdcConf, kdcDir);
     kdc.start();
@@ -70,12 +72,14 @@ public class TestUGIWithMiniKdc {
   public void testAutoRenewalThreadRetryWithKdc() throws Exception {
     GenericTestUtils.setLogLevel(UserGroupInformation.LOG, Level.DEBUG);
     final Configuration conf = new Configuration();
+    // can't rely on standard kinit, else test fails when user running
+    // the test is kinit'ed because the test renews _their TGT_.
+    conf.set("hadoop.kerberos.kinit.command", "bogus-kinit-cmd");
     // Relogin every 1 second
     conf.setLong(HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN, 1);
     SecurityUtil.setAuthenticationMethod(
         UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
-    UserGroupInformation.setEnableRenewThreadCreationForTest(true);
 
     LoginContext loginContext = null;
     try {
@@ -87,44 +91,10 @@ public class TestUGIWithMiniKdc {
       setupKdc();
       kdc.createPrincipal(keytab, principal);
 
-      // client login
-      final Subject subject =
-          new Subject(false, principals, new HashSet<>(), new HashSet<>());
-
-      loginContext = new LoginContext("", subject, null,
-          new javax.security.auth.login.Configuration() {
-            @Override
-            public AppConfigurationEntry[] getAppConfigurationEntry(
-                String name) {
-              Map<String, String> options = new HashMap<>();
-              options.put("principal", principal);
-              options.put("refreshKrb5Config", "true");
-              if (PlatformName.IBM_JAVA) {
-                options.put("useKeytab", keytab.getPath());
-                options.put("credsType", "both");
-              } else {
-                options.put("keyTab", keytab.getPath());
-                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(true));
-              }
-              String ticketCache = System.getenv("KRB5CCNAME");
-              if (ticketCache != null) {
-                options.put("ticketCache", ticketCache);
-              }
-              options.put("debug", "true");
-              return new AppConfigurationEntry[] {new AppConfigurationEntry(
-                  KerberosUtil.getKrb5LoginModuleName(),
-                  AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-                  options)};
-            }
-          });
-      loginContext.login();
-      final Subject loginSubject = loginContext.getSubject();
-      UserGroupInformation.loginUserFromSubject(loginSubject);
+      UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath());
+      UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+      // no ticket cache, so force the thread to test for failures.
+      ugi.spawnAutoRenewalThreadForUserCreds(true);
 
       // Verify retry happens. Do not verify retry count to reduce flakiness.
       // Detailed back-off logic is tested separately in

+ 95 - 18
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -38,6 +38,9 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -53,15 +56,19 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.lang.reflect.Method;
+import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.Date;
 import java.util.LinkedHashSet;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_TREAT_SUBJECT_EXTERNAL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL;
@@ -112,7 +119,14 @@ public class TestUserGroupInformation {
       throw new RuntimeException("UGI is not using its own security conf!");
     } 
   }
-  
+
+  // must be set immediately to avoid inconsistent testing issues.
+  static {
+    // fake the realm is kerberos is enabled
+    System.setProperty("java.security.krb5.kdc", "");
+    System.setProperty("java.security.krb5.realm", "DEFAULT.REALM");
+  }
+
   /** configure ugi */
   @BeforeClass
   public static void setup() {
@@ -123,9 +137,6 @@ public class TestUserGroupInformation {
     // that finds winutils.exe
     String home = System.getenv("HADOOP_HOME");
     System.setProperty("hadoop.home.dir", (home != null ? home : "."));
-    // fake the realm is kerberos is enabled
-    System.setProperty("java.security.krb5.kdc", "");
-    System.setProperty("java.security.krb5.realm", "DEFAULT.REALM");
   }
   
   @Before
@@ -1021,7 +1032,8 @@ public class TestUserGroupInformation {
     assertTrue(credsugiTokens.contains(token2));
   }
 
-  private void testCheckTGTAfterLoginFromSubjectHelper() throws Exception {
+  @Test
+  public void testCheckTGTAfterLoginFromSubject() throws Exception {
     // security on, default is remove default realm
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
@@ -1043,17 +1055,6 @@ public class TestUserGroupInformation {
     });
   }
 
-  @Test(expected = KerberosAuthException.class)
-  public void testCheckTGTAfterLoginFromSubject() throws Exception {
-    testCheckTGTAfterLoginFromSubjectHelper();
-  }
-
-  @Test
-  public void testCheckTGTAfterLoginFromSubjectFix() throws Exception {
-    conf.setBoolean(HADOOP_TREAT_SUBJECT_EXTERNAL_KEY, true);
-    testCheckTGTAfterLoginFromSubjectHelper();
-  }
-
   @Test
   public void testGetNextRetryTime() throws Exception {
     GenericTestUtils.setLogLevel(UserGroupInformation.LOG, Level.DEBUG);
@@ -1134,4 +1135,80 @@ public class TestUserGroupInformation {
     LOG.info(str);
     assertTrue(str, lower <= lastRetry && lastRetry < upper);
   }
+
+  // verify that getCurrentUser on the same and different subjects can be
+  // concurrent.  Ie. no synchronization.
+  @Test(timeout=8000)
+  public void testConcurrentGetCurrentUser() throws Exception {
+    final CyclicBarrier barrier = new CyclicBarrier(2);
+    final CountDownLatch latch = new CountDownLatch(1);
+
+    final UserGroupInformation testUgi1 =
+        UserGroupInformation.createRemoteUser("testUgi1");
+
+    final UserGroupInformation testUgi2 =
+        UserGroupInformation.createRemoteUser("testUgi2");
+
+    // swap the User with a spy to allow getCurrentUser to block when the
+    // spy is called for the user name.
+    Set<Principal> principals = testUgi1.getSubject().getPrincipals();
+    User user =
+        testUgi1.getSubject().getPrincipals(User.class).iterator().next();
+    final User spyUser = Mockito.spy(user);
+    principals.remove(user);
+    principals.add(spyUser);
+    when(spyUser.getName()).thenAnswer(new Answer<String>(){
+      @Override
+      public String answer(InvocationOnMock invocation) throws Throwable {
+        latch.countDown();
+        barrier.await();
+        return (String)invocation.callRealMethod();
+      }
+    });
+    // wait for the thread to block on the barrier in getCurrentUser.
+    Future<UserGroupInformation> blockingLookup =
+        Executors.newSingleThreadExecutor().submit(
+            new Callable<UserGroupInformation>(){
+              @Override
+              public UserGroupInformation call() throws Exception {
+                return testUgi1.doAs(
+                    new PrivilegedExceptionAction<UserGroupInformation>() {
+                      @Override
+                      public UserGroupInformation run() throws Exception {
+                        return UserGroupInformation.getCurrentUser();
+                      }
+                    });
+              }
+            });
+    latch.await();
+
+    // old versions of mockito synchronize on returning mocked answers so
+    // the blocked getCurrentUser will block all other calls to getName.
+    // workaround this by swapping out the spy with the original User.
+    principals.remove(spyUser);
+    principals.add(user);
+    // concurrent getCurrentUser on ugi1 should not be blocked.
+    UserGroupInformation ugi;
+    ugi = testUgi1.doAs(
+        new PrivilegedExceptionAction<UserGroupInformation>() {
+          @Override
+          public UserGroupInformation run() throws Exception {
+            return UserGroupInformation.getCurrentUser();
+          }
+        });
+    assertSame(testUgi1.getSubject(), ugi.getSubject());
+    // concurrent getCurrentUser on ugi2 should not be blocked.
+    ugi = testUgi2.doAs(
+        new PrivilegedExceptionAction<UserGroupInformation>() {
+          @Override
+          public UserGroupInformation run() throws Exception {
+            return UserGroupInformation.getCurrentUser();
+          }
+        });
+    assertSame(testUgi2.getSubject(), ugi.getSubject());
+
+    // unblock the original call.
+    barrier.await();
+    assertSame(testUgi1.getSubject(), blockingLookup.get().getSubject());
+  }
 }

Неке датотеке нису приказане због велике количине промена