REDUCE-3955. Change MR to use ProtobufRpcEngine from hadoop-common instead of ProtoOverHadoopRpcEngine. (Contributed by Jitendra Nath Pandey)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1306689 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
e7444b2a07
commit
db3e592df0
|
@ -133,6 +133,9 @@ Release 2.0.0 - UNRELEASED
|
|||
MAPREDUCE-3353. Add a channel between RM and AM to get information on
|
||||
nodes. (Bikas Saha via acmurthy)
|
||||
|
||||
MAPREDUCE-3955. Change MR to use ProtobufRpcEngine from hadoop-common
|
||||
instead of ProtoOverHadoopRpcEngine. (Jitendra Nath Pandey via sseth)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -21,12 +21,12 @@ package org.apache.hadoop.mapreduce.v2.app;
|
|||
import java.lang.annotation.Annotation;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocolPB;
|
||||
import org.apache.hadoop.security.KerberosInfo;
|
||||
import org.apache.hadoop.security.SecurityInfo;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
import org.apache.hadoop.security.token.TokenSelector;
|
||||
import org.apache.hadoop.yarn.proto.MRClientProtocol;
|
||||
import org.apache.hadoop.yarn.security.client.ClientTokenSelector;
|
||||
|
||||
public class MRClientSecurityInfo extends SecurityInfo {
|
||||
|
@ -38,7 +38,7 @@ public class MRClientSecurityInfo extends SecurityInfo {
|
|||
|
||||
@Override
|
||||
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol.equals(MRClientProtocol.MRClientProtocolService.BlockingInterface.class)) {
|
||||
if (!protocol.equals(MRClientProtocolPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new TokenInfo() {
|
||||
|
|
|
@ -19,10 +19,10 @@ 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.api.HSClientProtocolPB;
|
||||
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.
|
||||
|
@ -35,7 +35,7 @@ public class ClientHSPolicyProvider extends PolicyProvider {
|
|||
new Service[] {
|
||||
new Service(
|
||||
JHAdminConfig.MR_HS_SECURITY_SERVICE_AUTHORIZATION,
|
||||
HSClientProtocol.HSClientProtocolService.BlockingInterface.class)
|
||||
HSClientProtocolPB.class)
|
||||
};
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,9 +21,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocolPB;
|
||||
import org.apache.hadoop.security.authorize.PolicyProvider;
|
||||
import org.apache.hadoop.security.authorize.Service;
|
||||
import org.apache.hadoop.yarn.proto.MRClientProtocol;
|
||||
|
||||
/**
|
||||
* {@link PolicyProvider} for YARN MapReduce protocols.
|
||||
|
@ -39,7 +39,7 @@ public class MRAMPolicyProvider extends PolicyProvider {
|
|||
TaskUmbilicalProtocol.class),
|
||||
new Service(
|
||||
MRJobConfig.MR_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT,
|
||||
MRClientProtocol.MRClientProtocolService.BlockingInterface.class)
|
||||
MRClientProtocolPB.class)
|
||||
};
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* 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.api;
|
||||
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.HSClientProtocol.HSClientProtocolService;
|
||||
|
||||
@ProtocolInfo(protocolName = "org.apache.hadoop.mapreduce.v2.api.HSClientProtocolPB",
|
||||
protocolVersion = 1)
|
||||
public interface HSClientProtocolPB extends HSClientProtocolService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -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.
|
||||
*/
|
||||
package org.apache.hadoop.mapreduce.v2.api;
|
||||
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.MRClientProtocol.MRClientProtocolService;
|
||||
|
||||
@ProtocolInfo(
|
||||
protocolName = "org.apache.hadoop.mapreduce.v2.api.MRClientProtocolPB",
|
||||
protocolVersion = 1)
|
||||
public interface MRClientProtocolPB extends MRClientProtocolService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -22,10 +22,10 @@ import java.io.IOException;
|
|||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
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;
|
||||
import org.apache.hadoop.mapreduce.v2.api.HSClientProtocolPB;
|
||||
|
||||
public class HSClientProtocolPBClientImpl extends MRClientProtocolPBClientImpl
|
||||
implements HSClientProtocol {
|
||||
|
@ -33,9 +33,9 @@ public class HSClientProtocolPBClientImpl extends MRClientProtocolPBClientImpl
|
|||
public HSClientProtocolPBClientImpl(long clientVersion,
|
||||
InetSocketAddress addr, Configuration conf) throws IOException {
|
||||
super();
|
||||
RPC.setProtocolEngine(conf, HSClientProtocolService.BlockingInterface.class,
|
||||
ProtoOverHadoopRpcEngine.class);
|
||||
proxy = (HSClientProtocolService.BlockingInterface)RPC.getProxy(
|
||||
HSClientProtocolService.BlockingInterface.class, clientVersion, addr, conf);
|
||||
RPC.setProtocolEngine(conf, HSClientProtocolPB.class,
|
||||
ProtobufRpcEngine.class);
|
||||
proxy = (HSClientProtocolPB)RPC.getProxy(
|
||||
HSClientProtocolPB.class, clientVersion, addr, conf);
|
||||
}
|
||||
}
|
|
@ -23,8 +23,10 @@ import java.lang.reflect.UndeclaredThrowableException;
|
|||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocolPB;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptResponse;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersRequest;
|
||||
|
@ -86,21 +88,20 @@ import org.apache.hadoop.mapreduce.v2.proto.MRServiceProtos.KillJobRequestProto;
|
|||
import org.apache.hadoop.mapreduce.v2.proto.MRServiceProtos.KillTaskAttemptRequestProto;
|
||||
import org.apache.hadoop.mapreduce.v2.proto.MRServiceProtos.KillTaskRequestProto;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
|
||||
import org.apache.hadoop.yarn.proto.MRClientProtocol.MRClientProtocolService;
|
||||
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
||||
|
||||
protected MRClientProtocolService.BlockingInterface proxy;
|
||||
protected MRClientProtocolPB proxy;
|
||||
|
||||
public MRClientProtocolPBClientImpl() {};
|
||||
|
||||
public MRClientProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, MRClientProtocolService.BlockingInterface.class, ProtoOverHadoopRpcEngine.class);
|
||||
proxy = (MRClientProtocolService.BlockingInterface)RPC.getProxy(
|
||||
MRClientProtocolService.BlockingInterface.class, clientVersion, addr, conf);
|
||||
RPC.setProtocolEngine(conf, MRClientProtocolPB.class, ProtobufRpcEngine.class);
|
||||
proxy = (MRClientProtocolPB)RPC.getProxy(
|
||||
MRClientProtocolPB.class, clientVersion, addr, conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -110,13 +111,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new GetJobReportResponsePBImpl(proxy.getJobReport(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -127,13 +122,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new GetTaskReportResponsePBImpl(proxy.getTaskReport(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -144,13 +133,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new GetTaskAttemptReportResponsePBImpl(proxy.getTaskAttemptReport(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -161,13 +144,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new GetCountersResponsePBImpl(proxy.getCounters(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -178,13 +155,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new GetTaskAttemptCompletionEventsResponsePBImpl(proxy.getTaskAttemptCompletionEvents(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -195,13 +166,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new GetTaskReportsResponsePBImpl(proxy.getTaskReports(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -212,13 +177,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new GetDiagnosticsResponsePBImpl(proxy.getDiagnostics(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -231,13 +190,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
return new GetDelegationTokenResponsePBImpl(proxy.getDelegationToken(
|
||||
null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -248,13 +201,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new KillJobResponsePBImpl(proxy.killJob(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -265,13 +212,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new KillTaskResponsePBImpl(proxy.killTask(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -282,13 +223,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new KillTaskAttemptResponsePBImpl(proxy.killTaskAttempt(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -299,13 +234,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
|
|||
try {
|
||||
return new FailTaskAttemptResponsePBImpl(proxy.failTaskAttempt(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
package org.apache.hadoop.mapreduce.v2.api.impl.pb.service;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.api.HSClientProtocol;
|
||||
import org.apache.hadoop.yarn.proto.HSClientProtocol.HSClientProtocolService.BlockingInterface;
|
||||
import org.apache.hadoop.mapreduce.v2.api.HSClientProtocolPB;
|
||||
|
||||
public class HSClientProtocolPBServiceImpl extends MRClientProtocolPBServiceImpl
|
||||
implements BlockingInterface {
|
||||
implements HSClientProtocolPB {
|
||||
public HSClientProtocolPBServiceImpl(HSClientProtocol impl) {
|
||||
super(impl);
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.mapreduce.v2.api.impl.pb.service;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocolPB;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptResponse;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersRequest;
|
||||
|
@ -91,12 +92,11 @@ import org.apache.hadoop.mapreduce.v2.proto.MRServiceProtos.KillTaskAttemptRespo
|
|||
import org.apache.hadoop.mapreduce.v2.proto.MRServiceProtos.KillTaskRequestProto;
|
||||
import org.apache.hadoop.mapreduce.v2.proto.MRServiceProtos.KillTaskResponseProto;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.proto.MRClientProtocol.MRClientProtocolService.BlockingInterface;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class MRClientProtocolPBServiceImpl implements BlockingInterface {
|
||||
public class MRClientProtocolPBServiceImpl implements MRClientProtocolPB {
|
||||
|
||||
private MRClientProtocol real;
|
||||
|
||||
|
|
|
@ -21,20 +21,20 @@ package org.apache.hadoop.mapreduce.v2.security.client;
|
|||
import java.lang.annotation.Annotation;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapreduce.v2.api.HSClientProtocolPB;
|
||||
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
|
||||
import org.apache.hadoop.security.KerberosInfo;
|
||||
import org.apache.hadoop.security.SecurityInfo;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
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
|
||||
.equals(HSClientProtocol.HSClientProtocolService.BlockingInterface.class)) {
|
||||
.equals(HSClientProtocolPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new KerberosInfo() {
|
||||
|
@ -59,7 +59,7 @@ public class ClientHSSecurityInfo extends SecurityInfo {
|
|||
@Override
|
||||
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol
|
||||
.equals(HSClientProtocol.HSClientProtocolService.BlockingInterface.class)) {
|
||||
.equals(HSClientProtocolPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new TokenInfo() {
|
||||
|
|
|
@ -18,14 +18,23 @@
|
|||
|
||||
package org.apache.hadoop.yarn.proto;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocolPB;
|
||||
import org.apache.hadoop.yarn.proto.MRClientProtocol.MRClientProtocolService;
|
||||
|
||||
/**
|
||||
* Fake protocol to differentiate the blocking interfaces in the
|
||||
* security info class loaders.
|
||||
*/
|
||||
public interface HSClientProtocol {
|
||||
public abstract class HSClientProtocolService {
|
||||
public interface BlockingInterface extends
|
||||
MRClientProtocol.MRClientProtocolService.BlockingInterface {
|
||||
public interface BlockingInterface extends MRClientProtocolPB {
|
||||
}
|
||||
|
||||
public static com.google.protobuf.BlockingService newReflectiveBlockingService(
|
||||
final HSClientProtocolService.BlockingInterface impl) {
|
||||
// The cast is safe
|
||||
return MRClientProtocolService
|
||||
.newReflectiveBlockingService((MRClientProtocolService.BlockingInterface) impl);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.io.Text;
|
|||
import org.apache.hadoop.ipc.Server;
|
||||
import org.apache.hadoop.mapreduce.JobACL;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
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.FailTaskAttemptRequest;
|
||||
|
@ -96,7 +97,7 @@ public class HistoryClientService extends AbstractService {
|
|||
|
||||
private static final Log LOG = LogFactory.getLog(HistoryClientService.class);
|
||||
|
||||
private MRClientProtocol protocolHandler;
|
||||
private HSClientProtocol protocolHandler;
|
||||
private Server server;
|
||||
private WebApp webApp;
|
||||
private InetSocketAddress bindAddress;
|
||||
|
@ -107,7 +108,7 @@ public class HistoryClientService extends AbstractService {
|
|||
JHSDelegationTokenSecretManager jhsDTSecretManager) {
|
||||
super("HistoryClientService");
|
||||
this.history = history;
|
||||
this.protocolHandler = new MRClientProtocolHandler();
|
||||
this.protocolHandler = new HSClientProtocolHandler();
|
||||
this.jhsDTSecretManager = jhsDTSecretManager;
|
||||
}
|
||||
|
||||
|
@ -128,7 +129,7 @@ public class HistoryClientService extends AbstractService {
|
|||
}
|
||||
|
||||
server =
|
||||
rpc.getServer(MRClientProtocol.class, protocolHandler, address,
|
||||
rpc.getServer(HSClientProtocol.class, protocolHandler, address,
|
||||
conf, jhsDTSecretManager,
|
||||
conf.getInt(JHAdminConfig.MR_HISTORY_CLIENT_THREAD_COUNT,
|
||||
JHAdminConfig.DEFAULT_MR_HISTORY_CLIENT_THREAD_COUNT));
|
||||
|
@ -177,7 +178,7 @@ public class HistoryClientService extends AbstractService {
|
|||
return this.bindAddress;
|
||||
}
|
||||
|
||||
private class MRClientProtocolHandler implements MRClientProtocol {
|
||||
private class HSClientProtocolHandler implements HSClientProtocol {
|
||||
|
||||
private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce.Job;
|
|||
import org.apache.hadoop.mapreduce.MRConfig;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
||||
import org.apache.hadoop.mapreduce.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.api.HSClientProtocol;
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptResponse;
|
||||
|
@ -340,9 +341,10 @@ public class TestClientRedirect {
|
|||
}
|
||||
}
|
||||
|
||||
class HistoryService extends AMService {
|
||||
class HistoryService extends AMService implements HSClientProtocol {
|
||||
public HistoryService() {
|
||||
super(HSHOSTADDRESS);
|
||||
this.protocol = HSClientProtocol.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -357,6 +359,7 @@ public class TestClientRedirect {
|
|||
|
||||
class AMService extends AbstractService
|
||||
implements MRClientProtocol {
|
||||
protected Class<?> protocol;
|
||||
private InetSocketAddress bindAddress;
|
||||
private Server server;
|
||||
private final String hostAddress;
|
||||
|
@ -367,6 +370,7 @@ public class TestClientRedirect {
|
|||
|
||||
public AMService(String hostAddress) {
|
||||
super("AMService");
|
||||
this.protocol = MRClientProtocol.class;
|
||||
this.hostAddress = hostAddress;
|
||||
}
|
||||
|
||||
|
@ -383,7 +387,7 @@ public class TestClientRedirect {
|
|||
}
|
||||
|
||||
server =
|
||||
rpc.getServer(MRClientProtocol.class, this, address,
|
||||
rpc.getServer(protocol, this, address,
|
||||
conf, null, 1);
|
||||
server.start();
|
||||
this.bindAddress =
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|||
import org.apache.hadoop.mapreduce.Counters;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
|
||||
import org.apache.hadoop.mapreduce.v2.api.HSClientProtocol;
|
||||
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
|
@ -142,7 +142,7 @@ public class TestMRJobsWithHistoryService {
|
|||
LOG.info("CounterMR " + counterMR);
|
||||
Assert.assertEquals(counterHS, counterMR);
|
||||
|
||||
MRClientProtocol historyClient = instantiateHistoryProxy();
|
||||
HSClientProtocol historyClient = instantiateHistoryProxy();
|
||||
GetJobReportRequest gjReq = Records.newRecord(GetJobReportRequest.class);
|
||||
gjReq.setJobId(jobId);
|
||||
JobReport jobReport = historyClient.getJobReport(gjReq).getJobReport();
|
||||
|
@ -164,12 +164,12 @@ public class TestMRJobsWithHistoryService {
|
|||
&& jobReport.getFinishTime() >= jobReport.getStartTime());
|
||||
}
|
||||
|
||||
private MRClientProtocol instantiateHistoryProxy() {
|
||||
private HSClientProtocol instantiateHistoryProxy() {
|
||||
final String serviceAddr =
|
||||
mrCluster.getConfig().get(JHAdminConfig.MR_HISTORY_ADDRESS);
|
||||
final YarnRPC rpc = YarnRPC.create(conf);
|
||||
MRClientProtocol historyClient =
|
||||
(MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
|
||||
HSClientProtocol historyClient =
|
||||
(HSClientProtocol) rpc.getProxy(HSClientProtocol.class,
|
||||
NetUtils.createSocketAddr(serviceAddr), mrCluster.getConfig());
|
||||
return historyClient;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* 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.yarn.api;
|
||||
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.AMRMProtocol.AMRMProtocolService;
|
||||
|
||||
@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.AMRMProtocolPB",
|
||||
protocolVersion = 1)
|
||||
public interface AMRMProtocolPB extends AMRMProtocolService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* 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.yarn.api;
|
||||
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.ClientRMProtocol.ClientRMProtocolService;
|
||||
|
||||
@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.ClientRMProtocolPB",
|
||||
protocolVersion = 1)
|
||||
public interface ClientRMProtocolPB extends ClientRMProtocolService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -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.
|
||||
*/
|
||||
package org.apache.hadoop.yarn.api;
|
||||
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.ContainerManager.ContainerManagerService;
|
||||
|
||||
@ProtocolInfo(
|
||||
protocolName = "org.apache.hadoop.yarn.api.ContainerManagerPB",
|
||||
protocolVersion = 1)
|
||||
public interface ContainerManagerPB extends ContainerManagerService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -18,12 +18,16 @@
|
|||
|
||||
package org.apache.hadoop.yarn.exceptions.impl.pb;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.YarnRemoteExceptionProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.YarnRemoteExceptionProtoOrBuilder;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class YarnRemoteExceptionPBImpl extends YarnRemoteException {
|
||||
|
||||
|
@ -105,4 +109,30 @@ public class YarnRemoteExceptionPBImpl extends YarnRemoteException {
|
|||
}
|
||||
viaProto = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method that unwraps and throws appropriate exception.
|
||||
* @param se ServiceException
|
||||
* @throws YarnRemoteException
|
||||
* @throws UndeclaredThrowableException
|
||||
*/
|
||||
public static YarnRemoteException unwrapAndThrowException(ServiceException se)
|
||||
throws UndeclaredThrowableException {
|
||||
if (se.getCause() instanceof RemoteException) {
|
||||
try {
|
||||
throw ((RemoteException) se.getCause())
|
||||
.unwrapRemoteException(YarnRemoteExceptionPBImpl.class);
|
||||
} catch (YarnRemoteException ex) {
|
||||
return ex;
|
||||
} catch (IOException e1) {
|
||||
throw new UndeclaredThrowableException(e1);
|
||||
}
|
||||
} else if (se.getCause() instanceof YarnRemoteException) {
|
||||
return (YarnRemoteException)se.getCause();
|
||||
} else if (se.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)se.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(se);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -19,12 +19,13 @@
|
|||
package org.apache.hadoop.yarn.api.impl.pb.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.yarn.api.AMRMProtocol;
|
||||
import org.apache.hadoop.yarn.api.AMRMProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
||||
|
@ -38,8 +39,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMaste
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
|
||||
import org.apache.hadoop.yarn.proto.AMRMProtocol.AMRMProtocolService;
|
||||
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
|
||||
|
@ -48,12 +48,12 @@ import com.google.protobuf.ServiceException;
|
|||
|
||||
public class AMRMProtocolPBClientImpl implements AMRMProtocol {
|
||||
|
||||
private AMRMProtocolService.BlockingInterface proxy;
|
||||
private AMRMProtocolPB proxy;
|
||||
|
||||
public AMRMProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, AMRMProtocolService.BlockingInterface.class, ProtoOverHadoopRpcEngine.class);
|
||||
proxy = (AMRMProtocolService.BlockingInterface)RPC.getProxy(
|
||||
AMRMProtocolService.BlockingInterface.class, clientVersion, addr, conf);
|
||||
RPC.setProtocolEngine(conf, AMRMProtocolPB.class, ProtobufRpcEngine.class);
|
||||
proxy = (AMRMProtocolPB)RPC.getProxy(
|
||||
AMRMProtocolPB.class, clientVersion, addr, conf);
|
||||
}
|
||||
|
||||
|
||||
|
@ -64,13 +64,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol {
|
|||
try {
|
||||
return new AllocateResponsePBImpl(proxy.allocate(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -83,13 +77,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol {
|
|||
try {
|
||||
return new FinishApplicationMasterResponsePBImpl(proxy.finishApplicationMaster(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -100,13 +88,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol {
|
|||
try {
|
||||
return new RegisterApplicationMasterResponsePBImpl(proxy.registerApplicationMaster(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,12 +19,13 @@
|
|||
package org.apache.hadoop.yarn.api.impl.pb.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.yarn.api.ClientRMProtocol;
|
||||
import org.apache.hadoop.yarn.api.ClientRMProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
|
||||
|
@ -66,8 +67,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRespons
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
|
||||
import org.apache.hadoop.yarn.proto.ClientRMProtocol.ClientRMProtocolService;
|
||||
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterMetricsRequestProto;
|
||||
|
@ -83,12 +83,12 @@ import com.google.protobuf.ServiceException;
|
|||
|
||||
public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
||||
|
||||
private ClientRMProtocolService.BlockingInterface proxy;
|
||||
private ClientRMProtocolPB proxy;
|
||||
|
||||
public ClientRMProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, ClientRMProtocolService.BlockingInterface.class, ProtoOverHadoopRpcEngine.class);
|
||||
proxy = (ClientRMProtocolService.BlockingInterface)RPC.getProxy(
|
||||
ClientRMProtocolService.BlockingInterface.class, clientVersion, addr, conf);
|
||||
RPC.setProtocolEngine(conf, ClientRMProtocolPB.class, ProtobufRpcEngine.class);
|
||||
proxy = (ClientRMProtocolPB)RPC.getProxy(
|
||||
ClientRMProtocolPB.class, clientVersion, addr, conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -98,13 +98,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
try {
|
||||
return new KillApplicationResponsePBImpl(proxy.forceKillApplication(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -115,13 +109,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
try {
|
||||
return new GetApplicationReportResponsePBImpl(proxy.getApplicationReport(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -132,13 +120,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
try {
|
||||
return new GetClusterMetricsResponsePBImpl(proxy.getClusterMetrics(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -149,13 +131,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
try {
|
||||
return new GetNewApplicationResponsePBImpl(proxy.getNewApplication(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -166,13 +142,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
try {
|
||||
return new SubmitApplicationResponsePBImpl(proxy.submitApplication(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -185,13 +155,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
return new GetAllApplicationsResponsePBImpl(
|
||||
proxy.getAllApplications(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -204,13 +168,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
return new GetClusterNodesResponsePBImpl(
|
||||
proxy.getClusterNodes(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -223,13 +181,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
return new GetQueueInfoResponsePBImpl(
|
||||
proxy.getQueueInfo(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -242,13 +194,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
return new GetQueueUserAclsInfoResponsePBImpl(
|
||||
proxy.getQueueUserAcls(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -261,13 +207,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol {
|
|||
return new GetDelegationTokenResponsePBImpl(
|
||||
proxy.getDelegationToken(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,12 +19,13 @@
|
|||
package org.apache.hadoop.yarn.api.impl.pb.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.yarn.api.ContainerManager;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagerPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
||||
|
@ -38,8 +39,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponse
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
|
||||
import org.apache.hadoop.yarn.proto.ContainerManager.ContainerManagerService;
|
||||
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto;
|
||||
|
@ -48,12 +48,12 @@ import com.google.protobuf.ServiceException;
|
|||
|
||||
public class ContainerManagerPBClientImpl implements ContainerManager {
|
||||
|
||||
private ContainerManagerService.BlockingInterface proxy;
|
||||
private ContainerManagerPB proxy;
|
||||
|
||||
public ContainerManagerPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, ContainerManagerService.BlockingInterface.class, ProtoOverHadoopRpcEngine.class);
|
||||
proxy = (ContainerManagerService.BlockingInterface)RPC.getProxy(
|
||||
ContainerManagerService.BlockingInterface.class, clientVersion, addr, conf);
|
||||
RPC.setProtocolEngine(conf, ContainerManagerPB.class, ProtobufRpcEngine.class);
|
||||
proxy = (ContainerManagerPB)RPC.getProxy(
|
||||
ContainerManagerPB.class, clientVersion, addr, conf);
|
||||
}
|
||||
|
||||
public void close() {
|
||||
|
@ -69,13 +69,7 @@ public class ContainerManagerPBClientImpl implements ContainerManager {
|
|||
try {
|
||||
return new GetContainerStatusResponsePBImpl(proxy.getContainerStatus(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -86,31 +80,20 @@ public class ContainerManagerPBClientImpl implements ContainerManager {
|
|||
try {
|
||||
return new StartContainerResponsePBImpl(proxy.startContainer(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StopContainerResponse stopContainer(StopContainerRequest request)
|
||||
throws YarnRemoteException {
|
||||
StopContainerRequestProto requestProto = ((StopContainerRequestPBImpl)request).getProto();
|
||||
StopContainerRequestProto requestProto = ((StopContainerRequestPBImpl) request)
|
||||
.getProto();
|
||||
try {
|
||||
return new StopContainerResponsePBImpl(proxy.stopContainer(null, requestProto));
|
||||
return new StopContainerResponsePBImpl(proxy.stopContainer(null,
|
||||
requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.yarn.api.impl.pb.service;
|
||||
|
||||
import org.apache.hadoop.yarn.api.AMRMProtocol;
|
||||
import org.apache.hadoop.yarn.api.AMRMProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||
|
@ -29,7 +30,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMaste
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.proto.AMRMProtocol.AMRMProtocolService.BlockingInterface;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
|
||||
|
@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterR
|
|||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class AMRMProtocolPBServiceImpl implements BlockingInterface {
|
||||
public class AMRMProtocolPBServiceImpl implements AMRMProtocolPB {
|
||||
|
||||
private AMRMProtocol real;
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.yarn.api.impl.pb.service;
|
||||
|
||||
import org.apache.hadoop.yarn.api.ClientRMProtocol;
|
||||
import org.apache.hadoop.yarn.api.ClientRMProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
|
||||
|
@ -50,7 +51,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRespons
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.proto.ClientRMProtocol.ClientRMProtocolService.BlockingInterface;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto;
|
||||
|
@ -75,7 +75,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseP
|
|||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class ClientRMProtocolPBServiceImpl implements BlockingInterface {
|
||||
public class ClientRMProtocolPBServiceImpl implements ClientRMProtocolPB {
|
||||
|
||||
private ClientRMProtocol real;
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.yarn.api.impl.pb.service;
|
||||
|
||||
import org.apache.hadoop.yarn.api.ContainerManager;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagerPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
|
||||
|
@ -29,7 +30,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponse
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.proto.ContainerManager.ContainerManagerService.BlockingInterface;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
|
||||
|
@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerResponseProto
|
|||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class ContainerManagerPBServiceImpl implements BlockingInterface {
|
||||
public class ContainerManagerPBServiceImpl implements ContainerManagerPB {
|
||||
|
||||
private ContainerManager real;
|
||||
|
||||
|
|
|
@ -26,19 +26,23 @@ import java.net.InetSocketAddress;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
|
||||
import org.apache.hadoop.ipc.Server;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.security.token.SecretManager;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.yarn.YarnException;
|
||||
import org.apache.hadoop.yarn.factories.RpcServerFactory;
|
||||
import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
|
||||
|
||||
import com.google.protobuf.BlockingService;
|
||||
|
||||
public class RpcServerFactoryPBImpl implements RpcServerFactory {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(RpcServerFactoryPBImpl.class);
|
||||
private static final String PROTO_GEN_PACKAGE_NAME = "org.apache.hadoop.yarn.proto";
|
||||
private static final String PROTO_GEN_CLASS_SUFFIX = "Service";
|
||||
private static final String PB_IMPL_PACKAGE_SUFFIX = "impl.pb.service";
|
||||
|
@ -96,6 +100,7 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
|
|||
throw new YarnException(e);
|
||||
}
|
||||
|
||||
Class<?> pbProtocol = service.getClass().getInterfaces()[0];
|
||||
Method method = protoCache.get(protocol);
|
||||
if (method == null) {
|
||||
Class<?> protoClazz = null;
|
||||
|
@ -106,7 +111,8 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
|
|||
+ getProtoClassName(protocol) + "]", e);
|
||||
}
|
||||
try {
|
||||
method = protoClazz.getMethod("newReflectiveBlockingService", service.getClass().getInterfaces()[0]);
|
||||
method = protoClazz.getMethod("newReflectiveBlockingService",
|
||||
pbProtocol.getInterfaces()[0]);
|
||||
method.setAccessible(true);
|
||||
protoCache.putIfAbsent(protocol, method);
|
||||
} catch (NoSuchMethodException e) {
|
||||
|
@ -115,7 +121,7 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
|
|||
}
|
||||
|
||||
try {
|
||||
return createServer(addr, conf, secretManager, numHandlers,
|
||||
return createServer(pbProtocol, addr, conf, secretManager, numHandlers,
|
||||
(BlockingService)method.invoke(null, service));
|
||||
} catch (InvocationTargetException e) {
|
||||
throw new YarnException(e);
|
||||
|
@ -148,13 +154,15 @@ public class RpcServerFactoryPBImpl implements RpcServerFactory {
|
|||
return clazz.getPackage().getName();
|
||||
}
|
||||
|
||||
private Server createServer(InetSocketAddress addr, Configuration conf,
|
||||
private Server createServer(Class<?> pbProtocol, InetSocketAddress addr, Configuration conf,
|
||||
SecretManager<? extends TokenIdentifier> secretManager, int numHandlers,
|
||||
BlockingService blockingService) throws IOException {
|
||||
RPC.setProtocolEngine(conf, BlockingService.class, ProtoOverHadoopRpcEngine.class);
|
||||
Server server = RPC.getServer(BlockingService.class, blockingService,
|
||||
RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine.class);
|
||||
RPC.Server server = RPC.getServer(pbProtocol, blockingService,
|
||||
addr.getHostName(), addr.getPort(), numHandlers, false, conf,
|
||||
secretManager);
|
||||
LOG.info("Adding protocol "+pbProtocol.getCanonicalName()+" to the server");
|
||||
server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, pbProtocol, blockingService);
|
||||
return server;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,404 +0,0 @@
|
|||
/**
|
||||
* 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.yarn.ipc;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationHandler;
|
||||
import java.lang.reflect.Method;
|
||||
import java.lang.reflect.Proxy;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import javax.net.SocketFactory;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.DataOutputOutputStream;
|
||||
import org.apache.hadoop.io.ObjectWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.ipc.Client;
|
||||
import org.apache.hadoop.ipc.ProtocolMetaInfoPB;
|
||||
import org.apache.hadoop.ipc.ProtocolProxy;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.ipc.RpcEngine;
|
||||
import org.apache.hadoop.ipc.ClientCache;
|
||||
import org.apache.hadoop.ipc.Client.ConnectionId;
|
||||
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.SecretManager;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.util.ProtoUtil;
|
||||
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.ipc.RpcProtos.ProtoSpecificRpcRequest;
|
||||
import org.apache.hadoop.yarn.ipc.RpcProtos.ProtoSpecificRpcResponse;
|
||||
|
||||
import com.google.protobuf.BlockingService;
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
|
||||
|
||||
@InterfaceStability.Evolving
|
||||
public class ProtoOverHadoopRpcEngine implements RpcEngine {
|
||||
private static final Log LOG = LogFactory.getLog(RPC.class);
|
||||
|
||||
private static final ClientCache CLIENTS=new ClientCache();
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
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);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProtocolProxy<ProtocolMetaInfoPB> getProtocolMetaInfoProxy(
|
||||
ConnectionId connId, Configuration conf, SocketFactory factory)
|
||||
throws IOException {
|
||||
Class<ProtocolMetaInfoPB> protocol = ProtocolMetaInfoPB.class;
|
||||
return new ProtocolProxy<ProtocolMetaInfoPB>(protocol,
|
||||
(ProtocolMetaInfoPB) Proxy.newProxyInstance(protocol.getClassLoader(),
|
||||
new Class[] { protocol }, new Invoker(protocol, connId, conf,
|
||||
factory)), false);
|
||||
}
|
||||
|
||||
private static class Invoker implements InvocationHandler, Closeable {
|
||||
private Map<String, Message> returnTypes = new ConcurrentHashMap<String, Message>();
|
||||
private boolean isClosed = false;
|
||||
private Client.ConnectionId remoteId;
|
||||
private Client client;
|
||||
|
||||
public Invoker(Class<?> protocol, InetSocketAddress addr,
|
||||
UserGroupInformation ticket, Configuration conf, SocketFactory factory,
|
||||
int rpcTimeout) throws IOException {
|
||||
this(protocol, Client.ConnectionId.getConnectionId(addr, protocol,
|
||||
ticket, rpcTimeout, conf), conf, factory);
|
||||
}
|
||||
|
||||
public Invoker(Class<?> protocol, Client.ConnectionId connId,
|
||||
Configuration conf, SocketFactory factory) {
|
||||
this.remoteId = connId;
|
||||
this.client = CLIENTS.getClient(conf, factory,
|
||||
ProtoSpecificResponseWritable.class);
|
||||
}
|
||||
|
||||
private ProtoSpecificRpcRequest constructRpcRequest(Method method,
|
||||
Object[] params) throws ServiceException {
|
||||
ProtoSpecificRpcRequest rpcRequest;
|
||||
ProtoSpecificRpcRequest.Builder builder;
|
||||
|
||||
builder = ProtoSpecificRpcRequest.newBuilder();
|
||||
builder.setMethodName(method.getName());
|
||||
|
||||
if (params.length != 2) { // RpcController + Message
|
||||
throw new ServiceException("Too many parameters for request. Method: ["
|
||||
+ method.getName() + "]" + ", Expected: 2, Actual: "
|
||||
+ params.length);
|
||||
}
|
||||
if (params[1] == null) {
|
||||
throw new ServiceException("null param while calling Method: ["
|
||||
+ method.getName() + "]");
|
||||
}
|
||||
|
||||
Message param = (Message) params[1];
|
||||
builder.setRequestProto(param.toByteString());
|
||||
|
||||
rpcRequest = builder.build();
|
||||
return rpcRequest;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object invoke(Object proxy, Method method, Object[] args)
|
||||
throws Throwable {
|
||||
long startTime = 0;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
startTime = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
ProtoSpecificRpcRequest rpcRequest = constructRpcRequest(method, args);
|
||||
ProtoSpecificResponseWritable val = null;
|
||||
try {
|
||||
val = (ProtoSpecificResponseWritable) client.call(
|
||||
new ProtoSpecificRequestWritable(rpcRequest), remoteId);
|
||||
} catch (Exception e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
|
||||
ProtoSpecificRpcResponse response = val.message;
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
long callTime = System.currentTimeMillis() - startTime;
|
||||
LOG.debug("Call: " + method.getName() + " " + callTime);
|
||||
}
|
||||
|
||||
if (response.hasIsError() && response.getIsError() == true) {
|
||||
YarnRemoteExceptionPBImpl exception = new YarnRemoteExceptionPBImpl(response.getException());
|
||||
exception.fillInStackTrace();
|
||||
ServiceException se = new ServiceException(exception);
|
||||
throw se;
|
||||
}
|
||||
|
||||
Message prototype = null;
|
||||
try {
|
||||
prototype = getReturnProtoType(method);
|
||||
} catch (Exception e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
Message actualReturnMessage = prototype.newBuilderForType()
|
||||
.mergeFrom(response.getResponseProto()).build();
|
||||
return actualReturnMessage;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (!isClosed) {
|
||||
isClosed = true;
|
||||
CLIENTS.stopClient(client);
|
||||
}
|
||||
}
|
||||
|
||||
private Message getReturnProtoType(Method method) throws Exception {
|
||||
if (returnTypes.containsKey(method.getName())) {
|
||||
return returnTypes.get(method.getName());
|
||||
} else {
|
||||
Class<?> returnType = method.getReturnType();
|
||||
|
||||
Method newInstMethod = returnType.getMethod("getDefaultInstance");
|
||||
newInstMethod.setAccessible(true);
|
||||
Message prototype = (Message) newInstMethod.invoke(null,
|
||||
(Object[]) null);
|
||||
returnTypes.put(method.getName(), prototype);
|
||||
return prototype;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writable Wrapper for Protocol Buffer Requests
|
||||
*/
|
||||
private static class ProtoSpecificRequestWritable implements Writable {
|
||||
ProtoSpecificRpcRequest message;
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public ProtoSpecificRequestWritable() {
|
||||
}
|
||||
|
||||
ProtoSpecificRequestWritable(ProtoSpecificRpcRequest message) {
|
||||
this.message = message;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
((Message)message).writeDelimitedTo(
|
||||
DataOutputOutputStream.constructOutputStream(out));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
int length = ProtoUtil.readRawVarint32(in);
|
||||
byte[] bytes = new byte[length];
|
||||
in.readFully(bytes);
|
||||
message = ProtoSpecificRpcRequest.parseFrom(bytes);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writable Wrapper for Protocol Buffer Responses
|
||||
*/
|
||||
public static class ProtoSpecificResponseWritable implements Writable {
|
||||
ProtoSpecificRpcResponse message;
|
||||
|
||||
public ProtoSpecificResponseWritable() {
|
||||
}
|
||||
|
||||
public ProtoSpecificResponseWritable(ProtoSpecificRpcResponse message) {
|
||||
this.message = message;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
((Message)message).writeDelimitedTo(
|
||||
DataOutputOutputStream.constructOutputStream(out));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
int length = ProtoUtil.readRawVarint32(in);
|
||||
byte[] bytes = new byte[length];
|
||||
in.readFully(bytes);
|
||||
message = ProtoSpecificRpcResponse.parseFrom(bytes);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object[] call(Method method, Object[][] params,
|
||||
InetSocketAddress[] addrs, UserGroupInformation ticket, Configuration conf)
|
||||
throws IOException, InterruptedException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
// for unit testing only
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
static Client getClient(Configuration conf) {
|
||||
return CLIENTS.getClient(conf, SocketFactory.getDefault(),
|
||||
ProtoSpecificResponseWritable.class);
|
||||
}
|
||||
|
||||
public static class Server extends RPC.Server {
|
||||
|
||||
private BlockingService service;
|
||||
private boolean verbose;
|
||||
//
|
||||
// /**
|
||||
// * Construct an RPC server.
|
||||
// *
|
||||
// * @param instance
|
||||
// * the instance whose methods will be called
|
||||
// * @param conf
|
||||
// * the configuration to use
|
||||
// * @param bindAddress
|
||||
// * the address to bind on to listen for connection
|
||||
// * @param port
|
||||
// * the port to listen for connections on
|
||||
// */
|
||||
// public Server(Object instance, Configuration conf, String bindAddress,
|
||||
// int port) throws IOException {
|
||||
// this(instance, conf, bindAddress, port, 1, false, null);
|
||||
// }
|
||||
|
||||
private static String classNameBase(String className) {
|
||||
String[] names = className.split("\\.", -1);
|
||||
if (names == null || names.length == 0) {
|
||||
return className;
|
||||
}
|
||||
return names[names.length - 1];
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct an RPC server.
|
||||
*
|
||||
* @param instance
|
||||
* the instance whose methods will be called
|
||||
* @param conf
|
||||
* the configuration to use
|
||||
* @param bindAddress
|
||||
* the address to bind on to listen for connection
|
||||
* @param port
|
||||
* the port to listen for connections on
|
||||
* @param numHandlers
|
||||
* the number of method handler threads to run
|
||||
* @param verbose
|
||||
* whether each call should be logged
|
||||
*/
|
||||
public Server(Object instance, Configuration conf, String bindAddress,
|
||||
int port, int numHandlers, int numReaders,
|
||||
int queueSizePerHandler, boolean verbose,
|
||||
SecretManager<? extends TokenIdentifier> secretManager)
|
||||
throws IOException {
|
||||
super(bindAddress, port, ProtoSpecificRequestWritable.class, numHandlers,
|
||||
numReaders, queueSizePerHandler, conf, classNameBase(instance.getClass().getName()), secretManager);
|
||||
this.service = (BlockingService) instance;
|
||||
this.verbose = verbose;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writable call(RpcKind rpcKind, String protocol,
|
||||
Writable writableRequest, long receiveTime) throws IOException {
|
||||
ProtoSpecificRequestWritable request = (ProtoSpecificRequestWritable) writableRequest;
|
||||
ProtoSpecificRpcRequest rpcRequest = request.message;
|
||||
String methodName = rpcRequest.getMethodName();
|
||||
if (verbose) {
|
||||
log("Call: protocol=" + protocol + ", method="
|
||||
+ methodName);
|
||||
}
|
||||
MethodDescriptor methodDescriptor = service.getDescriptorForType()
|
||||
.findMethodByName(methodName);
|
||||
if (methodDescriptor == null) {
|
||||
String msg = "Unknown method " + methodName + " called on "
|
||||
+ protocol + " protocol.";
|
||||
LOG.warn(msg);
|
||||
return handleException(new IOException(msg));
|
||||
}
|
||||
Message prototype = service.getRequestPrototype(methodDescriptor);
|
||||
Message param = prototype.newBuilderForType()
|
||||
.mergeFrom(rpcRequest.getRequestProto()).build();
|
||||
Message result;
|
||||
try {
|
||||
result = service.callBlockingMethod(methodDescriptor, null, param);
|
||||
} catch (ServiceException e) {
|
||||
e.printStackTrace();
|
||||
return handleException(e);
|
||||
} catch (Exception e) {
|
||||
return handleException(e);
|
||||
}
|
||||
|
||||
ProtoSpecificRpcResponse response = constructProtoSpecificRpcSuccessResponse(result);
|
||||
return new ProtoSpecificResponseWritable(response);
|
||||
}
|
||||
|
||||
private ProtoSpecificResponseWritable handleException(Throwable e) {
|
||||
ProtoSpecificRpcResponse.Builder builder = ProtoSpecificRpcResponse
|
||||
.newBuilder();
|
||||
builder.setIsError(true);
|
||||
if (e.getCause() instanceof YarnRemoteExceptionPBImpl) {
|
||||
builder.setException(((YarnRemoteExceptionPBImpl) e.getCause())
|
||||
.getProto());
|
||||
} else {
|
||||
builder.setException(new YarnRemoteExceptionPBImpl(e).getProto());
|
||||
}
|
||||
ProtoSpecificRpcResponse response = builder.build();
|
||||
return new ProtoSpecificResponseWritable(response);
|
||||
}
|
||||
|
||||
private ProtoSpecificRpcResponse constructProtoSpecificRpcSuccessResponse(
|
||||
Message message) {
|
||||
ProtoSpecificRpcResponse res = ProtoSpecificRpcResponse.newBuilder()
|
||||
.setResponseProto(message.toByteString()).build();
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
private static void log(String value) {
|
||||
if (value != null && value.length() > 55)
|
||||
value = value.substring(0, 55) + "...";
|
||||
LOG.info(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RPC.Server getServer(Class<?> protocol, Object instance,
|
||||
String bindAddress, int port, int numHandlers,int numReaders,
|
||||
int queueSizePerHandler, boolean verbose,
|
||||
Configuration conf, SecretManager<? extends TokenIdentifier> secretManager)
|
||||
throws IOException {
|
||||
return new Server(instance, conf, bindAddress, port, numHandlers, numReaders, queueSizePerHandler,
|
||||
verbose, secretManager);
|
||||
}
|
||||
}
|
|
@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo;
|
|||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
import org.apache.hadoop.security.token.TokenSelector;
|
||||
import org.apache.hadoop.yarn.proto.ContainerManager;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagerPB;
|
||||
|
||||
public class ContainerManagerSecurityInfo extends SecurityInfo {
|
||||
|
||||
|
@ -38,7 +38,7 @@ public class ContainerManagerSecurityInfo extends SecurityInfo {
|
|||
@Override
|
||||
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol
|
||||
.equals(ContainerManager.ContainerManagerService.BlockingInterface.class)) {
|
||||
.equals(ContainerManagerPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new TokenInfo() {
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo;
|
|||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
import org.apache.hadoop.security.token.TokenSelector;
|
||||
import org.apache.hadoop.yarn.proto.AMRMProtocol;
|
||||
import org.apache.hadoop.yarn.api.AMRMProtocolPB;
|
||||
|
||||
public class SchedulerSecurityInfo extends SecurityInfo {
|
||||
|
||||
|
@ -37,7 +37,7 @@ public class SchedulerSecurityInfo extends SecurityInfo {
|
|||
|
||||
@Override
|
||||
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol.equals(AMRMProtocol.AMRMProtocolService.BlockingInterface.class)) {
|
||||
if (!protocol.equals(AMRMProtocolPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new TokenInfo() {
|
||||
|
|
|
@ -26,15 +26,15 @@ import org.apache.hadoop.security.SecurityInfo;
|
|||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
import org.apache.hadoop.security.token.TokenSelector;
|
||||
import org.apache.hadoop.yarn.api.ClientRMProtocolPB;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.proto.ClientRMProtocol;
|
||||
|
||||
public class ClientRMSecurityInfo extends SecurityInfo {
|
||||
|
||||
@Override
|
||||
public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol
|
||||
.equals(ClientRMProtocol.ClientRMProtocolService.BlockingInterface.class)) {
|
||||
.equals(ClientRMProtocolPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new KerberosInfo() {
|
||||
|
@ -59,7 +59,7 @@ public class ClientRMSecurityInfo extends SecurityInfo {
|
|||
@Override
|
||||
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol
|
||||
.equals(ClientRMProtocol.ClientRMProtocolService.BlockingInterface.class)) {
|
||||
.equals(ClientRMProtocolPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new TokenInfo() {
|
||||
|
|
|
@ -22,11 +22,14 @@ import java.net.InetSocketAddress;
|
|||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.hadoop.ipc.Server;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.ipc.Server;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.yarn.api.ClientRMProtocol;
|
||||
import org.apache.hadoop.yarn.api.ContainerManager;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagerPB;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
|
||||
|
@ -84,6 +87,8 @@ public class TestRPC {
|
|||
"Unknown method getNewApplication called on.*"
|
||||
+ "org.apache.hadoop.yarn.proto.ClientRMProtocol"
|
||||
+ "\\$ClientRMProtocolService\\$BlockingInterface protocol."));
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -101,6 +106,7 @@ public class TestRPC {
|
|||
Server server = rpc.getServer(ContainerManager.class,
|
||||
new DummyContainerManager(), addr, conf, null, 1);
|
||||
server.start();
|
||||
RPC.setProtocolEngine(conf, ContainerManagerPB.class, ProtobufRpcEngine.class);
|
||||
ContainerManager proxy = (ContainerManager)
|
||||
rpc.getProxy(ContainerManager.class,
|
||||
NetUtils.createSocketAddr("localhost:" + server.getPort()), conf);
|
||||
|
@ -144,11 +150,11 @@ public class TestRPC {
|
|||
proxy.stopContainer(stopRequest);
|
||||
} catch (YarnRemoteException e) {
|
||||
exception = true;
|
||||
System.err.println(e.getMessage());
|
||||
System.err.println(e.getCause().getMessage());
|
||||
Assert.assertTrue(EXCEPTION_MSG.equals(e.getMessage()));
|
||||
Assert.assertTrue(EXCEPTION_CAUSE.equals(e.getCause().getMessage()));
|
||||
Assert.assertTrue(e.getMessage().contains(EXCEPTION_MSG));
|
||||
Assert.assertTrue(e.getMessage().contains(EXCEPTION_CAUSE));
|
||||
System.out.println("Test Exception is " + RPCUtil.toString(e));
|
||||
} catch (Exception ex) {
|
||||
ex.printStackTrace();
|
||||
}
|
||||
Assert.assertTrue(exception);
|
||||
|
||||
|
|
|
@ -25,13 +25,13 @@ import org.apache.hadoop.security.KerberosInfo;
|
|||
import org.apache.hadoop.security.SecurityInfo;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.proto.ResourceTracker;
|
||||
import org.apache.hadoop.yarn.server.api.ResourceTrackerPB;
|
||||
|
||||
public class RMNMSecurityInfoClass extends SecurityInfo {
|
||||
|
||||
@Override
|
||||
public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol.equals(ResourceTracker.ResourceTrackerService.BlockingInterface.class)) {
|
||||
if (!protocol.equals(ResourceTrackerPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new KerberosInfo() {
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
package org.apache.hadoop.yarn.server.api;
|
||||
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.ResourceTracker.ResourceTrackerService;
|
||||
|
||||
@ProtocolInfo(
|
||||
protocolName = "org.apache.hadoop.yarn.server.api.ResourceTrackerPB",
|
||||
protocolVersion = 1)
|
||||
public interface ResourceTrackerPB extends ResourceTrackerService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -19,17 +19,17 @@
|
|||
package org.apache.hadoop.yarn.server.api.impl.pb.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
|
||||
import org.apache.hadoop.yarn.proto.ResourceTracker.ResourceTrackerService;
|
||||
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerRequestProto;
|
||||
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
||||
import org.apache.hadoop.yarn.server.api.ResourceTrackerPB;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
||||
|
@ -43,12 +43,12 @@ import com.google.protobuf.ServiceException;
|
|||
|
||||
public class ResourceTrackerPBClientImpl implements ResourceTracker {
|
||||
|
||||
private ResourceTrackerService.BlockingInterface proxy;
|
||||
private ResourceTrackerPB proxy;
|
||||
|
||||
public ResourceTrackerPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, ResourceTrackerService.BlockingInterface.class, ProtoOverHadoopRpcEngine.class);
|
||||
proxy = (ResourceTrackerService.BlockingInterface)RPC.getProxy(
|
||||
ResourceTrackerService.BlockingInterface.class, clientVersion, addr, conf);
|
||||
RPC.setProtocolEngine(conf, ResourceTrackerPB.class, ProtobufRpcEngine.class);
|
||||
proxy = (ResourceTrackerPB)RPC.getProxy(
|
||||
ResourceTrackerPB.class, clientVersion, addr, conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -58,13 +58,7 @@ private ResourceTrackerService.BlockingInterface proxy;
|
|||
try {
|
||||
return new RegisterNodeManagerResponsePBImpl(proxy.registerNodeManager(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -75,13 +69,7 @@ private ResourceTrackerService.BlockingInterface proxy;
|
|||
try {
|
||||
return new NodeHeartbeatResponsePBImpl(proxy.nodeHeartbeat(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
package org.apache.hadoop.yarn.server.api.impl.pb.service;
|
||||
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.proto.ResourceTracker.ResourceTrackerService.BlockingInterface;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerResponseProto;
|
||||
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
||||
import org.apache.hadoop.yarn.server.api.ResourceTrackerPB;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeHeartbeatRequestPBImpl;
|
||||
|
@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterNodeMan
|
|||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class ResourceTrackerPBServiceImpl implements BlockingInterface {
|
||||
public class ResourceTrackerPBServiceImpl implements ResourceTrackerPB {
|
||||
|
||||
private ResourceTracker real;
|
||||
|
||||
|
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* 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.yarn.server.nodemanager.api;
|
||||
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.LocalizationProtocol.LocalizationProtocolService;
|
||||
|
||||
@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB",
|
||||
protocolVersion = 1)
|
||||
public interface LocalizationProtocolPB extends LocalizationProtocolService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -18,32 +18,31 @@
|
|||
package org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.client;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
|
||||
import org.apache.hadoop.yarn.proto.LocalizationProtocol.LocalizationProtocolService;
|
||||
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerStatusProto;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalizerHeartbeatResponsePBImpl;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalizerStatusPBImpl;
|
||||
import static org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerStatusProto;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class LocalizationProtocolPBClientImpl implements LocalizationProtocol {
|
||||
|
||||
private LocalizationProtocolService.BlockingInterface proxy;
|
||||
private LocalizationProtocolPB proxy;
|
||||
|
||||
public LocalizationProtocolPBClientImpl(long clientVersion, InetSocketAddress addr, Configuration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, LocalizationProtocolService.BlockingInterface.class, ProtoOverHadoopRpcEngine.class);
|
||||
proxy = (LocalizationProtocolService.BlockingInterface)RPC.getProxy(
|
||||
LocalizationProtocolService.BlockingInterface.class, clientVersion, addr, conf);
|
||||
RPC.setProtocolEngine(conf, LocalizationProtocolPB.class, ProtobufRpcEngine.class);
|
||||
proxy = (LocalizationProtocolPB)RPC.getProxy(
|
||||
LocalizationProtocolPB.class, clientVersion, addr, conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -54,13 +53,7 @@ public class LocalizationProtocolPBClientImpl implements LocalizationProtocol {
|
|||
return new LocalizerHeartbeatResponsePBImpl(
|
||||
proxy.heartbeat(null, statusProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -24,13 +24,13 @@ import com.google.protobuf.RpcController;
|
|||
import com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.proto.LocalizationProtocol.LocalizationProtocolService.BlockingInterface;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerStatusProto;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
|
||||
|
||||
public class LocalizationProtocolPBServiceImpl implements BlockingInterface {
|
||||
public class LocalizationProtocolPBServiceImpl implements LocalizationProtocolPB {
|
||||
|
||||
private LocalizationProtocol real;
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.hadoop.security.SecurityInfo;
|
|||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
import org.apache.hadoop.security.token.TokenSelector;
|
||||
import org.apache.hadoop.yarn.proto.LocalizationProtocol;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
|
||||
|
||||
public class LocalizerSecurityInfo extends SecurityInfo {
|
||||
|
||||
|
@ -38,7 +38,7 @@ public class LocalizerSecurityInfo extends SecurityInfo {
|
|||
@Override
|
||||
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol
|
||||
.equals(LocalizationProtocol.LocalizationProtocolService.BlockingInterface.class)) {
|
||||
.equals(LocalizationProtocolPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new TokenInfo() {
|
||||
|
|
|
@ -21,9 +21,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.security.authorize.PolicyProvider;
|
||||
import org.apache.hadoop.security.authorize.Service;
|
||||
import org.apache.hadoop.yarn.proto.ContainerManager;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagerPB;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.proto.LocalizationProtocol;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
|
||||
|
||||
/**
|
||||
* {@link PolicyProvider} for YARN NodeManager protocols.
|
||||
|
@ -36,9 +36,9 @@ public class NMPolicyProvider extends PolicyProvider {
|
|||
new Service[] {
|
||||
new Service(
|
||||
YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER,
|
||||
ContainerManager.ContainerManagerService.BlockingInterface.class),
|
||||
ContainerManagerPB.class),
|
||||
new Service(YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER,
|
||||
LocalizationProtocol.LocalizationProtocolService.BlockingInterface.class)
|
||||
LocalizationProtocolPB.class)
|
||||
};
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.api;
|
||||
|
||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||
import org.apache.hadoop.yarn.proto.RMAdminProtocol.RMAdminProtocolService;
|
||||
|
||||
@ProtocolInfo(
|
||||
protocolName = "org.apache.hadoop.yarn.server.nodemanager.api.RMAdminProtocolPB",
|
||||
protocolVersion = 1)
|
||||
public interface RMAdminProtocolPB extends RMAdminProtocolService.BlockingInterface {
|
||||
|
||||
}
|
|
@ -23,10 +23,10 @@ import java.lang.reflect.UndeclaredThrowableException;
|
|||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine;
|
||||
import org.apache.hadoop.yarn.proto.RMAdminProtocol.RMAdminProtocolService;
|
||||
import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto;
|
||||
|
@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Refre
|
|||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshSuperUserGroupsConfigurationRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshUserToGroupsMappingsRequestProto;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocol;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocolPB;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsResponse;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesRequest;
|
||||
|
@ -64,14 +65,14 @@ import com.google.protobuf.ServiceException;
|
|||
|
||||
public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
|
||||
|
||||
private RMAdminProtocolService.BlockingInterface proxy;
|
||||
private RMAdminProtocolPB proxy;
|
||||
|
||||
public RMAdminProtocolPBClientImpl(long clientVersion, InetSocketAddress addr,
|
||||
Configuration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, RMAdminProtocolService.BlockingInterface.class,
|
||||
ProtoOverHadoopRpcEngine.class);
|
||||
proxy = (RMAdminProtocolService.BlockingInterface)RPC.getProxy(
|
||||
RMAdminProtocolService.BlockingInterface.class, clientVersion, addr, conf);
|
||||
RPC.setProtocolEngine(conf, RMAdminProtocolPB.class,
|
||||
ProtobufRpcEngine.class);
|
||||
proxy = (RMAdminProtocolPB)RPC.getProxy(
|
||||
RMAdminProtocolPB.class, clientVersion, addr, conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -83,13 +84,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
|
|||
return new RefreshQueuesResponsePBImpl(
|
||||
proxy.refreshQueues(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -102,13 +97,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
|
|||
return new RefreshNodesResponsePBImpl(
|
||||
proxy.refreshNodes(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -122,13 +111,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
|
|||
return new RefreshSuperUserGroupsConfigurationResponsePBImpl(
|
||||
proxy.refreshSuperUserGroupsConfiguration(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -141,13 +124,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
|
|||
return new RefreshUserToGroupsMappingsResponsePBImpl(
|
||||
proxy.refreshUserToGroupsMappings(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -160,13 +137,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
|
|||
return new RefreshAdminAclsResponsePBImpl(
|
||||
proxy.refreshAdminAcls(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -175,18 +146,12 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
|
|||
RefreshServiceAclsRequest request) throws YarnRemoteException {
|
||||
RefreshServiceAclsRequestProto requestProto =
|
||||
((RefreshServiceAclsRequestPBImpl)request).getProto();
|
||||
try {
|
||||
return new RefreshServiceAclsResponsePBImpl(
|
||||
proxy.refreshServiceAcls(null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
if (e.getCause() instanceof YarnRemoteException) {
|
||||
throw (YarnRemoteException)e.getCause();
|
||||
} else if (e.getCause() instanceof UndeclaredThrowableException) {
|
||||
throw (UndeclaredThrowableException)e.getCause();
|
||||
} else {
|
||||
throw new UndeclaredThrowableException(e);
|
||||
}
|
||||
}
|
||||
try {
|
||||
return new RefreshServiceAclsResponsePBImpl(proxy.refreshServiceAcls(
|
||||
null, requestProto));
|
||||
} catch (ServiceException e) {
|
||||
throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -19,11 +19,11 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.api.impl.pb.service;
|
||||
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.proto.RMAdminProtocol.RMAdminProtocolService.BlockingInterface;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsResponseProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.*;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocol;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocolPB;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsResponse;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesResponse;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshQueuesResponse;
|
||||
|
@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.impl.pb
|
|||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class RMAdminProtocolPBServiceImpl implements BlockingInterface {
|
||||
public class RMAdminProtocolPBServiceImpl implements RMAdminProtocolPB {
|
||||
|
||||
private RMAdminProtocol real;
|
||||
|
||||
|
|
|
@ -25,13 +25,13 @@ import org.apache.hadoop.security.KerberosInfo;
|
|||
import org.apache.hadoop.security.SecurityInfo;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.proto.RMAdminProtocol;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocolPB;
|
||||
|
||||
public class AdminSecurityInfo extends SecurityInfo {
|
||||
|
||||
@Override
|
||||
public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
|
||||
if (!protocol.equals(RMAdminProtocol.RMAdminProtocolService.BlockingInterface.class)) {
|
||||
if (!protocol.equals(RMAdminProtocolPB.class)) {
|
||||
return null;
|
||||
}
|
||||
return new KerberosInfo() {
|
||||
|
|
|
@ -21,12 +21,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.security.authorize.PolicyProvider;
|
||||
import org.apache.hadoop.security.authorize.Service;
|
||||
import org.apache.hadoop.yarn.api.AMRMProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.ClientRMProtocolPB;
|
||||
import org.apache.hadoop.yarn.api.ContainerManagerPB;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.proto.ContainerManager;
|
||||
import org.apache.hadoop.yarn.proto.ResourceTracker;
|
||||
import org.apache.hadoop.yarn.proto.RMAdminProtocol;
|
||||
import org.apache.hadoop.yarn.proto.ClientRMProtocol;
|
||||
import org.apache.hadoop.yarn.proto.AMRMProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.ResourceTrackerPB;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocolPB;
|
||||
|
||||
/**
|
||||
* {@link PolicyProvider} for YARN ResourceManager protocols.
|
||||
|
@ -39,19 +39,19 @@ public class RMPolicyProvider extends PolicyProvider {
|
|||
new Service[] {
|
||||
new Service(
|
||||
YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER,
|
||||
ResourceTracker.ResourceTrackerService.BlockingInterface.class),
|
||||
ResourceTrackerPB.class),
|
||||
new Service(
|
||||
YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CLIENT_RESOURCEMANAGER,
|
||||
ClientRMProtocol.ClientRMProtocolService.BlockingInterface.class),
|
||||
ClientRMProtocolPB.class),
|
||||
new Service(
|
||||
YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONMASTER_RESOURCEMANAGER,
|
||||
AMRMProtocol.AMRMProtocolService.BlockingInterface.class),
|
||||
AMRMProtocolPB.class),
|
||||
new Service(
|
||||
YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_ADMIN,
|
||||
RMAdminProtocol.RMAdminProtocolService.BlockingInterface.class),
|
||||
RMAdminProtocolPB.class),
|
||||
new Service(
|
||||
YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGER,
|
||||
ContainerManager.ContainerManagerService.BlockingInterface.class),
|
||||
ContainerManagerPB.class),
|
||||
};
|
||||
|
||||
@Override
|
||||
|
|
|
@ -235,10 +235,11 @@ public class TestAMAuthorization {
|
|||
client.registerApplicationMaster(request);
|
||||
Assert.fail("Should fail with authorization error");
|
||||
} catch (YarnRemoteException e) {
|
||||
Assert.assertEquals("Unauthorized request from ApplicationMaster. "
|
||||
+ "Expected ApplicationAttemptID: "
|
||||
+ applicationAttemptId.toString() + " Found: "
|
||||
+ otherAppAttemptId.toString(), e.getMessage());
|
||||
Assert.assertTrue(e.getMessage().contains(
|
||||
"Unauthorized request from ApplicationMaster. "
|
||||
+ "Expected ApplicationAttemptID: "
|
||||
+ applicationAttemptId.toString() + " Found: "
|
||||
+ otherAppAttemptId.toString()));
|
||||
} finally {
|
||||
rm.stop();
|
||||
}
|
||||
|
|
|
@ -323,8 +323,10 @@ public class TestApplicationACLs {
|
|||
Assert.fail("App killing by the enemy should fail!!");
|
||||
} catch (YarnRemoteException e) {
|
||||
LOG.info("Got exception while killing app as the enemy", e);
|
||||
Assert.assertEquals("User enemy cannot perform operation MODIFY_APP on "
|
||||
+ applicationId, e.getMessage());
|
||||
Assert
|
||||
.assertTrue(e.getMessage().contains(
|
||||
"User enemy cannot perform operation MODIFY_APP on "
|
||||
+ applicationId));
|
||||
}
|
||||
|
||||
rmClient.forceKillApplication(finishAppRequest);
|
||||
|
|
|
@ -247,10 +247,12 @@ public class TestContainerManagerSecurity {
|
|||
Assert.assertEquals(
|
||||
java.lang.reflect.UndeclaredThrowableException.class
|
||||
.getCanonicalName(), e.getClass().getCanonicalName());
|
||||
Assert.assertEquals(
|
||||
"DIGEST-MD5: digest response format violation. "
|
||||
+ "Mismatched response.", e.getCause().getCause()
|
||||
.getMessage());
|
||||
Assert.assertTrue(e
|
||||
.getCause()
|
||||
.getMessage()
|
||||
.contains(
|
||||
"DIGEST-MD5: digest response format violation. "
|
||||
+ "Mismatched response."));
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -468,9 +470,10 @@ public class TestContainerManagerSecurity {
|
|||
+ "access is expected to fail.");
|
||||
} catch (YarnRemoteException e) {
|
||||
LOG.info("Got exception : ", e);
|
||||
Assert.assertEquals("Unauthorized request to start container. "
|
||||
+ "\nExpected containerId: " + tokenId.getContainerID()
|
||||
+ " Found: " + newContainerId.toString(), e.getMessage());
|
||||
Assert.assertTrue(e.getMessage().contains(
|
||||
"Unauthorized request to start container. "
|
||||
+ "\nExpected containerId: " + tokenId.getContainerID()
|
||||
+ " Found: " + newContainerId.toString()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue