Browse Source

MAPREDUCE-3698. Client cannot talk to the history server in secure mode. (mahadev) - Merging r1234120 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1234125 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 13 năm trước cách đây
mục cha
commit
6f27abfe72

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

@@ -469,6 +469,9 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3582. Move successfully passing MR1 tests to MR2 maven tree.
     (ahmed via tucu)
 
+    MAPREDUCE-3698. Client cannot talk to the history server in secure mode.
+    (mahadev)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 45 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/security/authorize/ClientHSPolicyProvider.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.hadoop.mapreduce.v2.app.security.authorize;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.Service;
+import org.apache.hadoop.yarn.proto.HSClientProtocol;
+
+/**
+ * {@link PolicyProvider} for YARN MapReduce protocols.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ClientHSPolicyProvider extends PolicyProvider {
+  
+  private static final Service[] mrHSServices = 
+      new Service[] {
+    new Service(
+        JHAdminConfig.MR_HS_SECURITY_SERVICE_AUTHORIZATION,
+        HSClientProtocol.HSClientProtocolService.BlockingInterface.class)
+  };
+
+  @Override
+  public Service[] getServices() {
+    return mrHSServices;
+  }
+}

+ 8 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/impl/pb/client/HSClientProtocolPBClientImpl.java

@@ -22,13 +22,20 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.mapreduce.v2.api.HSClientProtocol;
+import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
+import org.apache.hadoop.yarn.proto.HSClientProtocol.HSClientProtocolService;
 
 public class HSClientProtocolPBClientImpl extends MRClientProtocolPBClientImpl
   implements HSClientProtocol {
 
   public HSClientProtocolPBClientImpl(long clientVersion,
       InetSocketAddress addr, Configuration conf) throws IOException {
-    super(clientVersion, addr, conf);
+    super();
+    RPC.setProtocolEngine(conf, HSClientProtocolService.BlockingInterface.class,
+        ProtoOverHadoopRpcEngine.class);
+    proxy = (HSClientProtocolService.BlockingInterface)RPC.getProxy(
+        HSClientProtocolService.BlockingInterface.class, clientVersion, addr, conf);
   }
 }

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/impl/pb/client/MRClientProtocolPBClientImpl.java

@@ -93,7 +93,9 @@ import com.google.protobuf.ServiceException;
 
 public class MRClientProtocolPBClientImpl implements MRClientProtocol {
 
-  private MRClientProtocolService.BlockingInterface proxy;
+  protected MRClientProtocolService.BlockingInterface proxy;
+  
+  public MRClientProtocolPBClientImpl() {};
   
   public MRClientProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
     RPC.setProtocolEngine(conf, MRClientProtocolService.BlockingInterface.class, ProtoOverHadoopRpcEngine.class);

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java

@@ -111,4 +111,9 @@ public class JHAdminConfig {
   public static final int DEFAULT_MR_HISTORY_WEBAPP_PORT = 19888;
   public static final String DEFAULT_MR_HISTORY_WEBAPP_ADDRESS =
     "0.0.0.0:" + DEFAULT_MR_HISTORY_WEBAPP_PORT;
+  /*
+   * HS Service Authorization
+   */
+  public static final String MR_HS_SECURITY_SERVICE_AUTHORIZATION =
+      "security.mrhs.client.protocol.acl";
 }

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/client/ClientHSSecurityInfo.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.mapreduce.v2.security.client;
 
 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;
@@ -30,7 +32,7 @@ import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.yarn.proto.HSClientProtocol;
 
 public class ClientHSSecurityInfo extends SecurityInfo {
-  
+    
   @Override
   public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
     if (!protocol

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java

@@ -66,7 +66,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
-import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
+import org.apache.hadoop.mapreduce.v2.app.security.authorize.ClientHSPolicyProvider;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.HsWebApp;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.net.NetUtils;
@@ -136,9 +136,9 @@ public class HistoryClientService extends AbstractService {
     if (conf.getBoolean(
         CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
         false)) {
-      server.refreshServiceAcl(conf, new MRAMPolicyProvider());
+      server.refreshServiceAcl(conf, new ClientHSPolicyProvider());
     }
-
+    
     server.start();
     this.bindAddress =
         NetUtils.createSocketAddr(hostNameResolved.getHostAddress()

+ 0 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java

@@ -67,7 +67,6 @@ public class ProtoOverHadoopRpcEngine implements RpcEngine {
   public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
       InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
       SocketFactory factory, int rpcTimeout) throws IOException {
-
     return new ProtocolProxy<T>(protocol, (T) Proxy.newProxyInstance(protocol
         .getClassLoader(), new Class[] { protocol }, new Invoker(protocol,
         addr, ticket, conf, factory, rpcTimeout)), false);