Browse Source

HADOOP-8726. The Secrets in Credentials are not available to MR tasks (daryn and Benoy Antony via bobby)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1379105 13f79535-47bb-0310-9956-ffa450edef68
Robert Joseph Evans 12 years ago
parent
commit
0fb6e7626f

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -812,6 +812,9 @@ Release 0.23.3 - UNRELEASED
 
 
     HADOOP-8725. MR is broken when security is off (daryn via bobby)
     HADOOP-8725. MR is broken when security is off (daryn via bobby)
 
 
+    HADOOP-8726. The Secrets in Credentials are not available to MR tasks
+    (daryn and Benoy Antony via bobby)
+
 Release 0.23.2 - UNRELEASED 
 Release 0.23.2 - UNRELEASED 
 
 
   NEW FEATURES
   NEW FEATURES

+ 0 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java

@@ -274,10 +274,4 @@ public class Credentials implements Writable {
       }
       }
     }
     }
   }
   }
-  
-  public void addTokensToUGI(UserGroupInformation ugi) {
-    for (Map.Entry<Text, Token<?>> token: tokenMap.entrySet()) {
-      ugi.addToken(token.getKey(), token.getValue());
-    }
-  }
 }
 }

+ 26 - 49
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -27,7 +27,6 @@ import java.security.Principal;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
@@ -646,7 +645,7 @@ public class UserGroupInformation {
           // user.
           // user.
           Credentials cred = Credentials.readTokenStorageFile(
           Credentials cred = Credentials.readTokenStorageFile(
               new Path("file:///" + fileLocation), conf);
               new Path("file:///" + fileLocation), conf);
-          cred.addTokensToUGI(loginUser);
+          loginUser.addCredentials(cred);
         }
         }
         loginUser.spawnAutoRenewalThreadForUserCreds();
         loginUser.spawnAutoRenewalThreadForUserCreds();
       } catch (LoginException le) {
       } catch (LoginException le) {
@@ -1176,41 +1175,6 @@ public class UserGroupInformation {
   public synchronized Set<TokenIdentifier> getTokenIdentifiers() {
   public synchronized Set<TokenIdentifier> getTokenIdentifiers() {
     return subject.getPublicCredentials(TokenIdentifier.class);
     return subject.getPublicCredentials(TokenIdentifier.class);
   }
   }
-
-  // wrapper to retain the creds key for the token
-  private class NamedToken {
-    Text alias;
-    Token<? extends TokenIdentifier> token;
-    NamedToken(Text alias, Token<? extends TokenIdentifier> token) {
-      this.alias = alias;
-      this.token = token;
-    }
-    @Override
-    public boolean equals(Object o) {
-      boolean equals;
-      if (o == this) {
-        equals = true;
-      } else if (!(o instanceof NamedToken)) {
-        equals = false;
-      } else {
-        Text otherAlias = ((NamedToken)o).alias;
-        if (alias == otherAlias) {
-          equals = true;
-        } else {
-          equals = (otherAlias != null && otherAlias.equals(alias));
-        }
-      }
-      return equals;
-    }
-    @Override
-    public int hashCode() {
-      return (alias != null) ? alias.hashCode() : -1; 
-    }
-    @Override
-    public String toString() {
-      return "NamedToken: alias="+alias+" token="+token;
-    }
-  }
   
   
   /**
   /**
    * Add a token to this UGI
    * Add a token to this UGI
@@ -1219,7 +1183,7 @@ public class UserGroupInformation {
    * @return true on successful add of new token
    * @return true on successful add of new token
    */
    */
   public synchronized boolean addToken(Token<? extends TokenIdentifier> token) {
   public synchronized boolean addToken(Token<? extends TokenIdentifier> token) {
-    return addToken(token.getService(), token);
+    return (token != null) ? addToken(token.getService(), token) : false;
   }
   }
 
 
   /**
   /**
@@ -1231,10 +1195,8 @@ public class UserGroupInformation {
    */
    */
   public synchronized boolean addToken(Text alias,
   public synchronized boolean addToken(Text alias,
                                        Token<? extends TokenIdentifier> token) {
                                        Token<? extends TokenIdentifier> token) {
-    NamedToken namedToken = new NamedToken(alias, token);
-    Collection<Object> ugiCreds = subject.getPrivateCredentials();
-    ugiCreds.remove(namedToken); // allow token to be replaced
-    return ugiCreds.add(new NamedToken(alias, token));
+    getCredentialsInternal().addToken(alias, token);
+    return true;
   }
   }
   
   
   /**
   /**
@@ -1244,8 +1206,8 @@ public class UserGroupInformation {
    */
    */
   public synchronized
   public synchronized
   Collection<Token<? extends TokenIdentifier>> getTokens() {
   Collection<Token<? extends TokenIdentifier>> getTokens() {
-    return Collections.unmodifiableList(
-        new ArrayList<Token<?>>(getCredentials().getAllTokens()));
+    return Collections.unmodifiableCollection(
+        getCredentialsInternal().getAllTokens());
   }
   }
 
 
   /**
   /**
@@ -1254,11 +1216,26 @@ public class UserGroupInformation {
    * @return Credentials of tokens associated with this user
    * @return Credentials of tokens associated with this user
    */
    */
   public synchronized Credentials getCredentials() {
   public synchronized Credentials getCredentials() {
-    final Credentials credentials = new Credentials();
-    final Set<NamedToken> namedTokens =
-        subject.getPrivateCredentials(NamedToken.class);
-    for (final NamedToken namedToken : namedTokens) {
-      credentials.addToken(namedToken.alias, namedToken.token);
+    return new Credentials(getCredentialsInternal());
+  }
+  
+  /**
+   * Add the given Credentials to this user.
+   * @param credentials of tokens and secrets
+   */
+  public synchronized void addCredentials(Credentials credentials) {
+    getCredentialsInternal().addAll(credentials);
+  }
+
+  private synchronized Credentials getCredentialsInternal() {
+    final Credentials credentials;
+    final Set<Credentials> credentialsSet =
+      subject.getPrivateCredentials(Credentials.class);
+    if (!credentialsSet.isEmpty()){
+      credentials = credentialsSet.iterator().next();
+    } else {
+      credentials = new Credentials();
+      subject.getPrivateCredentials().add(credentials);
     }
     }
     return credentials;
     return credentials;
   }
   }

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java

@@ -220,7 +220,7 @@ public class TestCredentials {
     for (int i=0; i < service.length; i++) {
     for (int i=0; i < service.length; i++) {
       creds.addToken(service[i], token[i]);
       creds.addToken(service[i], token[i]);
     }
     }
-    creds.addTokensToUGI(ugi);
+    ugi.addCredentials(creds);
 
 
     creds = ugi.getCredentials();
     creds = ugi.getCredentials();
     for (int i=0; i < service.length; i++) {
     for (int i=0; i < service.length; i++) {

+ 73 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -250,6 +250,70 @@ public class TestUserGroupInformation {
     ugi.addToken(t1);
     ugi.addToken(t1);
     checkTokens(ugi, t1, t2, t3);    
     checkTokens(ugi, t1, t2, t3);    
   }
   }
+
+  @SuppressWarnings("unchecked") // from Mockito mocks
+  @Test
+  public <T extends TokenIdentifier> void testGetCreds() throws Exception {
+    UserGroupInformation ugi = 
+        UserGroupInformation.createRemoteUser("someone"); 
+    
+    Text service = new Text("service");
+    Token<T> t1 = mock(Token.class);
+    when(t1.getService()).thenReturn(service);
+    Token<T> t2 = mock(Token.class);
+    when(t2.getService()).thenReturn(new Text("service2"));
+    Token<T> t3 = mock(Token.class);
+    when(t3.getService()).thenReturn(service);
+    
+    // add token to ugi
+    ugi.addToken(t1);
+    ugi.addToken(t2);
+    checkTokens(ugi, t1, t2);
+
+    Credentials creds = ugi.getCredentials();
+    creds.addToken(t3.getService(), t3);
+    assertSame(t3, creds.getToken(service));
+    // check that ugi wasn't modified
+    checkTokens(ugi, t1, t2);
+  }
+
+  @SuppressWarnings("unchecked") // from Mockito mocks
+  @Test
+  public <T extends TokenIdentifier> void testAddCreds() throws Exception {
+    UserGroupInformation ugi = 
+        UserGroupInformation.createRemoteUser("someone"); 
+    
+    Text service = new Text("service");
+    Token<T> t1 = mock(Token.class);
+    when(t1.getService()).thenReturn(service);
+    Token<T> t2 = mock(Token.class);
+    when(t2.getService()).thenReturn(new Text("service2"));
+    byte[] secret = new byte[]{};
+    Text secretKey = new Text("sshhh");
+
+    // fill credentials
+    Credentials creds = new Credentials();
+    creds.addToken(t1.getService(), t1);
+    creds.addToken(t2.getService(), t2);
+    creds.addSecretKey(secretKey, secret);
+    
+    // add creds to ugi, and check ugi
+    ugi.addCredentials(creds);
+    checkTokens(ugi, t1, t2);
+    assertSame(secret, ugi.getCredentials().getSecretKey(secretKey));
+  }
+
+  @SuppressWarnings("unchecked") // from Mockito mocks
+  @Test
+  public <T extends TokenIdentifier> void testGetCredsNotSame()
+      throws Exception {
+    UserGroupInformation ugi = 
+        UserGroupInformation.createRemoteUser("someone"); 
+    Credentials creds = ugi.getCredentials();
+    // should always get a new copy
+    assertNotSame(creds, ugi.getCredentials());
+  }
+
   
   
   private void checkTokens(UserGroupInformation ugi, Token<?> ... tokens) {
   private void checkTokens(UserGroupInformation ugi, Token<?> ... tokens) {
     // check the ugi's token collection
     // check the ugi's token collection
@@ -299,13 +363,22 @@ public class TestUserGroupInformation {
     Token<T> t2 = mock(Token.class);
     Token<T> t2 = mock(Token.class);
     when(t2.getService()).thenReturn(new Text("t2"));
     when(t2.getService()).thenReturn(new Text("t2"));
     
     
+    Credentials creds = new Credentials();
+    byte[] secretKey = new byte[]{};
+    Text secretName = new Text("shhh");
+    creds.addSecretKey(secretName, secretKey);
+    
     ugi.addToken(t1);
     ugi.addToken(t1);
     ugi.addToken(t2);
     ugi.addToken(t2);
+    ugi.addCredentials(creds);
     
     
     Collection<Token<? extends TokenIdentifier>> z = ugi.getTokens();
     Collection<Token<? extends TokenIdentifier>> z = ugi.getTokens();
     assertTrue(z.contains(t1));
     assertTrue(z.contains(t1));
     assertTrue(z.contains(t2));
     assertTrue(z.contains(t2));
     assertEquals(2, z.size());
     assertEquals(2, z.size());
+    Credentials ugiCreds = ugi.getCredentials();
+    assertSame(secretKey, ugiCreds.getSecretKey(secretName));
+    assertEquals(1, ugiCreds.numberOfSecretKeys());
     
     
     try {
     try {
       z.remove(t1);
       z.remove(t1);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java

@@ -141,7 +141,7 @@ class YarnChild {
       childUGI = UserGroupInformation.createRemoteUser(System
       childUGI = UserGroupInformation.createRemoteUser(System
           .getenv(ApplicationConstants.Environment.USER.toString()));
           .getenv(ApplicationConstants.Environment.USER.toString()));
       // Add tokens to new user so that it may execute its task correctly.
       // Add tokens to new user so that it may execute its task correctly.
-      job.getCredentials().addTokensToUGI(childUGI);
+      childUGI.addCredentials(credentials);
 
 
       // Create a final reference to the task for the doAs block
       // Create a final reference to the task for the doAs block
       final Task taskFinal = task;
       final Task taskFinal = task;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -487,7 +487,7 @@ public class MRAppMaster extends CompositeService {
         fsTokens.addAll(Credentials.readTokenStorageFile(jobTokenFile, conf));
         fsTokens.addAll(Credentials.readTokenStorageFile(jobTokenFile, conf));
         LOG.info("jobSubmitDir=" + jobSubmitDir + " jobTokenFile="
         LOG.info("jobSubmitDir=" + jobSubmitDir + " jobTokenFile="
             + jobTokenFile);
             + jobTokenFile);
-        fsTokens.addTokensToUGI(currentUser); // For use by AppMaster itself.
+        currentUser.addCredentials(fsTokens); // For use by AppMaster itself.
       }
       }
     } catch (IOException e) {
     } catch (IOException e) {
       throw new YarnException(e);
       throw new YarnException(e);

+ 12 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJob.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -55,14 +56,20 @@ public class TestJob {
 
 
   @Test
   @Test
   public void testUGICredentialsPropogation() throws Exception {
   public void testUGICredentialsPropogation() throws Exception {
+    Credentials creds = new Credentials();
     Token<?> token = mock(Token.class);
     Token<?> token = mock(Token.class);
-    Text service = new Text("service");
-    
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    ugi.addToken(service, token);
+    Text tokenService = new Text("service");
+    Text secretName = new Text("secret");
+    byte secret[] = new byte[]{};
+        
+    creds.addToken(tokenService,  token);
+    creds.addSecretKey(secretName, secret);
+    UserGroupInformation.getLoginUser().addCredentials(creds);
     
     
     JobConf jobConf = new JobConf();
     JobConf jobConf = new JobConf();
     Job job = new Job(jobConf);
     Job job = new Job(jobConf);
-    assertSame(token, job.getCredentials().getToken(service));
+
+    assertSame(token, job.getCredentials().getToken(tokenService));
+    assertSame(secret, job.getCredentials().getSecretKey(secretName));
   }
   }
 }
 }