Merge r1214114 and r1214499 from trunk for HADOOP-7920.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23-PB@1230896 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
c312a25e8a
commit
239789c65c
|
@ -3,6 +3,7 @@ Hadoop Change Log
|
|||
Release 0.23-PB - Unreleased
|
||||
|
||||
NEW FEATURES
|
||||
HADOOP-7920. Remove Avro Rpc. (suresh)
|
||||
|
||||
HADOOP-7773. Add support for protocol buffer based RPC engine.
|
||||
(suresh)
|
||||
|
|
|
@ -239,11 +239,6 @@
|
|||
<artifactId>avro</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro-ipc</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>net.sf.kosmosfs</groupId>
|
||||
<artifactId>kfs</artifactId>
|
||||
|
@ -282,7 +277,6 @@
|
|||
<phase>generate-test-sources</phase>
|
||||
<goals>
|
||||
<goal>schema</goal>
|
||||
<goal>protocol</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
|
|
|
@ -1,269 +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.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.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
|
||||
import javax.net.SocketFactory;
|
||||
|
||||
import org.apache.avro.ipc.Responder;
|
||||
import org.apache.avro.ipc.Transceiver;
|
||||
import org.apache.avro.ipc.reflect.ReflectRequestor;
|
||||
import org.apache.avro.ipc.reflect.ReflectResponder;
|
||||
import org.apache.avro.ipc.specific.SpecificRequestor;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
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;
|
||||
|
||||
/** Tunnel Avro-format RPC requests over a Hadoop {@link RPC} connection. This
|
||||
* does not give cross-language wire compatibility, since the Hadoop RPC wire
|
||||
* format is non-standard, but it does permit use of Avro's protocol versioning
|
||||
* features for inter-Java RPCs. */
|
||||
@InterfaceStability.Evolving
|
||||
public class AvroRpcEngine implements RpcEngine {
|
||||
private static final Log LOG = LogFactory.getLog(RPC.class);
|
||||
|
||||
private static int VERSION = 1;
|
||||
|
||||
// the implementation we tunnel through
|
||||
private static final RpcEngine ENGINE = new WritableRpcEngine();
|
||||
|
||||
/** Tunnel an Avro RPC request and response through Hadoop's RPC. */
|
||||
private static interface TunnelProtocol extends VersionedProtocol {
|
||||
//WritableRpcEngine expects a versionID in every protocol.
|
||||
public static final long versionID = VERSION;
|
||||
/** All Avro methods and responses go through this. */
|
||||
BufferListWritable call(String protocol, BufferListWritable request)
|
||||
throws IOException;
|
||||
}
|
||||
|
||||
/** A Writable that holds a List<ByteBuffer>, The Avro RPC Transceiver's
|
||||
* basic unit of data transfer.*/
|
||||
private static class BufferListWritable implements Writable {
|
||||
private List<ByteBuffer> buffers;
|
||||
|
||||
public BufferListWritable() {} // required for RPC Writables
|
||||
|
||||
public BufferListWritable(List<ByteBuffer> buffers) {
|
||||
this.buffers = buffers;
|
||||
}
|
||||
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
int size = in.readInt();
|
||||
buffers = new ArrayList<ByteBuffer>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
int length = in.readInt();
|
||||
ByteBuffer buffer = ByteBuffer.allocate(length);
|
||||
in.readFully(buffer.array(), 0, length);
|
||||
buffers.add(buffer);
|
||||
}
|
||||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeInt(buffers.size());
|
||||
for (ByteBuffer buffer : buffers) {
|
||||
out.writeInt(buffer.remaining());
|
||||
out.write(buffer.array(), buffer.position(), buffer.remaining());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** An Avro RPC Transceiver that tunnels client requests through Hadoop
|
||||
* RPC. */
|
||||
private static class ClientTransceiver extends Transceiver {
|
||||
private TunnelProtocol tunnel;
|
||||
private InetSocketAddress remote;
|
||||
private String protocol;
|
||||
|
||||
public ClientTransceiver(InetSocketAddress addr,
|
||||
UserGroupInformation ticket,
|
||||
Configuration conf, SocketFactory factory,
|
||||
int rpcTimeout, String protocol)
|
||||
throws IOException {
|
||||
this.tunnel = ENGINE.getProxy(TunnelProtocol.class, VERSION,
|
||||
addr, ticket, conf, factory,
|
||||
rpcTimeout).getProxy();
|
||||
this.remote = addr;
|
||||
this.protocol = protocol;
|
||||
}
|
||||
|
||||
public String getRemoteName() { return remote.toString(); }
|
||||
|
||||
public List<ByteBuffer> transceive(List<ByteBuffer> request)
|
||||
throws IOException {
|
||||
return tunnel.call(protocol, new BufferListWritable(request)).buffers;
|
||||
}
|
||||
|
||||
public List<ByteBuffer> readBuffers() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
public void writeBuffers(List<ByteBuffer> buffers) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
RPC.stopProxy(tunnel);
|
||||
}
|
||||
}
|
||||
|
||||
/** Construct a client-side proxy object that implements the named protocol,
|
||||
* talking to a server at the named address.
|
||||
* @param <T>*/
|
||||
@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);
|
||||
}
|
||||
|
||||
private class Invoker implements InvocationHandler, Closeable {
|
||||
private final ClientTransceiver tx;
|
||||
private final SpecificRequestor requestor;
|
||||
public Invoker(Class<?> protocol, InetSocketAddress addr,
|
||||
UserGroupInformation ticket, Configuration conf,
|
||||
SocketFactory factory,
|
||||
int rpcTimeout) throws IOException {
|
||||
this.tx = new ClientTransceiver(addr, ticket, conf, factory, rpcTimeout,
|
||||
protocol.getName());
|
||||
this.requestor = createRequestor(protocol, tx);
|
||||
}
|
||||
@Override public Object invoke(Object proxy, Method method, Object[] args)
|
||||
throws Throwable {
|
||||
return requestor.invoke(proxy, method, args);
|
||||
}
|
||||
public void close() throws IOException {
|
||||
tx.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected SpecificRequestor createRequestor(Class<?> protocol,
|
||||
Transceiver transeiver) throws IOException {
|
||||
return new ReflectRequestor(protocol, transeiver);
|
||||
}
|
||||
|
||||
protected Responder createResponder(Class<?> iface, Object impl) {
|
||||
return new ReflectResponder(iface, impl);
|
||||
}
|
||||
|
||||
/** An Avro RPC Responder that can process requests passed via Hadoop RPC. */
|
||||
private class TunnelResponder implements TunnelProtocol {
|
||||
private Map<String, Responder> responders =
|
||||
new HashMap<String, Responder>();
|
||||
|
||||
public void addProtocol(Class<?> iface, Object impl) {
|
||||
responders.put(iface.getName(), createResponder(iface, impl));
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getProtocolVersion(String protocol, long version)
|
||||
throws IOException {
|
||||
return VERSION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProtocolSignature getProtocolSignature(
|
||||
String protocol, long version, int clientMethodsHashCode)
|
||||
throws IOException {
|
||||
return ProtocolSignature.getProtocolSignature
|
||||
(clientMethodsHashCode, VERSION, TunnelProtocol.class);
|
||||
}
|
||||
|
||||
public BufferListWritable call(String protocol, BufferListWritable request)
|
||||
throws IOException {
|
||||
Responder responder = responders.get(protocol);
|
||||
if (responder == null)
|
||||
throw new IOException("No responder for: "+protocol);
|
||||
return new BufferListWritable(responder.respond(request.buffers));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public Object[] call(Method method, Object[][] params,
|
||||
InetSocketAddress[] addrs, UserGroupInformation ticket,
|
||||
Configuration conf) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
private class Server extends WritableRpcEngine.Server {
|
||||
private TunnelResponder responder = new TunnelResponder();
|
||||
|
||||
public Server(Class<?> iface, Object impl, String bindAddress,
|
||||
int port, int numHandlers, int numReaders,
|
||||
int queueSizePerHandler, boolean verbose,
|
||||
Configuration conf,
|
||||
SecretManager<? extends TokenIdentifier> secretManager
|
||||
) throws IOException {
|
||||
super((Class)null, new Object(), conf,
|
||||
bindAddress, port, numHandlers, numReaders,
|
||||
queueSizePerHandler, verbose, secretManager);
|
||||
// RpcKind is WRITABLE since Avro is tunneled through WRITABLE
|
||||
super.addProtocol(RpcKind.RPC_WRITABLE, TunnelProtocol.class, responder);
|
||||
responder.addProtocol(iface, impl);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Server
|
||||
addProtocol(RpcKind rpcKind, Class<?> protocolClass, Object protocolImpl)
|
||||
throws IOException {
|
||||
responder.addProtocol(protocolClass, protocolImpl);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
/** Construct a server for a protocol implementation instance listening on a
|
||||
* port and address. */
|
||||
public RPC.Server getServer(Class<?> iface, Object impl, String bindAddress,
|
||||
int port, int numHandlers, int numReaders,
|
||||
int queueSizePerHandler, boolean verbose,
|
||||
Configuration conf,
|
||||
SecretManager<? extends TokenIdentifier> secretManager
|
||||
) throws IOException {
|
||||
return new Server
|
||||
(iface, impl, bindAddress, port, numHandlers, numReaders,
|
||||
queueSizePerHandler, verbose, conf, secretManager);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,45 +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.ipc;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.avro.ipc.Responder;
|
||||
import org.apache.avro.ipc.Transceiver;
|
||||
import org.apache.avro.ipc.specific.SpecificRequestor;
|
||||
import org.apache.avro.ipc.specific.SpecificResponder;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* AvroRpcEngine which uses Avro's "specific" APIs. The protocols generated
|
||||
* via Avro IDL needs to use this Engine.
|
||||
*/
|
||||
@InterfaceStability.Evolving
|
||||
public class AvroSpecificRpcEngine extends AvroRpcEngine {
|
||||
|
||||
protected SpecificRequestor createRequestor(Class<?> protocol,
|
||||
Transceiver transeiver) throws IOException {
|
||||
return new SpecificRequestor(protocol, transeiver);
|
||||
}
|
||||
|
||||
protected Responder createResponder(Class<?> iface, Object impl) {
|
||||
return new SpecificResponder(iface, impl);
|
||||
}
|
||||
|
||||
}
|
|
@ -56,9 +56,8 @@ public class RpcPayloadHeader implements Writable {
|
|||
public enum RpcKind {
|
||||
RPC_BUILTIN ((short) 1), // Used for built in calls by tests
|
||||
RPC_WRITABLE ((short) 2), // Use WritableRpcEngine
|
||||
RPC_PROTOCOL_BUFFER ((short) 3), // Use ProtobufRpcEngine
|
||||
RPC_AVRO ((short) 4); // Use AvroRpcEngine
|
||||
static final short MAX_INDEX = RPC_AVRO.value; // used for array size
|
||||
RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
|
||||
final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
|
||||
private static final short FIRST_INDEX = RPC_BUILTIN.value;
|
||||
private final short value;
|
||||
|
||||
|
|
|
@ -1,42 +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.
|
||||
|
||||
{
|
||||
"protocol" : "AvroSpecificTestProtocol",
|
||||
"namespace" : "org.apache.hadoop.ipc",
|
||||
|
||||
"messages" : {
|
||||
"echo" : {
|
||||
"request" : [ {
|
||||
"name" : "message",
|
||||
"type" : "string"
|
||||
} ],
|
||||
"response" : "string"
|
||||
},
|
||||
|
||||
"add" : {
|
||||
"request" : [ {
|
||||
"name" : "arg1",
|
||||
"type" : "int"
|
||||
}, {
|
||||
"name" : "arg2",
|
||||
"type" : "int",
|
||||
"default" : 0
|
||||
} ],
|
||||
"response" : "int"
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,32 +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.ipc;
|
||||
|
||||
import org.apache.avro.AvroRemoteException;
|
||||
|
||||
@SuppressWarnings("serial")
|
||||
public interface AvroTestProtocol {
|
||||
public static class Problem extends AvroRemoteException {
|
||||
public Problem() {}
|
||||
}
|
||||
void ping();
|
||||
String echo(String value);
|
||||
int add(int v1, int v2);
|
||||
int error() throws Problem;
|
||||
}
|
|
@ -1,227 +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.ipc;
|
||||
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import javax.security.sasl.Sasl;
|
||||
|
||||
import junit.framework.Assert;
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.avro.AvroRemoteException;
|
||||
import org.apache.avro.util.Utf8;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
|
||||
import org.apache.hadoop.ipc.TestSaslRPC.CustomSecurityInfo;
|
||||
import org.apache.hadoop.ipc.TestSaslRPC.TestTokenIdentifier;
|
||||
import org.apache.hadoop.ipc.TestSaslRPC.TestTokenSecretManager;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.SaslRpcServer;
|
||||
import org.apache.hadoop.security.SecurityInfo;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
|
||||
/** Unit tests for AvroRpc. */
|
||||
public class TestAvroRpc extends TestCase {
|
||||
private static final String ADDRESS = "0.0.0.0";
|
||||
|
||||
public static final Log LOG =
|
||||
LogFactory.getLog(TestAvroRpc.class);
|
||||
|
||||
int datasize = 1024*100;
|
||||
int numThreads = 50;
|
||||
|
||||
public TestAvroRpc(String name) { super(name); }
|
||||
|
||||
public static interface EmptyProtocol {}
|
||||
public static class EmptyImpl implements EmptyProtocol {}
|
||||
|
||||
public static class TestImpl implements AvroTestProtocol {
|
||||
|
||||
public void ping() {}
|
||||
|
||||
public String echo(String value) { return value; }
|
||||
|
||||
public int add(int v1, int v2) { return v1 + v2; }
|
||||
|
||||
public int error() throws Problem {
|
||||
throw new Problem();
|
||||
}
|
||||
}
|
||||
|
||||
public void testReflect() throws Exception {
|
||||
testReflect(false);
|
||||
}
|
||||
|
||||
public void testSecureReflect() throws Exception {
|
||||
testReflect(true);
|
||||
}
|
||||
|
||||
public void testSpecific() throws Exception {
|
||||
testSpecific(false);
|
||||
}
|
||||
|
||||
public void testSecureSpecific() throws Exception {
|
||||
testSpecific(true);
|
||||
}
|
||||
|
||||
private void testReflect(boolean secure) throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
TestTokenSecretManager sm = null;
|
||||
if (secure) {
|
||||
makeSecure(conf);
|
||||
sm = new TestTokenSecretManager();
|
||||
}
|
||||
UserGroupInformation.setConfiguration(conf);
|
||||
RPC.setProtocolEngine(conf, EmptyProtocol.class, AvroRpcEngine.class);
|
||||
RPC.setProtocolEngine(conf, AvroTestProtocol.class, AvroRpcEngine.class);
|
||||
RPC.Server server = RPC.getServer(EmptyProtocol.class, new EmptyImpl(),
|
||||
ADDRESS, 0, 5, true, conf, sm);
|
||||
server.addProtocol(RpcKind.RPC_WRITABLE,
|
||||
AvroTestProtocol.class, new TestImpl());
|
||||
|
||||
try {
|
||||
server.start();
|
||||
InetSocketAddress addr = NetUtils.getConnectAddress(server);
|
||||
|
||||
if (secure) {
|
||||
addToken(sm, addr);
|
||||
//QOP must be auth
|
||||
Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
|
||||
}
|
||||
|
||||
AvroTestProtocol proxy =
|
||||
(AvroTestProtocol)RPC.getProxy(AvroTestProtocol.class, 0, addr, conf);
|
||||
|
||||
proxy.ping();
|
||||
|
||||
String echo = proxy.echo("hello world");
|
||||
assertEquals("hello world", echo);
|
||||
|
||||
int intResult = proxy.add(1, 2);
|
||||
assertEquals(3, intResult);
|
||||
|
||||
boolean caught = false;
|
||||
try {
|
||||
proxy.error();
|
||||
} catch (AvroRemoteException e) {
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("Caught " + e);
|
||||
}
|
||||
caught = true;
|
||||
}
|
||||
assertTrue(caught);
|
||||
|
||||
} finally {
|
||||
resetSecurity();
|
||||
server.stop();
|
||||
}
|
||||
}
|
||||
|
||||
private void makeSecure(Configuration conf) {
|
||||
conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
|
||||
conf.set("hadoop.rpc.socket.factory.class.default", "");
|
||||
//Avro doesn't work with security annotations on protocol.
|
||||
//Avro works ONLY with custom security context
|
||||
SecurityUtil.setSecurityInfoProviders(new CustomSecurityInfo());
|
||||
}
|
||||
|
||||
private void resetSecurity() {
|
||||
SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]);
|
||||
}
|
||||
|
||||
private void addToken(TestTokenSecretManager sm,
|
||||
InetSocketAddress addr) throws IOException {
|
||||
final UserGroupInformation current = UserGroupInformation.getCurrentUser();
|
||||
|
||||
TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
|
||||
.getUserName()));
|
||||
Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
|
||||
sm);
|
||||
Text host = new Text(addr.getAddress().getHostAddress() + ":"
|
||||
+ addr.getPort());
|
||||
token.setService(host);
|
||||
LOG.info("Service IP address for token is " + host);
|
||||
current.addToken(token);
|
||||
}
|
||||
|
||||
private void testSpecific(boolean secure) throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
TestTokenSecretManager sm = null;
|
||||
if (secure) {
|
||||
makeSecure(conf);
|
||||
sm = new TestTokenSecretManager();
|
||||
}
|
||||
UserGroupInformation.setConfiguration(conf);
|
||||
RPC.setProtocolEngine(conf, AvroSpecificTestProtocol.class,
|
||||
AvroSpecificRpcEngine.class);
|
||||
Server server = RPC.getServer(AvroSpecificTestProtocol.class,
|
||||
new AvroSpecificTestProtocolImpl(), ADDRESS, 0, 5, true,
|
||||
conf, sm);
|
||||
try {
|
||||
server.start();
|
||||
InetSocketAddress addr = NetUtils.getConnectAddress(server);
|
||||
|
||||
if (secure) {
|
||||
addToken(sm, addr);
|
||||
//QOP must be auth
|
||||
Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
|
||||
}
|
||||
|
||||
AvroSpecificTestProtocol proxy =
|
||||
(AvroSpecificTestProtocol)RPC.getProxy(AvroSpecificTestProtocol.class,
|
||||
0, addr, conf);
|
||||
|
||||
CharSequence echo = proxy.echo("hello world");
|
||||
assertEquals("hello world", echo.toString());
|
||||
|
||||
int intResult = proxy.add(1, 2);
|
||||
assertEquals(3, intResult);
|
||||
|
||||
} finally {
|
||||
resetSecurity();
|
||||
server.stop();
|
||||
}
|
||||
}
|
||||
|
||||
public static class AvroSpecificTestProtocolImpl implements
|
||||
AvroSpecificTestProtocol {
|
||||
|
||||
@Override
|
||||
public int add(int arg1, int arg2) throws AvroRemoteException {
|
||||
return arg1 + arg2;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence echo(CharSequence msg) throws AvroRemoteException {
|
||||
return msg;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -402,11 +402,6 @@
|
|||
<artifactId>avro</artifactId>
|
||||
<version>1.5.3</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro-ipc</artifactId>
|
||||
<version>1.5.3</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>net.sf.kosmosfs</groupId>
|
||||
<artifactId>kfs</artifactId>
|
||||
|
|
Loading…
Reference in New Issue