You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by ss...@apache.org on 2012/03/02 20:28:04 UTC

svn commit: r1296401 - in /hadoop/common/trunk/hadoop-mapreduce-project: ./ hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main...

Author: sseth
Date: Fri Mar  2 19:28:04 2012
New Revision: 1296401

URL: http://svn.apache.org/viewvc?rev=1296401&view=rev
Log:
MAPREDUCE-3896. Add user information to the delegation token issued by the history server. (Contributed by Vinod Kumar Vavilapalli)

Added:
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/krb5.conf
Modified:
    hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
    hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java

Modified: hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt?rev=1296401&r1=1296400&r2=1296401&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt Fri Mar  2 19:28:04 2012
@@ -235,6 +235,9 @@ Release 0.23.2 - UNRELEASED
     MAPREDUCE-3706. Fix circular redirect error in job-attempts page. (bobby
     via acmurthy)
 
+    MAPREDUCE-3896. Add user information to the delegation token issued by the
+    history server. (Vinod Kumar Vavilapalli via sseth)
+
 Release 0.23.1 - 2012-02-17
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java?rev=1296401&r1=1296400&r2=1296401&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java Fri Mar  2 19:28:04 2012
@@ -20,8 +20,6 @@ package org.apache.hadoop.mapreduce.v2.s
 
 import java.lang.annotation.Annotation;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.security.KerberosInfo;

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java?rev=1296401&r1=1296400&r2=1296401&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java Fri Mar  2 19:28:04 2012
@@ -29,8 +29,10 @@ import java.util.Collection;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.TypeConverter;
@@ -121,7 +123,6 @@ public class HistoryClientService extend
     InetAddress hostNameResolved = null;
     try {
       hostNameResolved = InetAddress.getLocalHost(); 
-      //address.getAddress().getLocalHost();
     } catch (UnknownHostException e) {
       throw new YarnException(e);
     }
@@ -166,6 +167,16 @@ public class HistoryClientService extend
     super.stop();
   }
 
+  @Private
+  public MRClientProtocol getClientHandler() {
+    return this.protocolHandler;
+  }
+
+  @Private
+  public InetSocketAddress getBindAddress() {
+    return this.bindAddress;
+  }
+
   private class MRClientProtocolHandler implements MRClientProtocol {
 
     private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
@@ -294,9 +305,12 @@ public class HistoryClientService extend
         GetDelegationTokenRequest request) throws YarnRemoteException {
 
       try {
+
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
       // Verify that the connection is kerberos authenticated
       AuthenticationMethod authMethod = UserGroupInformation
-        .getRealAuthenticationMethod(UserGroupInformation.getCurrentUser());
+        .getRealAuthenticationMethod(ugi);
       if (UserGroupInformation.isSecurityEnabled()
           && (authMethod != AuthenticationMethod.KERBEROS)) {
        throw new IOException(
@@ -305,8 +319,16 @@ public class HistoryClientService extend
 
       GetDelegationTokenResponse response = recordFactory.newRecordInstance(
           GetDelegationTokenResponse.class);
+
+      String user = ugi.getUserName();
+      Text owner = new Text(user);
+      Text realUser = null;
+      if (ugi.getRealUser() != null) {
+        realUser = new Text(ugi.getRealUser().getUserName());
+      }
       MRDelegationTokenIdentifier tokenIdentifier =
-          new MRDelegationTokenIdentifier();
+          new MRDelegationTokenIdentifier(owner, new Text(
+            request.getRenewer()), realUser);
       Token<MRDelegationTokenIdentifier> realJHSToken =
           new Token<MRDelegationTokenIdentifier>(tokenIdentifier,
               jhsDTSecretManager);

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java?rev=1296401&r1=1296400&r2=1296401&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java Fri Mar  2 19:28:04 2012
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.MRConfig;
@@ -107,7 +108,12 @@ public class JobHistoryServer extends Co
     jhsDTSecretManager.stopThreads();
     super.stop();
   }
-  
+
+  @Private
+  public HistoryClientService getClientService() {
+    return this.clientService;
+  }
+
   public static void main(String[] args) {
     StringUtils.startupShutdownMessage(JobHistoryServer.class, args, LOG);
     try {

Added: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java?rev=1296401&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java (added)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java Fri Mar  2 19:28:04 2012
@@ -0,0 +1,120 @@
+/**
+* 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.hadoop.mapreduce.security;
+
+import java.io.IOException;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.v2.api.HSClientProtocol;
+import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
+import org.apache.hadoop.mapreduce.v2.api.MRDelegationTokenIdentifier;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
+import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.junit.Test;
+
+public class TestJHSSecurity {
+
+  @Test
+  public void testDelegationToken() throws IOException, InterruptedException {
+
+    Logger rootLogger = LogManager.getRootLogger();
+    rootLogger.setLevel(Level.DEBUG);
+
+    final YarnConfiguration conf = new YarnConfiguration(new JobConf());
+    // Just a random principle
+    conf.set(JHAdminConfig.MR_HISTORY_PRINCIPAL,
+      "RandomOrc/localhost@apache.org");
+
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+      "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+
+    final JobHistoryServer jobHistoryServer = new JobHistoryServer() {
+      protected void doSecureLogin(Configuration conf) throws IOException {
+        // no keytab based login
+      };
+    };
+    jobHistoryServer.init(conf);
+    jobHistoryServer.start();
+
+    // Fake the authentication-method
+    UserGroupInformation loggedInUser = UserGroupInformation.getCurrentUser();
+    loggedInUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
+
+    // Get the delegation token directly as it is a little difficult to setup
+    // the kerberos based rpc.
+    DelegationToken token =
+        loggedInUser.doAs(new PrivilegedExceptionAction<DelegationToken>() {
+          @Override
+          public DelegationToken run() throws YarnRemoteException {
+            GetDelegationTokenRequest request =
+                Records.newRecord(GetDelegationTokenRequest.class);
+            request.setRenewer("OneRenewerToRuleThemAll");
+            return jobHistoryServer.getClientService().getClientHandler()
+              .getDelegationToken(request).getDelegationToken();
+          }
+        });
+
+    // Now try talking to JHS using the delegation token
+    UserGroupInformation ugi =
+        UserGroupInformation.createRemoteUser("TheDarkLord");
+    ugi.addToken(new Token<MRDelegationTokenIdentifier>(token.getIdentifier()
+      .array(), token.getPassword().array(), new Text(token.getKind()),
+      new Text(token.getService())));
+    final YarnRPC rpc = YarnRPC.create(conf);
+    MRClientProtocol userUsingDT =
+        ugi.doAs(new PrivilegedAction<MRClientProtocol>() {
+          @Override
+          public MRClientProtocol run() {
+            return (MRClientProtocol) rpc.getProxy(HSClientProtocol.class,
+              jobHistoryServer.getClientService().getBindAddress(), conf);
+          }
+        });
+    GetJobReportRequest jobReportRequest =
+        Records.newRecord(GetJobReportRequest.class);
+    jobReportRequest.setJobId(MRBuilderUtils.newJobId(123456, 1, 1));
+    try {
+      userUsingDT.getJobReport(jobReportRequest);
+    } catch (YarnRemoteException e) {
+      Assert.assertEquals("Unknown job job_123456_0001", e.getMessage());
+    }
+  }
+
+}

Added: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/krb5.conf
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/krb5.conf?rev=1296401&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/krb5.conf (added)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/krb5.conf Fri Mar  2 19:28:04 2012
@@ -0,0 +1,28 @@
+#
+# 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.
+# 
+[libdefaults]
+	default_realm = APACHE.ORG
+	udp_preference_limit = 1
+	extra_addresses = 127.0.0.1
+[realms]
+	APACHE.ORG = {
+		admin_server = localhost:88
+		kdc = localhost:88
+	}
+[domain_realm]
+	localhost = APACHE.ORG