diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index bbb5d90d31a..10f5f7e21f5 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -5,6 +5,11 @@ Trunk (unreleased changes)
INCOMPATIBLE CHANGES
NEW FEATURES
+ HADOOP-7773. Add support for protocol buffer based RPC engine.
+ (suresh)
+
+ HADOOP-7875. Add helper class to unwrap protobuf ServiceException.
+ (suresh)
IMPROVEMENTS
@@ -61,6 +66,12 @@ Trunk (unreleased changes)
HADOOP-7590. Mavenize streaming and MR examples. (tucu)
+ HADOOP-7862. Move the support for multiple protocols to lower layer so
+ that Writable, PB and Avro can all use it (Sanjay)
+
+ HADOOP-7876. Provided access to encoded key in DelegationKey for
+ use in protobuf based RPCs. (suresh)
+
BUGS
HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
@@ -102,13 +113,12 @@ Trunk (unreleased changes)
HDFS-2614. hadoop dist tarball is missing hdfs headers. (tucu)
+ HADOOP-7874. native libs should be under lib/native/ dir. (tucu)
+
OPTIMIZATIONS
HADOOP-7761. Improve the performance of raw comparisons. (todd)
- HADOOP-7773. Add support for protocol buffer based RPC engine.
- (suresh)
-
Release 0.23.1 - Unreleased
INCOMPATIBLE CHANGES
@@ -134,6 +144,9 @@ Release 0.23.1 - Unreleased
HADOOP-7804. Enable hadoop config generator to set configurations to enable
short circuit read. (Arpit Gupta via jitendra)
+ HADOOP-7877. Update balancer CLI usage documentation to include the new
+ -policy option. (szetszwo)
+
OPTIMIZATIONS
BUG FIXES
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh
index 8bf9aecef4d..e53ec737f5a 100644
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh
@@ -186,29 +186,16 @@ fi
# setup 'java.library.path' for native-hadoop code if necessary
if [ -d "${HADOOP_PREFIX}/build/native" -o -d "${HADOOP_PREFIX}/lib/native" ]; then
- JAVA_PLATFORM=`CLASSPATH=${CLASSPATH} ${JAVA} -Xmx32m ${HADOOP_JAVA_PLATFORM_OPTS} org.apache.hadoop.util.PlatformName | sed -e "s/ /_/g"`
-
- if [ -d "$HADOOP_PREFIX/build/native" ]; then
- if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
- JAVA_LIBRARY_PATH=${JAVA_LIBRARY_PATH}:${HADOOP_PREFIX}/build/native/${JAVA_PLATFORM}/lib
- else
- JAVA_LIBRARY_PATH=${HADOOP_PREFIX}/build/native/${JAVA_PLATFORM}/lib
- fi
- fi
-
+
if [ -d "${HADOOP_PREFIX}/lib/native" ]; then
if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
- JAVA_LIBRARY_PATH=${JAVA_LIBRARY_PATH}:${HADOOP_PREFIX}/lib/native/${JAVA_PLATFORM}
+ JAVA_LIBRARY_PATH=${JAVA_LIBRARY_PATH}:${HADOOP_PREFIX}/lib/native
else
- JAVA_LIBRARY_PATH=${HADOOP_PREFIX}/lib/native/${JAVA_PLATFORM}
+ JAVA_LIBRARY_PATH=${HADOOP_PREFIX}/lib/native
fi
fi
fi
-if [ -e "${HADOOP_PREFIX}/lib/libhadoop.a" ]; then
- JAVA_LIBRARY_PATH=${HADOOP_PREFIX}/lib
-fi
-
# cygwin path translation
if $cygwin; then
JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`
diff --git a/hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/commands_manual.xml b/hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/commands_manual.xml
index b46b1285a17..b3f25af40e4 100644
--- a/hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/commands_manual.xml
+++ b/hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/commands_manual.xml
@@ -445,14 +445,22 @@
Rebalancer .
- Usage: hadoop balancer [-threshold <threshold>]
+ Usage: hadoop balancer [-policy <blockpool|datanode>] [-threshold <threshold>]
COMMAND_OPTION Description
-
+
+ -policy <blockpool|datanode>
+ The balancing policy.
+ datanode
: Cluster is balance if the disk usage of each datanode is balance.
+ blockpool
: Cluster is balance if the disk usage of each block pool in each datanode is balance.
+ Note that blockpool
is a condition stronger than datanode
.
+ The default policy is datanode
.
+
+
-threshold <threshold>
- Percentage of disk capacity. This overwrites the default threshold.
+ Percentage of disk capacity. This default threshold is 10%.
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java
index 12aa04ff855..8fec3d22b84 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java
@@ -44,6 +44,7 @@ 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;
@@ -237,14 +238,15 @@ public class AvroRpcEngine implements RpcEngine {
super((Class)null, new Object(), conf,
bindAddress, port, numHandlers, numReaders,
queueSizePerHandler, verbose, secretManager);
- super.addProtocol(TunnelProtocol.class, responder);
+ // 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(Class protocolClass, IMPL protocolImpl)
+ public Server
+ addProtocol(RpcKind rpcKind, Class> protocolClass, Object protocolImpl)
throws IOException {
responder.addProtocol(protocolClass, protocolImpl);
return this;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 55e8a23d483..a6c2b472825 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -1002,17 +1002,19 @@ public class Client {
}
/**
- * Same as {@link #call(RpcKind, Writable, ConnectionId)} for Writable
+ * Same as {@link #call(RpcPayloadHeader.RpcKind, Writable, ConnectionId)}
+ * for RPC_BUILTIN
*/
public Writable call(Writable param, InetSocketAddress address)
throws InterruptedException, IOException {
- return call(RpcKind.RPC_WRITABLE, param, address);
+ return call(RpcKind.RPC_BUILTIN, param, address);
}
/** Make a call, passing param
, to the IPC server running at
* address
, returning the value. Throws exceptions if there are
* network problems or if the remote code threw an exception.
- * @deprecated Use {@link #call(RpcKind, Writable, ConnectionId)} instead
+ * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
+ * ConnectionId)} instead
*/
@Deprecated
public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress address)
@@ -1025,7 +1027,8 @@ public class Client {
* the value.
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
- * @deprecated Use {@link #call(RpcKind, Writable, ConnectionId)} instead
+ * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
+ * ConnectionId)} instead
*/
@Deprecated
public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr,
@@ -1042,7 +1045,8 @@ public class Client {
* timeout, returning the value.
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
- * @deprecated Use {@link #call(RpcKind, Writable, ConnectionId)} instead
+ * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
+ * ConnectionId)} instead
*/
@Deprecated
public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr,
@@ -1056,7 +1060,7 @@ public class Client {
/**
- * Same as {@link #call(RpcKind, Writable, InetSocketAddress,
+ * Same as {@link #call(RpcPayloadHeader.RpcKind, Writable, InetSocketAddress,
* Class, UserGroupInformation, int, Configuration)}
* except that rpcKind is writable.
*/
@@ -1066,7 +1070,7 @@ public class Client {
throws InterruptedException, IOException {
ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
ticket, rpcTimeout, conf);
- return call(RpcKind.RPC_WRITABLE, param, remoteId);
+ return call(RpcKind.RPC_BUILTIN, param, remoteId);
}
/**
@@ -1087,21 +1091,28 @@ public class Client {
}
/**
- * Same as {link {@link #call(RpcKind, Writable, ConnectionId)}
- * except the rpcKind is RPC_WRITABLE
+ * Same as {link {@link #call(RpcPayloadHeader.RpcKind, Writable, ConnectionId)}
+ * except the rpcKind is RPC_BUILTIN
*/
public Writable call(Writable param, ConnectionId remoteId)
throws InterruptedException, IOException {
- return call(RpcKind.RPC_WRITABLE, param, remoteId);
+ return call(RpcKind.RPC_BUILTIN, param, remoteId);
}
- /** Make a call, passing param
, to the IPC server defined by
- * remoteId
, returning the value.
+ /**
+ * Make a call, passing rpcRequest
, to the IPC server defined by
+ * remoteId
, returning the rpc respond.
+ *
+ * @param rpcKind
+ * @param rpcRequest - contains serialized method and method parameters
+ * @param remoteId - the target rpc server
+ * @returns the rpc response
* Throws exceptions if there are network problems or if the remote code
- * threw an exception. */
- public Writable call(RpcKind rpcKind, Writable param, ConnectionId remoteId)
- throws InterruptedException, IOException {
- Call call = new Call(rpcKind, param);
+ * threw an exception.
+ */
+ public Writable call(RpcKind rpcKind, Writable rpcRequest,
+ ConnectionId remoteId) throws InterruptedException, IOException {
+ Call call = new Call(rpcKind, rpcRequest);
Connection connection = getConnection(remoteId, call);
connection.sendParam(call); // send the parameter
boolean interrupted = false;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java
new file mode 100644
index 00000000000..7f029618fa4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java
@@ -0,0 +1,46 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+
+import com.google.protobuf.ServiceException;
+
+/**
+ * Helper methods for protobuf related RPC implementation
+ */
+@InterfaceAudience.Private
+public class ProtobufHelper {
+ private ProtobufHelper() {
+ // Hidden constructor for class with only static helper methods
+ }
+
+ /**
+ * Return the RemoteException wrapped in ServiceException as cause.
+ * @param se ServiceException that wraps RemoteException
+ * @return RemoteException wrapped in ServiceException or
+ * a new IOException that wraps unexpected ServiceException.
+ */
+ public static IOException getRemoteException(ServiceException se) {
+ Throwable e = se.getCause();
+ return ((e instanceof RemoteException) ? (IOException) e :
+ new IOException(se));
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index dad94227ca2..3db73859e58 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RPC.RpcInvoker;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcExceptionProto;
import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcRequestProto;
@@ -60,6 +61,12 @@ import com.google.protobuf.ServiceException;
@InterfaceStability.Evolving
public class ProtobufRpcEngine implements RpcEngine {
private static final Log LOG = LogFactory.getLog(ProtobufRpcEngine.class);
+
+ static { // Register the rpcRequest deserializer for WritableRpcEngine
+ org.apache.hadoop.ipc.Server.registerProtocolEngine(
+ RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWritable.class,
+ new Server.ProtoBufRpcInvoker());
+ }
private static final ClientCache CLIENTS = new ClientCache();
@@ -75,10 +82,13 @@ public class ProtobufRpcEngine implements RpcEngine {
}
private static class Invoker implements InvocationHandler, Closeable {
- private Map returnTypes = new ConcurrentHashMap();
+ private final Map returnTypes =
+ new ConcurrentHashMap();
private boolean isClosed = false;
- private Client.ConnectionId remoteId;
- private Client client;
+ private final Client.ConnectionId remoteId;
+ private final Client client;
+ private final long clientProtocolVersion;
+ private final String protocolName;
public Invoker(Class> protocol, InetSocketAddress addr,
UserGroupInformation ticket, Configuration conf, SocketFactory factory,
@@ -87,6 +97,8 @@ public class ProtobufRpcEngine implements RpcEngine {
ticket, rpcTimeout, conf);
this.client = CLIENTS.getClient(conf, factory,
RpcResponseWritable.class);
+ this.clientProtocolVersion = RPC.getProtocolVersion(protocol);
+ this.protocolName = RPC.getProtocolName(protocol);
}
private HadoopRpcRequestProto constructRpcRequest(Method method,
@@ -108,6 +120,19 @@ public class ProtobufRpcEngine implements RpcEngine {
Message param = (Message) params[1];
builder.setRequest(param.toByteString());
+ // For protobuf, {@code protocol} used when creating client side proxy is
+ // the interface extending BlockingInterface, which has the annotations
+ // such as ProtocolName etc.
+ //
+ // Using Method.getDeclaringClass(), as in WritableEngine to get at
+ // the protocol interface will return BlockingInterface, from where
+ // the annotation ProtocolName and Version cannot be
+ // obtained.
+ //
+ // Hence we simply use the protocol class used to create the proxy.
+ // For PB this may limit the use of mixins on client side.
+ builder.setDeclaringClassProtocolName(protocolName);
+ builder.setClientProtocolVersion(clientProtocolVersion);
rpcRequest = builder.build();
return rpcRequest;
}
@@ -272,15 +297,16 @@ public class ProtobufRpcEngine implements RpcEngine {
RpcResponseWritable.class);
}
+
@Override
- public RPC.Server getServer(Class> protocol, Object instance,
+ public RPC.Server getServer(Class> protocol, Object protocolImpl,
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);
+ return new Server(protocol, protocolImpl, conf, bindAddress, port,
+ numHandlers, numReaders, queueSizePerHandler, verbose, secretManager);
}
private static RemoteException getRemoteException(Exception e) {
@@ -289,87 +315,31 @@ public class ProtobufRpcEngine implements RpcEngine {
}
public static class Server extends RPC.Server {
- private BlockingService service;
- private boolean verbose;
-
- 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 protocolClass the class of protocol
+ * @param protocolImpl the protocolImpl 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)
+ public Server(Class> protocolClass, Object protocolImpl,
+ Configuration conf, String bindAddress, int port, int numHandlers,
+ int numReaders, int queueSizePerHandler, boolean verbose,
+ SecretManager extends TokenIdentifier> secretManager)
throws IOException {
super(bindAddress, port, RpcRequestWritable.class, numHandlers,
- numReaders, queueSizePerHandler, conf, classNameBase(instance
+ numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
.getClass().getName()), secretManager);
- this.service = (BlockingService) instance;
- this.verbose = verbose;
+ this.verbose = verbose;
+ registerProtocolAndImpl(RpcKind.RPC_PROTOCOL_BUFFER,
+ protocolClass, protocolImpl);
}
- /**
- * This is a server side method, which is invoked over RPC. On success
- * the return response has protobuf response payload. On failure, the
- * exception name and the stack trace are return in the resposne. See {@link HadoopRpcResponseProto}
- *
- * In this method there three types of exceptions possible and they are
- * returned in response as follows.
- *
- * Exceptions encountered in this method that are returned as {@link RpcServerException}
- * Exceptions thrown by the service is wrapped in ServiceException. In that
- * this method returns in response the exception thrown by the service.
- * Other exceptions thrown by the service. They are returned as
- * it is.
- *
- */
- @Override
- public Writable call(String protocol, Writable writableRequest,
- long receiveTime) throws IOException {
- RpcRequestWritable request = (RpcRequestWritable) writableRequest;
- HadoopRpcRequestProto rpcRequest = request.message;
- String methodName = rpcRequest.getMethodName();
- if (verbose)
- LOG.info("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 RpcServerException(msg));
- }
- Message prototype = service.getRequestPrototype(methodDescriptor);
- Message param = prototype.newBuilderForType()
- .mergeFrom(rpcRequest.getRequest()).build();
- Message result;
- try {
- result = service.callBlockingMethod(methodDescriptor, null, param);
- } catch (ServiceException e) {
- Throwable cause = e.getCause();
- return handleException(cause != null ? cause : e);
- } catch (Exception e) {
- return handleException(e);
- }
-
- HadoopRpcResponseProto response = constructProtoSpecificRpcSuccessResponse(result);
- return new RpcResponseWritable(response);
- }
-
- private RpcResponseWritable handleException(Throwable e) {
+ private static RpcResponseWritable handleException(Throwable e) {
HadoopRpcExceptionProto exception = HadoopRpcExceptionProto.newBuilder()
.setExceptionName(e.getClass().getName())
.setStackTrace(StringUtils.stringifyException(e)).build();
@@ -378,7 +348,7 @@ public class ProtobufRpcEngine implements RpcEngine {
return new RpcResponseWritable(response);
}
- private HadoopRpcResponseProto constructProtoSpecificRpcSuccessResponse(
+ private static HadoopRpcResponseProto constructProtoSpecificRpcSuccessResponse(
Message message) {
HadoopRpcResponseProto res = HadoopRpcResponseProto.newBuilder()
.setResponse(message.toByteString())
@@ -386,5 +356,81 @@ public class ProtobufRpcEngine implements RpcEngine {
.build();
return res;
}
+
+ /**
+ * Protobuf invoker for {@link RpcInvoker}
+ */
+ static class ProtoBufRpcInvoker implements RpcInvoker {
+
+ @Override
+ /**
+ * This is a server side method, which is invoked over RPC. On success
+ * the return response has protobuf response payload. On failure, the
+ * exception name and the stack trace are return in the resposne.
+ * See {@link HadoopRpcResponseProto}
+ *
+ * In this method there three types of exceptions possible and they are
+ * returned in response as follows.
+ *
+ * Exceptions encountered in this method that are returned
+ * as {@link RpcServerException}
+ * Exceptions thrown by the service is wrapped in ServiceException.
+ * In that this method returns in response the exception thrown by the
+ * service.
+ * Other exceptions thrown by the service. They are returned as
+ * it is.
+ *
+ */
+ public Writable call(RPC.Server server, String protocol,
+ Writable writableRequest, long receiveTime) throws IOException {
+ RpcRequestWritable request = (RpcRequestWritable) writableRequest;
+ HadoopRpcRequestProto rpcRequest = request.message;
+ String methodName = rpcRequest.getMethodName();
+ String protoName = rpcRequest.getDeclaringClassProtocolName();
+ long clientVersion = rpcRequest.getClientProtocolVersion();
+ if (server.verbose)
+ LOG.info("Call: protocol=" + protocol + ", method=" + methodName);
+
+ ProtoNameVer pv = new ProtoNameVer(protoName, clientVersion);
+ ProtoClassProtoImpl protocolImpl =
+ server.getProtocolImplMap(RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
+ if (protocolImpl == null) { // no match for Protocol AND Version
+ VerProtocolImpl highest =
+ server.getHighestSupportedProtocol(RpcKind.RPC_PROTOCOL_BUFFER,
+ protoName);
+ if (highest == null) {
+ throw new IOException("Unknown protocol: " + protoName);
+ }
+ // protocol supported but not the version that client wants
+ throw new RPC.VersionMismatch(protoName, clientVersion,
+ highest.version);
+ }
+
+ BlockingService service = (BlockingService) protocolImpl.protocolImpl;
+ MethodDescriptor methodDescriptor = service.getDescriptorForType()
+ .findMethodByName(methodName);
+ if (methodDescriptor == null) {
+ String msg = "Unknown method " + methodName + " called on " + protocol
+ + " protocol.";
+ LOG.warn(msg);
+ return handleException(new RpcServerException(msg));
+ }
+ Message prototype = service.getRequestPrototype(methodDescriptor);
+ Message param = prototype.newBuilderForType()
+ .mergeFrom(rpcRequest.getRequest()).build();
+ Message result;
+ try {
+ result = service.callBlockingMethod(methodDescriptor, null, param);
+ } catch (ServiceException e) {
+ Throwable cause = e.getCause();
+ return handleException(cause != null ? cause : e);
+ } catch (Exception e) {
+ return handleException(e);
+ }
+
+ HadoopRpcResponseProto response = constructProtoSpecificRpcSuccessResponse(result);
+ return new RpcResponseWritable(response);
+ }
+ }
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolInfo.java
index 924fa8b1501..2bdd1830500 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolInfo.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolInfo.java
@@ -35,4 +35,5 @@ import java.lang.annotation.RetentionPolicy;
@Retention(RetentionPolicy.RUNTIME)
public @interface ProtocolInfo {
String protocolName(); // the name of the protocol (i.e. rpc service)
+ long protocolVersion() default -1; // default means not defined use old way
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java
index 937031c6748..f15c0837dc8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java
@@ -57,19 +57,11 @@ public class ProtocolProxy {
private void fetchServerMethods(Method method) throws IOException {
long clientVersion;
- try {
- Field versionField = method.getDeclaringClass().getField("versionID");
- versionField.setAccessible(true);
- clientVersion = versionField.getLong(method.getDeclaringClass());
- } catch (NoSuchFieldException ex) {
- throw new RuntimeException(ex);
- } catch (IllegalAccessException ex) {
- throw new RuntimeException(ex);
- }
+ clientVersion = RPC.getProtocolVersion(method.getDeclaringClass());
int clientMethodsHash = ProtocolSignature.getFingerprint(method
.getDeclaringClass().getMethods());
ProtocolSignature serverInfo = ((VersionedProtocol) proxy)
- .getProtocolSignature(protocol.getName(), clientVersion,
+ .getProtocolSignature(RPC.getProtocolName(protocol), clientVersion,
clientMethodsHash);
long serverVersion = serverInfo.getVersion();
if (serverVersion != clientVersion) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java
index 04d08c5142d..6be1c2ab6b6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
+import com.google.common.annotations.VisibleForTesting;
+
public class ProtocolSignature implements Writable {
static { // register a ctor
WritableFactories.setFactory
@@ -164,10 +166,15 @@ public class ProtocolSignature implements Writable {
/**
* A cache that maps a protocol's name to its signature & finger print
*/
- final private static HashMap
+ private final static HashMap
PROTOCOL_FINGERPRINT_CACHE =
new HashMap();
+ @VisibleForTesting
+ public static void resetCache() {
+ PROTOCOL_FINGERPRINT_CACHE.clear();
+ }
+
/**
* Return a protocol's signature and finger print from cache
*
@@ -177,7 +184,7 @@ public class ProtocolSignature implements Writable {
*/
private static ProtocolSigFingerprint getSigFingerprint(
Class extends VersionedProtocol> protocol, long serverVersion) {
- String protocolName = protocol.getName();
+ String protocolName = RPC.getProtocolName(protocol);
synchronized (PROTOCOL_FINGERPRINT_CACHE) {
ProtocolSigFingerprint sig = PROTOCOL_FINGERPRINT_CACHE.get(protocolName);
if (sig == null) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
index 9500c9bfe96..cf597468682 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.ipc;
+import java.lang.reflect.Field;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.Proxy;
import java.lang.reflect.Method;
@@ -28,6 +29,9 @@ import java.net.NoRouteToHostException;
import java.net.SocketTimeoutException;
import java.io.*;
import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
import java.util.Map;
import java.util.HashMap;
@@ -36,6 +40,7 @@ import javax.net.SocketFactory;
import org.apache.commons.logging.*;
import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SaslRpcServer;
import org.apache.hadoop.security.UserGroupInformation;
@@ -63,8 +68,54 @@ import org.apache.hadoop.util.ReflectionUtils;
* the protocol instance is transmitted.
*/
public class RPC {
+
+ interface RpcInvoker {
+ /**
+ * Process a client call on the server side
+ * @param server the server within whose context this rpc call is made
+ * @param protocol - the protocol name (the class of the client proxy
+ * used to make calls to the rpc server.
+ * @param rpcRequest - deserialized
+ * @param receiveTime time at which the call received (for metrics)
+ * @return the call's return
+ * @throws IOException
+ **/
+ public Writable call(Server server, String protocol,
+ Writable rpcRequest, long receiveTime) throws IOException ;
+ }
+
static final Log LOG = LogFactory.getLog(RPC.class);
+ /**
+ * Get all superInterfaces that extend VersionedProtocol
+ * @param childInterfaces
+ * @return the super interfaces that extend VersionedProtocol
+ */
+ static Class>[] getSuperInterfaces(Class>[] childInterfaces) {
+ List> allInterfaces = new ArrayList>();
+
+ for (Class> childInterface : childInterfaces) {
+ if (VersionedProtocol.class.isAssignableFrom(childInterface)) {
+ allInterfaces.add(childInterface);
+ allInterfaces.addAll(
+ Arrays.asList(
+ getSuperInterfaces(childInterface.getInterfaces())));
+ } else {
+ LOG.warn("Interface " + childInterface +
+ " ignored because it does not extend VersionedProtocol");
+ }
+ }
+ return allInterfaces.toArray(new Class[allInterfaces.size()]);
+ }
+
+ /**
+ * Get all interfaces that the given protocol implements or extends
+ * which are assignable from VersionedProtocol.
+ */
+ static Class>[] getProtocolInterfaces(Class> protocol) {
+ Class>[] interfaces = protocol.getInterfaces();
+ return getSuperInterfaces(interfaces);
+ }
/**
* Get the protocol name.
@@ -75,9 +126,36 @@ public class RPC {
if (protocol == null) {
return null;
}
- ProtocolInfo anno = (ProtocolInfo) protocol.getAnnotation(ProtocolInfo.class);
+ ProtocolInfo anno = protocol.getAnnotation(ProtocolInfo.class);
return (anno == null) ? protocol.getName() : anno.protocolName();
}
+
+ /**
+ * Get the protocol version from protocol class.
+ * If the protocol class has a ProtocolAnnotation, then get the protocol
+ * name from the annotation; otherwise the class name is the protocol name.
+ */
+ static public long getProtocolVersion(Class> protocol) {
+ if (protocol == null) {
+ throw new IllegalArgumentException("Null protocol");
+ }
+ long version;
+ ProtocolInfo anno = protocol.getAnnotation(ProtocolInfo.class);
+ if (anno != null) {
+ version = anno.protocolVersion();
+ if (version != -1)
+ return version;
+ }
+ try {
+ Field versionField = protocol.getField("versionID");
+ versionField.setAccessible(true);
+ return versionField.getLong(protocol);
+ } catch (NoSuchFieldException ex) {
+ throw new RuntimeException(ex);
+ } catch (IllegalAccessException ex) {
+ throw new RuntimeException(ex);
+ }
+ }
private RPC() {} // no public ctor
@@ -590,6 +668,144 @@ public class RPC {
/** An RPC Server. */
public abstract static class Server extends org.apache.hadoop.ipc.Server {
+ boolean verbose;
+ static String classNameBase(String className) {
+ String[] names = className.split("\\.", -1);
+ if (names == null || names.length == 0) {
+ return className;
+ }
+ return names[names.length-1];
+ }
+
+ /**
+ * Store a map of protocol and version to its implementation
+ */
+ /**
+ * The key in Map
+ */
+ static class ProtoNameVer {
+ final String protocol;
+ final long version;
+ ProtoNameVer(String protocol, long ver) {
+ this.protocol = protocol;
+ this.version = ver;
+ }
+ @Override
+ public boolean equals(Object o) {
+ if (o == null)
+ return false;
+ if (this == o)
+ return true;
+ if (! (o instanceof ProtoNameVer))
+ return false;
+ ProtoNameVer pv = (ProtoNameVer) o;
+ return ((pv.protocol.equals(this.protocol)) &&
+ (pv.version == this.version));
+ }
+ @Override
+ public int hashCode() {
+ return protocol.hashCode() * 37 + (int) version;
+ }
+ }
+
+ /**
+ * The value in map
+ */
+ static class ProtoClassProtoImpl {
+ final Class> protocolClass;
+ final Object protocolImpl;
+ ProtoClassProtoImpl(Class> protocolClass, Object protocolImpl) {
+ this.protocolClass = protocolClass;
+ this.protocolImpl = protocolImpl;
+ }
+ }
+
+ ArrayList> protocolImplMapArray =
+ new ArrayList>(RpcKind.MAX_INDEX);
+
+ Map getProtocolImplMap(RpcKind rpcKind) {
+ if (protocolImplMapArray.size() == 0) {// initialize for all rpc kinds
+ for (int i=0; i <= RpcKind.MAX_INDEX; ++i) {
+ protocolImplMapArray.add(
+ new HashMap(10));
+ }
+ }
+ return protocolImplMapArray.get(rpcKind.ordinal());
+ }
+
+ // Register protocol and its impl for rpc calls
+ void registerProtocolAndImpl(RpcKind rpcKind, Class> protocolClass,
+ Object protocolImpl) throws IOException {
+ String protocolName = RPC.getProtocolName(protocolClass);
+ long version;
+
+
+ try {
+ version = RPC.getProtocolVersion(protocolClass);
+ } catch (Exception ex) {
+ LOG.warn("Protocol " + protocolClass +
+ " NOT registered as cannot get protocol version ");
+ return;
+ }
+
+
+ getProtocolImplMap(rpcKind).put(new ProtoNameVer(protocolName, version),
+ new ProtoClassProtoImpl(protocolClass, protocolImpl));
+ LOG.debug("RpcKind = " + rpcKind + " Protocol Name = " + protocolName + " version=" + version +
+ " ProtocolImpl=" + protocolImpl.getClass().getName() +
+ " protocolClass=" + protocolClass.getName());
+ }
+
+ static class VerProtocolImpl {
+ final long version;
+ final ProtoClassProtoImpl protocolTarget;
+ VerProtocolImpl(long ver, ProtoClassProtoImpl protocolTarget) {
+ this.version = ver;
+ this.protocolTarget = protocolTarget;
+ }
+ }
+
+
+ @SuppressWarnings("unused") // will be useful later.
+ VerProtocolImpl[] getSupportedProtocolVersions(RpcKind rpcKind,
+ String protocolName) {
+ VerProtocolImpl[] resultk =
+ new VerProtocolImpl[getProtocolImplMap(rpcKind).size()];
+ int i = 0;
+ for (Map.Entry pv :
+ getProtocolImplMap(rpcKind).entrySet()) {
+ if (pv.getKey().protocol.equals(protocolName)) {
+ resultk[i++] =
+ new VerProtocolImpl(pv.getKey().version, pv.getValue());
+ }
+ }
+ if (i == 0) {
+ return null;
+ }
+ VerProtocolImpl[] result = new VerProtocolImpl[i];
+ System.arraycopy(resultk, 0, result, 0, i);
+ return result;
+ }
+
+ VerProtocolImpl getHighestSupportedProtocol(RpcKind rpcKind,
+ String protocolName) {
+ Long highestVersion = 0L;
+ ProtoClassProtoImpl highest = null;
+ System.out.println("Size of protoMap for " + rpcKind + " =" + getProtocolImplMap(rpcKind).size());
+ for (Map.Entry pv :
+ getProtocolImplMap(rpcKind).entrySet()) {
+ if (pv.getKey().protocol.equals(protocolName)) {
+ if ((highest == null) || (pv.getKey().version > highestVersion)) {
+ highest = pv.getValue();
+ highestVersion = pv.getKey().version;
+ }
+ }
+ }
+ if (highest == null) {
+ return null;
+ }
+ return new VerProtocolImpl(highestVersion, highest);
+ }
protected Server(String bindAddress, int port,
Class extends Writable> paramClass, int handlerCount,
@@ -606,11 +822,17 @@ public class RPC {
* @param protocolImpl - the impl of the protocol that will be called
* @return the server (for convenience)
*/
- public
- Server addProtocol(Class protocolClass, IMPL protocolImpl
- ) throws IOException {
- throw new IOException("addProtocol Not Implemented");
+ public Server addProtocol(RpcKind rpcKind, Class> protocolClass,
+ Object protocolImpl) throws IOException {
+ registerProtocolAndImpl(rpcKind, protocolClass, protocolImpl);
+ return this;
+ }
+
+ @Override
+ public Writable call(RpcKind rpcKind, String protocol,
+ Writable rpcRequest, long receiveTime) throws IOException {
+ return getRpcInvoker(rpcKind).call(this, protocol, rpcRequest,
+ receiveTime);
}
}
-
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
index 1b62f0caa7f..430e0a9dea2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
@@ -54,13 +54,14 @@ public class RpcPayloadHeader implements Writable {
}
public enum RpcKind {
- RPC_BUILTIN ((short ) 1), // Used for built in calls
- RPC_WRITABLE ((short ) 2),
- RPC_PROTOCOL_BUFFER ((short)3),
- RPC_AVRO ((short)4);
-
+ 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
+ private static final short FIRST_INDEX = RPC_BUILTIN.value;
private final short value;
- private static final short FIRST_INDEX = RPC_BUILTIN.value;
+
RpcKind(short val) {
this.value = val;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index 0a778729663..97c36fdafdb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -43,6 +43,7 @@ import java.nio.channels.WritableByteChannel;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
@@ -66,6 +67,7 @@ import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.ipc.RPC.RpcInvoker;
import org.apache.hadoop.ipc.RPC.VersionMismatch;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcPayloadOperation;
import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
@@ -133,6 +135,59 @@ public abstract class Server {
* Initial and max size of response buffer
*/
static int INITIAL_RESP_BUF_SIZE = 10240;
+
+ static class RpcKindMapValue {
+ final Class extends Writable> rpcRequestWrapperClass;
+ final RpcInvoker rpcInvoker;
+ RpcKindMapValue (Class extends Writable> rpcRequestWrapperClass,
+ RpcInvoker rpcInvoker) {
+ this.rpcInvoker = rpcInvoker;
+ this.rpcRequestWrapperClass = rpcRequestWrapperClass;
+ }
+ }
+ static Map rpcKindMap = new
+ HashMap(4);
+
+
+
+ /**
+ * Register a RPC kind and the class to deserialize the rpc request.
+ *
+ * Called by static initializers of rpcKind Engines
+ * @param rpcKind
+ * @param rpcRequestWrapperClass - this class is used to deserialze the
+ * the rpc request.
+ * @param rpcInvoker - use to process the calls on SS.
+ */
+
+ public static void registerProtocolEngine(RpcKind rpcKind,
+ Class extends Writable> rpcRequestWrapperClass,
+ RpcInvoker rpcInvoker) {
+ RpcKindMapValue old =
+ rpcKindMap.put(rpcKind, new RpcKindMapValue(rpcRequestWrapperClass, rpcInvoker));
+ if (old != null) {
+ rpcKindMap.put(rpcKind, old);
+ throw new IllegalArgumentException("ReRegistration of rpcKind: " +
+ rpcKind);
+ }
+ LOG.info("rpcKind=" + rpcKind +
+ ", rpcRequestWrapperClass=" + rpcRequestWrapperClass +
+ ", rpcInvoker=" + rpcInvoker);
+ }
+
+ public Class extends Writable> getRpcRequestWrapper(
+ RpcKind rpcKind) {
+ if (rpcRequestClass != null)
+ return rpcRequestClass;
+ RpcKindMapValue val = rpcKindMap.get(rpcKind);
+ return (val == null) ? null : val.rpcRequestWrapperClass;
+ }
+
+ public static RpcInvoker getRpcInvoker(RpcKind rpcKind) {
+ RpcKindMapValue val = rpcKindMap.get(rpcKind);
+ return (val == null) ? null : val.rpcInvoker;
+ }
+
public static final Log LOG = LogFactory.getLog(Server.class);
public static final Log AUDITLOG =
@@ -197,7 +252,7 @@ public abstract class Server {
private int port; // port we listen on
private int handlerCount; // number of handler threads
private int readThreads; // number of read threads
- private Class extends Writable> paramClass; // class of call parameters
+ private Class extends Writable> rpcRequestClass; // class used for deserializing the rpc request
private int maxIdleTime; // the maximum idle time after
// which a client may be disconnected
private int thresholdIdleConnections; // the number of idle connections
@@ -1425,9 +1480,27 @@ public abstract class Server {
throw new IOException("IPC Server does not implement operation" +
header.getOperation());
}
+ // If we know the rpc kind, get its class so that we can deserialize
+ // (Note it would make more sense to have the handler deserialize but
+ // we continue with this original design.
+ Class extends Writable> rpcRequestClass =
+ getRpcRequestWrapper(header.getkind());
+ if (rpcRequestClass == null) {
+ LOG.warn("Unknown rpc kind " + header.getkind() +
+ " from client " + getHostAddress());
+ final Call readParamsFailedCall =
+ new Call(header.getCallId(), null, this);
+ ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
+
+ setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null,
+ IOException.class.getName(),
+ "Unknown rpc kind " + header.getkind());
+ responder.doRespond(readParamsFailedCall);
+ return;
+ }
Writable rpcRequest;
try { //Read the rpc request
- rpcRequest = ReflectionUtils.newInstance(paramClass, conf);
+ rpcRequest = ReflectionUtils.newInstance(rpcRequestClass, conf);
rpcRequest.readFields(dis);
} catch (Throwable t) {
LOG.warn("Unable to read call parameters for client " +
@@ -1519,7 +1592,7 @@ public abstract class Server {
// Make the call as the user via Subject.doAs, thus associating
// the call with the Subject
if (call.connection.user == null) {
- value = call(call.connection.protocolName, call.rpcRequest,
+ value = call(call.rpcKind, call.connection.protocolName, call.rpcRequest,
call.timestamp);
} else {
value =
@@ -1528,7 +1601,7 @@ public abstract class Server {
@Override
public Writable run() throws Exception {
// make the call
- return call(call.connection.protocolName,
+ return call(call.rpcKind, call.connection.protocolName,
call.rpcRequest, call.timestamp);
}
@@ -1590,24 +1663,33 @@ public abstract class Server {
Configuration conf)
throws IOException
{
- this(bindAddress, port, paramClass, handlerCount, -1, -1, conf, Integer.toString(port), null);
+ this(bindAddress, port, paramClass, handlerCount, -1, -1, conf, Integer
+ .toString(port), null);
}
- /** Constructs a server listening on the named port and address. Parameters passed must
+ /**
+ * Constructs a server listening on the named port and address. Parameters passed must
* be of the named class. The handlerCount determines
* the number of handler threads that will be used to process calls.
* If queueSizePerHandler or numReaders are not -1 they will be used instead of parameters
* from configuration. Otherwise the configuration will be picked up.
+ *
+ * If rpcRequestClass is null then the rpcRequestClass must have been
+ * registered via {@link #registerProtocolEngine(RpcPayloadHeader.RpcKind,
+ * Class, RPC.RpcInvoker)}
+ * This parameter has been retained for compatibility with existing tests
+ * and usage.
*/
@SuppressWarnings("unchecked")
- protected Server(String bindAddress, int port,
- Class extends Writable> paramClass, int handlerCount, int numReaders, int queueSizePerHandler,
- Configuration conf, String serverName, SecretManager extends TokenIdentifier> secretManager)
+ protected Server(String bindAddress, int port,
+ Class extends Writable> rpcRequestClass, int handlerCount,
+ int numReaders, int queueSizePerHandler, Configuration conf,
+ String serverName, SecretManager extends TokenIdentifier> secretManager)
throws IOException {
this.bindAddress = bindAddress;
this.conf = conf;
this.port = port;
- this.paramClass = paramClass;
+ this.rpcRequestClass = rpcRequestClass;
this.handlerCount = handlerCount;
this.socketSendBufferSize = 0;
if (queueSizePerHandler != -1) {
@@ -1797,17 +1879,17 @@ public abstract class Server {
/**
* Called for each call.
- * @deprecated Use {@link #call(String, Writable, long)} instead
+ * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, String,
+ * Writable, long)} instead
*/
@Deprecated
public Writable call(Writable param, long receiveTime) throws IOException {
- return call(null, param, receiveTime);
+ return call(RpcKind.RPC_BUILTIN, null, param, receiveTime);
}
/** Called for each call. */
- public abstract Writable call(String protocol,
- Writable param, long receiveTime)
- throws IOException;
+ public abstract Writable call(RpcKind rpcKind, String protocol,
+ Writable param, long receiveTime) throws IOException;
/**
* Authorize the incoming client connection.
@@ -1957,5 +2039,5 @@ public abstract class Server {
int nBytes = initialRemaining - buf.remaining();
return (nBytes > 0) ? nBytes : ret;
- }
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
index 69538148914..25f46f13e23 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.ipc;
-import java.lang.reflect.Field;
import java.lang.reflect.Proxy;
import java.lang.reflect.Method;
import java.lang.reflect.Array;
@@ -27,18 +26,14 @@ import java.lang.reflect.InvocationTargetException;
import java.net.InetSocketAddress;
import java.io.*;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
import java.io.Closeable;
-import java.util.Map;
-import java.util.HashMap;
import javax.net.SocketFactory;
import org.apache.commons.logging.*;
import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.RPC.RpcInvoker;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.UserGroupInformation;
@@ -53,36 +48,9 @@ import org.apache.hadoop.conf.*;
public class WritableRpcEngine implements RpcEngine {
private static final Log LOG = LogFactory.getLog(RPC.class);
-
- /**
- * Get all superInterfaces that extend VersionedProtocol
- * @param childInterfaces
- * @return the super interfaces that extend VersionedProtocol
- */
- private static Class>[] getSuperInterfaces(Class>[] childInterfaces) {
- List> allInterfaces = new ArrayList>();
-
- for (Class> childInterface : childInterfaces) {
- if (VersionedProtocol.class.isAssignableFrom(childInterface)) {
- allInterfaces.add(childInterface);
- allInterfaces.addAll(
- Arrays.asList(
- getSuperInterfaces(childInterface.getInterfaces())));
- } else {
- LOG.warn("Interface " + childInterface +
- " ignored because it does not extend VersionedProtocol");
- }
- }
- return (Class>[]) allInterfaces.toArray(new Class[allInterfaces.size()]);
- }
-
- /**
- * Get all interfaces that the given protocol implements or extends
- * which are assignable from VersionedProtocol.
- */
- private static Class>[] getProtocolInterfaces(Class> protocol) {
- Class>[] interfaces = protocol.getInterfaces();
- return getSuperInterfaces(interfaces);
+ static { // Register the rpcRequest deserializer for WritableRpcEngine
+ org.apache.hadoop.ipc.Server.registerProtocolEngine(RpcKind.RPC_WRITABLE,
+ Invocation.class, new Server.WritableRpcInvoker());
}
@@ -120,15 +88,7 @@ public class WritableRpcEngine implements RpcEngine {
clientVersion = 0;
clientMethodsHash = 0;
} else {
- try {
- Field versionField = method.getDeclaringClass().getField("versionID");
- versionField.setAccessible(true);
- this.clientVersion = versionField.getLong(method.getDeclaringClass());
- } catch (NoSuchFieldException ex) {
- throw new RuntimeException(ex);
- } catch (IllegalAccessException ex) {
- throw new RuntimeException(ex);
- }
+ this.clientVersion = RPC.getProtocolVersion(method.getDeclaringClass());
this.clientMethodsHash = ProtocolSignature.getFingerprint(method
.getDeclaringClass().getMethods());
}
@@ -329,140 +289,25 @@ public class WritableRpcEngine implements RpcEngine {
/** An RPC Server. */
public static class Server extends RPC.Server {
- private boolean verbose;
-
- /**
- * The key in Map
- */
- static class ProtoNameVer {
- final String protocol;
- final long version;
- ProtoNameVer(String protocol, long ver) {
- this.protocol = protocol;
- this.version = ver;
- }
- @Override
- public boolean equals(Object o) {
- if (o == null)
- return false;
- if (this == o)
- return true;
- if (! (o instanceof ProtoNameVer))
- return false;
- ProtoNameVer pv = (ProtoNameVer) o;
- return ((pv.protocol.equals(this.protocol)) &&
- (pv.version == this.version));
- }
- @Override
- public int hashCode() {
- return protocol.hashCode() * 37 + (int) version;
- }
- }
-
- /**
- * The value in map
- */
- static class ProtoClassProtoImpl {
- final Class> protocolClass;
- final Object protocolImpl;
- ProtoClassProtoImpl(Class> protocolClass, Object protocolImpl) {
- this.protocolClass = protocolClass;
- this.protocolImpl = protocolImpl;
- }
- }
-
- private Map protocolImplMap =
- new HashMap(10);
-
- // Register protocol and its impl for rpc calls
- private void registerProtocolAndImpl(Class> protocolClass,
- Object protocolImpl) throws IOException {
- String protocolName = RPC.getProtocolName(protocolClass);
- VersionedProtocol vp = (VersionedProtocol) protocolImpl;
- long version;
- try {
- version = vp.getProtocolVersion(protocolName, 0);
- } catch (Exception ex) {
- LOG.warn("Protocol " + protocolClass +
- " NOT registered as getProtocolVersion throws exception ");
- return;
- }
- protocolImplMap.put(new ProtoNameVer(protocolName, version),
- new ProtoClassProtoImpl(protocolClass, protocolImpl));
- LOG.debug("Protocol Name = " + protocolName + " version=" + version +
- " ProtocolImpl=" + protocolImpl.getClass().getName() +
- " protocolClass=" + protocolClass.getName());
- }
-
- private static class VerProtocolImpl {
- final long version;
- final ProtoClassProtoImpl protocolTarget;
- VerProtocolImpl(long ver, ProtoClassProtoImpl protocolTarget) {
- this.version = ver;
- this.protocolTarget = protocolTarget;
- }
- }
-
-
- @SuppressWarnings("unused") // will be useful later.
- private VerProtocolImpl[] getSupportedProtocolVersions(
- String protocolName) {
- VerProtocolImpl[] resultk = new VerProtocolImpl[protocolImplMap.size()];
- int i = 0;
- for (Map.Entry pv :
- protocolImplMap.entrySet()) {
- if (pv.getKey().protocol.equals(protocolName)) {
- resultk[i++] =
- new VerProtocolImpl(pv.getKey().version, pv.getValue());
- }
- }
- if (i == 0) {
- return null;
- }
- VerProtocolImpl[] result = new VerProtocolImpl[i];
- System.arraycopy(resultk, 0, result, 0, i);
- return result;
- }
-
- private VerProtocolImpl getHighestSupportedProtocol(String protocolName) {
- Long highestVersion = 0L;
- ProtoClassProtoImpl highest = null;
- for (Map.Entry pv : protocolImplMap
- .entrySet()) {
- if (pv.getKey().protocol.equals(protocolName)) {
- if ((highest == null) || (pv.getKey().version > highestVersion)) {
- highest = pv.getValue();
- highestVersion = pv.getKey().version;
- }
- }
- }
- if (highest == null) {
- return null;
- }
- return new VerProtocolImpl(highestVersion, highest);
- }
-
-
- /** Construct an RPC server.
+ /**
+ * 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
*
- * @deprecated Use #Server(Class, Object, Configuration, String, int)
- *
+ * @deprecated Use #Server(Class, Object, Configuration, String, int)
*/
@Deprecated
public Server(Object instance, Configuration conf, String bindAddress,
- int port)
- throws IOException {
+ int port) throws IOException {
this(null, instance, conf, bindAddress, port);
}
/** Construct an RPC server.
- * @param protocol class
- * @param instance the instance whose methods will be called
+ * @param protocolClass class
+ * @param protocolImpl 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
@@ -474,16 +319,8 @@ public class WritableRpcEngine implements RpcEngine {
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.
+ /**
+ * Construct an RPC server.
* @param protocolImpl the instance whose methods will be called
* @param conf the configuration to use
* @param bindAddress the address to bind on to listen for connection
@@ -505,7 +342,8 @@ public class WritableRpcEngine implements RpcEngine {
}
- /** Construct an RPC server.
+ /**
+ * Construct an RPC server.
* @param protocolClass - the protocol being registered
* can be null for compatibility with old usage (see below for details)
* @param protocolImpl the protocol impl that will be called
@@ -520,7 +358,7 @@ public class WritableRpcEngine implements RpcEngine {
int numHandlers, int numReaders, int queueSizePerHandler,
boolean verbose, SecretManager extends TokenIdentifier> secretManager)
throws IOException {
- super(bindAddress, port, Invocation.class, numHandlers, numReaders,
+ super(bindAddress, port, null, numHandlers, numReaders,
queueSizePerHandler, conf,
classNameBase(protocolImpl.getClass().getName()), secretManager);
@@ -535,7 +373,7 @@ public class WritableRpcEngine implements RpcEngine {
* the protocolImpl is derived from the protocolClass(es)
* we register all interfaces extended by the protocolImpl
*/
- protocols = getProtocolInterfaces(protocolImpl.getClass());
+ protocols = RPC.getProtocolInterfaces(protocolImpl.getClass());
} else {
if (!protocolClass.isAssignableFrom(protocolImpl.getClass())) {
@@ -544,132 +382,125 @@ public class WritableRpcEngine implements RpcEngine {
protocolImpl.getClass());
}
// register protocol class and its super interfaces
- registerProtocolAndImpl(protocolClass, protocolImpl);
- protocols = getProtocolInterfaces(protocolClass);
+ registerProtocolAndImpl(RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
+ protocols = RPC.getProtocolInterfaces(protocolClass);
}
for (Class> p : protocols) {
if (!p.equals(VersionedProtocol.class)) {
- registerProtocolAndImpl(p, protocolImpl);
+ registerProtocolAndImpl(RpcKind.RPC_WRITABLE, p, protocolImpl);
}
}
}
-
- @Override
- public Server
- addProtocol(
- Class protocolClass, IMPL protocolImpl) throws IOException {
- registerProtocolAndImpl(protocolClass, protocolImpl);
- return this;
+ private static void log(String value) {
+ if (value!= null && value.length() > 55)
+ value = value.substring(0, 55)+"...";
+ LOG.info(value);
}
- /**
- * Process a client call
- * @param protocolName - the protocol name (the class of the client proxy
- * used to make calls to the rpc server.
- * @param param parameters
- * @param receivedTime time at which the call receoved (for metrics)
- * @return the call's return
- * @throws IOException
- */
- public Writable call(String protocolName, Writable param, long receivedTime)
- throws IOException {
- try {
- Invocation call = (Invocation)param;
- if (verbose) log("Call: " + call);
+ static class WritableRpcInvoker implements RpcInvoker {
- // Verify rpc version
- if (call.getRpcVersion() != writableRpcVersion) {
- // Client is using a different version of WritableRpc
- throw new IOException(
- "WritableRpc version mismatch, client side version="
- + call.getRpcVersion() + ", server side version="
- + writableRpcVersion);
- }
+ @Override
+ public Writable call(org.apache.hadoop.ipc.RPC.Server server,
+ String protocolName, Writable rpcRequest, long receivedTime)
+ throws IOException {
+ try {
+ Invocation call = (Invocation)rpcRequest;
+ if (server.verbose) log("Call: " + call);
- long clientVersion = call.getProtocolVersion();
- final String protoName;
- ProtoClassProtoImpl protocolImpl;
- if (call.declaringClassProtocolName.equals(VersionedProtocol.class.getName())) {
- // VersionProtocol methods are often used by client to figure out
- // which version of protocol to use.
- //
- // Versioned protocol methods should go the protocolName protocol
- // rather than the declaring class of the method since the
- // the declaring class is VersionedProtocol which is not
- // registered directly.
- // Send the call to the highest protocol version
- protocolImpl =
- getHighestSupportedProtocol(protocolName).protocolTarget;
- } else {
- protoName = call.declaringClassProtocolName;
+ // Verify rpc version
+ if (call.getRpcVersion() != writableRpcVersion) {
+ // Client is using a different version of WritableRpc
+ throw new IOException(
+ "WritableRpc version mismatch, client side version="
+ + call.getRpcVersion() + ", server side version="
+ + writableRpcVersion);
+ }
- // Find the right impl for the protocol based on client version.
- ProtoNameVer pv =
- new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
- protocolImpl = protocolImplMap.get(pv);
- if (protocolImpl == null) { // no match for Protocol AND Version
- VerProtocolImpl highest =
- getHighestSupportedProtocol(protoName);
+ long clientVersion = call.getProtocolVersion();
+ final String protoName;
+ ProtoClassProtoImpl protocolImpl;
+ if (call.declaringClassProtocolName.equals(VersionedProtocol.class.getName())) {
+ // VersionProtocol methods are often used by client to figure out
+ // which version of protocol to use.
+ //
+ // Versioned protocol methods should go the protocolName protocol
+ // rather than the declaring class of the method since the
+ // the declaring class is VersionedProtocol which is not
+ // registered directly.
+ // Send the call to the highest protocol version
+ VerProtocolImpl highest = server.getHighestSupportedProtocol(
+ RpcKind.RPC_WRITABLE, protocolName);
if (highest == null) {
- throw new IOException("Unknown protocol: " + protoName);
- } else { // protocol supported but not the version that client wants
- throw new RPC.VersionMismatch(protoName, clientVersion,
- highest.version);
+ throw new IOException("Unknown protocol: " + protocolName);
+ }
+ protocolImpl = highest.protocolTarget;
+ } else {
+ protoName = call.declaringClassProtocolName;
+
+ // Find the right impl for the protocol based on client version.
+ ProtoNameVer pv =
+ new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
+ protocolImpl =
+ server.getProtocolImplMap(RpcKind.RPC_WRITABLE).get(pv);
+ if (protocolImpl == null) { // no match for Protocol AND Version
+ VerProtocolImpl highest =
+ server.getHighestSupportedProtocol(RpcKind.RPC_WRITABLE,
+ protoName);
+ if (highest == null) {
+ throw new IOException("Unknown protocol: " + protoName);
+ } else { // protocol supported but not the version that client wants
+ throw new RPC.VersionMismatch(protoName, clientVersion,
+ highest.version);
+ }
}
}
- }
-
+
- // Invoke the protocol method
+ // Invoke the protocol method
- long startTime = System.currentTimeMillis();
- Method method =
- protocolImpl.protocolClass.getMethod(call.getMethodName(),
- call.getParameterClasses());
- method.setAccessible(true);
- rpcDetailedMetrics.init(protocolImpl.protocolClass);
- Object value =
- method.invoke(protocolImpl.protocolImpl, call.getParameters());
- int processingTime = (int) (System.currentTimeMillis() - startTime);
- int qTime = (int) (startTime-receivedTime);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Served: " + call.getMethodName() +
- " queueTime= " + qTime +
- " procesingTime= " + processingTime);
- }
- rpcMetrics.addRpcQueueTime(qTime);
- rpcMetrics.addRpcProcessingTime(processingTime);
- rpcDetailedMetrics.addProcessingTime(call.getMethodName(),
- processingTime);
- if (verbose) log("Return: "+value);
+ long startTime = System.currentTimeMillis();
+ Method method =
+ protocolImpl.protocolClass.getMethod(call.getMethodName(),
+ call.getParameterClasses());
+ method.setAccessible(true);
+ server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
+ Object value =
+ method.invoke(protocolImpl.protocolImpl, call.getParameters());
+ int processingTime = (int) (System.currentTimeMillis() - startTime);
+ int qTime = (int) (startTime-receivedTime);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Served: " + call.getMethodName() +
+ " queueTime= " + qTime +
+ " procesingTime= " + processingTime);
+ }
+ server.rpcMetrics.addRpcQueueTime(qTime);
+ server.rpcMetrics.addRpcProcessingTime(processingTime);
+ server.rpcDetailedMetrics.addProcessingTime(call.getMethodName(),
+ processingTime);
+ if (server.verbose) log("Return: "+value);
- return new ObjectWritable(method.getReturnType(), value);
+ return new ObjectWritable(method.getReturnType(), value);
- } catch (InvocationTargetException e) {
- Throwable target = e.getTargetException();
- if (target instanceof IOException) {
- throw (IOException)target;
- } else {
- IOException ioe = new IOException(target.toString());
- ioe.setStackTrace(target.getStackTrace());
+ } catch (InvocationTargetException e) {
+ Throwable target = e.getTargetException();
+ if (target instanceof IOException) {
+ throw (IOException)target;
+ } else {
+ IOException ioe = new IOException(target.toString());
+ ioe.setStackTrace(target.getStackTrace());
+ throw ioe;
+ }
+ } catch (Throwable e) {
+ if (!(e instanceof IOException)) {
+ LOG.error("Unexpected throwable object ", e);
+ }
+ IOException ioe = new IOException(e.toString());
+ ioe.setStackTrace(e.getStackTrace());
throw ioe;
}
- } catch (Throwable e) {
- if (!(e instanceof IOException)) {
- LOG.error("Unexpected throwable object ", e);
- }
- IOException ioe = new IOException(e.toString());
- ioe.setStackTrace(e.getStackTrace());
- throw ioe;
}
}
}
-
- private static void log(String value) {
- if (value!= null && value.length() > 55)
- value = value.substring(0, 55)+"...";
- LOG.info(value);
- }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/protobuf/HadoopRpcProtos.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/protobuf/HadoopRpcProtos.java
index 2086f3d86b6..b6d1577eb29 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/protobuf/HadoopRpcProtos.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/protobuf/HadoopRpcProtos.java
@@ -1,3 +1,20 @@
+/**
+ * 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.
+ */
// Generated by the protocol buffer compiler. DO NOT EDIT!
// source: hadoop_rpc.proto
@@ -18,6 +35,14 @@ public final class HadoopRpcProtos {
// optional bytes request = 2;
boolean hasRequest();
com.google.protobuf.ByteString getRequest();
+
+ // required string declaringClassProtocolName = 3;
+ boolean hasDeclaringClassProtocolName();
+ String getDeclaringClassProtocolName();
+
+ // required uint64 clientProtocolVersion = 4;
+ boolean hasClientProtocolVersion();
+ long getClientProtocolVersion();
}
public static final class HadoopRpcRequestProto extends
com.google.protobuf.GeneratedMessage
@@ -90,9 +115,53 @@ public final class HadoopRpcProtos {
return request_;
}
+ // required string declaringClassProtocolName = 3;
+ public static final int DECLARINGCLASSPROTOCOLNAME_FIELD_NUMBER = 3;
+ private java.lang.Object declaringClassProtocolName_;
+ public boolean hasDeclaringClassProtocolName() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ public String getDeclaringClassProtocolName() {
+ java.lang.Object ref = declaringClassProtocolName_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+ declaringClassProtocolName_ = s;
+ }
+ return s;
+ }
+ }
+ private com.google.protobuf.ByteString getDeclaringClassProtocolNameBytes() {
+ java.lang.Object ref = declaringClassProtocolName_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+ declaringClassProtocolName_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // required uint64 clientProtocolVersion = 4;
+ public static final int CLIENTPROTOCOLVERSION_FIELD_NUMBER = 4;
+ private long clientProtocolVersion_;
+ public boolean hasClientProtocolVersion() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ public long getClientProtocolVersion() {
+ return clientProtocolVersion_;
+ }
+
private void initFields() {
methodName_ = "";
request_ = com.google.protobuf.ByteString.EMPTY;
+ declaringClassProtocolName_ = "";
+ clientProtocolVersion_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -103,6 +172,14 @@ public final class HadoopRpcProtos {
memoizedIsInitialized = 0;
return false;
}
+ if (!hasDeclaringClassProtocolName()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasClientProtocolVersion()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
memoizedIsInitialized = 1;
return true;
}
@@ -116,6 +193,12 @@ public final class HadoopRpcProtos {
if (((bitField0_ & 0x00000002) == 0x00000002)) {
output.writeBytes(2, request_);
}
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeBytes(3, getDeclaringClassProtocolNameBytes());
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeUInt64(4, clientProtocolVersion_);
+ }
getUnknownFields().writeTo(output);
}
@@ -133,6 +216,14 @@ public final class HadoopRpcProtos {
size += com.google.protobuf.CodedOutputStream
.computeBytesSize(2, request_);
}
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(3, getDeclaringClassProtocolNameBytes());
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(4, clientProtocolVersion_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -166,6 +257,16 @@ public final class HadoopRpcProtos {
result = result && getRequest()
.equals(other.getRequest());
}
+ result = result && (hasDeclaringClassProtocolName() == other.hasDeclaringClassProtocolName());
+ if (hasDeclaringClassProtocolName()) {
+ result = result && getDeclaringClassProtocolName()
+ .equals(other.getDeclaringClassProtocolName());
+ }
+ result = result && (hasClientProtocolVersion() == other.hasClientProtocolVersion());
+ if (hasClientProtocolVersion()) {
+ result = result && (getClientProtocolVersion()
+ == other.getClientProtocolVersion());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -183,6 +284,14 @@ public final class HadoopRpcProtos {
hash = (37 * hash) + REQUEST_FIELD_NUMBER;
hash = (53 * hash) + getRequest().hashCode();
}
+ if (hasDeclaringClassProtocolName()) {
+ hash = (37 * hash) + DECLARINGCLASSPROTOCOLNAME_FIELD_NUMBER;
+ hash = (53 * hash) + getDeclaringClassProtocolName().hashCode();
+ }
+ if (hasClientProtocolVersion()) {
+ hash = (37 * hash) + CLIENTPROTOCOLVERSION_FIELD_NUMBER;
+ hash = (53 * hash) + hashLong(getClientProtocolVersion());
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
return hash;
}
@@ -303,6 +412,10 @@ public final class HadoopRpcProtos {
bitField0_ = (bitField0_ & ~0x00000001);
request_ = com.google.protobuf.ByteString.EMPTY;
bitField0_ = (bitField0_ & ~0x00000002);
+ declaringClassProtocolName_ = "";
+ bitField0_ = (bitField0_ & ~0x00000004);
+ clientProtocolVersion_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000008);
return this;
}
@@ -349,6 +462,14 @@ public final class HadoopRpcProtos {
to_bitField0_ |= 0x00000002;
}
result.request_ = request_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.declaringClassProtocolName_ = declaringClassProtocolName_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.clientProtocolVersion_ = clientProtocolVersion_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -371,6 +492,12 @@ public final class HadoopRpcProtos {
if (other.hasRequest()) {
setRequest(other.getRequest());
}
+ if (other.hasDeclaringClassProtocolName()) {
+ setDeclaringClassProtocolName(other.getDeclaringClassProtocolName());
+ }
+ if (other.hasClientProtocolVersion()) {
+ setClientProtocolVersion(other.getClientProtocolVersion());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -380,6 +507,14 @@ public final class HadoopRpcProtos {
return false;
}
+ if (!hasDeclaringClassProtocolName()) {
+
+ return false;
+ }
+ if (!hasClientProtocolVersion()) {
+
+ return false;
+ }
return true;
}
@@ -416,6 +551,16 @@ public final class HadoopRpcProtos {
request_ = input.readBytes();
break;
}
+ case 26: {
+ bitField0_ |= 0x00000004;
+ declaringClassProtocolName_ = input.readBytes();
+ break;
+ }
+ case 32: {
+ bitField0_ |= 0x00000008;
+ clientProtocolVersion_ = input.readUInt64();
+ break;
+ }
}
}
}
@@ -482,6 +627,63 @@ public final class HadoopRpcProtos {
return this;
}
+ // required string declaringClassProtocolName = 3;
+ private java.lang.Object declaringClassProtocolName_ = "";
+ public boolean hasDeclaringClassProtocolName() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ public String getDeclaringClassProtocolName() {
+ java.lang.Object ref = declaringClassProtocolName_;
+ if (!(ref instanceof String)) {
+ String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+ declaringClassProtocolName_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setDeclaringClassProtocolName(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000004;
+ declaringClassProtocolName_ = value;
+ onChanged();
+ return this;
+ }
+ public Builder clearDeclaringClassProtocolName() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ declaringClassProtocolName_ = getDefaultInstance().getDeclaringClassProtocolName();
+ onChanged();
+ return this;
+ }
+ void setDeclaringClassProtocolName(com.google.protobuf.ByteString value) {
+ bitField0_ |= 0x00000004;
+ declaringClassProtocolName_ = value;
+ onChanged();
+ }
+
+ // required uint64 clientProtocolVersion = 4;
+ private long clientProtocolVersion_ ;
+ public boolean hasClientProtocolVersion() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ public long getClientProtocolVersion() {
+ return clientProtocolVersion_;
+ }
+ public Builder setClientProtocolVersion(long value) {
+ bitField0_ |= 0x00000008;
+ clientProtocolVersion_ = value;
+ onChanged();
+ return this;
+ }
+ public Builder clearClientProtocolVersion() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ clientProtocolVersion_ = 0L;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:HadoopRpcRequestProto)
}
@@ -1706,16 +1908,18 @@ public final class HadoopRpcProtos {
descriptor;
static {
java.lang.String[] descriptorData = {
- "\n\020hadoop_rpc.proto\"<\n\025HadoopRpcRequestPr" +
+ "\n\020hadoop_rpc.proto\"\177\n\025HadoopRpcRequestPr" +
"oto\022\022\n\nmethodName\030\001 \002(\t\022\017\n\007request\030\002 \001(\014" +
- "\"D\n\027HadoopRpcExceptionProto\022\025\n\rexception" +
- "Name\030\001 \001(\t\022\022\n\nstackTrace\030\002 \001(\t\"\272\001\n\026Hadoo" +
- "pRpcResponseProto\0226\n\006status\030\001 \002(\0162&.Hado" +
- "opRpcResponseProto.ResponseStatus\022\020\n\010res" +
- "ponse\030\002 \001(\014\022+\n\texception\030\003 \001(\0132\030.HadoopR" +
- "pcExceptionProto\")\n\016ResponseStatus\022\013\n\007SU" +
- "CCESS\020\001\022\n\n\006ERRROR\020\002B4\n\036org.apache.hadoop" +
- ".ipc.protobufB\017HadoopRpcProtos\240\001\001"
+ "\022\"\n\032declaringClassProtocolName\030\003 \002(\t\022\035\n\025" +
+ "clientProtocolVersion\030\004 \002(\004\"D\n\027HadoopRpc" +
+ "ExceptionProto\022\025\n\rexceptionName\030\001 \001(\t\022\022\n" +
+ "\nstackTrace\030\002 \001(\t\"\272\001\n\026HadoopRpcResponseP" +
+ "roto\0226\n\006status\030\001 \002(\0162&.HadoopRpcResponse" +
+ "Proto.ResponseStatus\022\020\n\010response\030\002 \001(\014\022+" +
+ "\n\texception\030\003 \001(\0132\030.HadoopRpcExceptionPr" +
+ "oto\")\n\016ResponseStatus\022\013\n\007SUCCESS\020\001\022\n\n\006ER",
+ "RROR\020\002B4\n\036org.apache.hadoop.ipc.protobuf" +
+ "B\017HadoopRpcProtos\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -1727,7 +1931,7 @@ public final class HadoopRpcProtos {
internal_static_HadoopRpcRequestProto_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_HadoopRpcRequestProto_descriptor,
- new java.lang.String[] { "MethodName", "Request", },
+ new java.lang.String[] { "MethodName", "Request", "DeclaringClassProtocolName", "ClientProtocolVersion", },
org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcRequestProto.class,
org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcRequestProto.Builder.class);
internal_static_HadoopRpcExceptionProto_descriptor =
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
index 28d82537fbd..7cea67923e7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
@@ -42,15 +42,20 @@ public class DelegationKey implements Writable {
@Nullable
private byte[] keyBytes = null;
+ /** Default constructore required for Writable */
public DelegationKey() {
- this(0, 0L, null);
+ this(0, 0L, (SecretKey)null);
}
public DelegationKey(int keyId, long expiryDate, SecretKey key) {
+ this(keyId, expiryDate, key != null ? key.getEncoded() : null);
+ }
+
+ public DelegationKey(int keyId, long expiryDate, byte[] encodedKey) {
this.keyId = keyId;
this.expiryDate = expiryDate;
- if (key!=null) {
- this.keyBytes = key.getEncoded();
+ if (encodedKey != null) {
+ this.keyBytes = encodedKey;
}
}
@@ -70,6 +75,10 @@ public class DelegationKey implements Writable {
return key;
}
}
+
+ public byte[] getEncodedKey() {
+ return keyBytes;
+ }
public void setExpiryDate(long expiryDate) {
this.expiryDate = expiryDate;
diff --git a/hadoop-common-project/hadoop-common/src/proto/hadoop_rpc.proto b/hadoop-common-project/hadoop-common/src/proto/hadoop_rpc.proto
index d37455434d8..7af8eae182f 100644
--- a/hadoop-common-project/hadoop-common/src/proto/hadoop_rpc.proto
+++ b/hadoop-common-project/hadoop-common/src/proto/hadoop_rpc.proto
@@ -34,6 +34,12 @@ message HadoopRpcRequestProto {
/** Bytes corresponding to the client protobuf request */
optional bytes request = 2;
+
+ /** protocol name of class declaring the called method */
+ required string declaringClassProtocolName = 3;
+
+ /** protocol version of class declaring the called method */
+ required uint64 clientProtocolVersion = 4;
}
/**
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java
index e7b6657a5c5..5ce3359428a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java
@@ -34,6 +34,7 @@ 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;
@@ -101,7 +102,8 @@ public class TestAvroRpc extends TestCase {
RPC.setProtocolEngine(conf, AvroTestProtocol.class, AvroRpcEngine.class);
RPC.Server server = RPC.getServer(EmptyProtocol.class, new EmptyImpl(),
ADDRESS, 0, 5, true, conf, sm);
- server.addProtocol(AvroTestProtocol.class, new TestImpl());
+ server.addProtocol(RpcKind.RPC_WRITABLE,
+ AvroTestProtocol.class, new TestImpl());
try {
server.start();
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
index 1515ba6216d..1f3e67a4f9a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.*;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.net.NetUtils;
@@ -96,8 +97,8 @@ public class TestIPC {
}
@Override
- public Writable call(String protocol, Writable param, long receiveTime)
- throws IOException {
+ public Writable call(RpcKind rpcKind, String protocol, Writable param,
+ long receiveTime) throws IOException {
if (sleep) {
// sleep a bit
try {
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
index d4400effa7f..5675cbfddf9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
/**
@@ -72,8 +73,8 @@ public class TestIPCServerResponder extends TestCase {
}
@Override
- public Writable call(String protocol, Writable param, long receiveTime)
- throws IOException {
+ public Writable call(RpcKind rpcKind, String protocol, Writable param,
+ long receiveTime) throws IOException {
if (sleep) {
try {
Thread.sleep(RANDOM.nextInt(20)); // sleep a bit
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
index 203c2855bcb..f5acd93eb2f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
@@ -23,10 +23,15 @@ import java.net.InetSocketAddress;
import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.TestProtoBufRpc.PBServerImpl;
+import org.apache.hadoop.ipc.TestProtoBufRpc.TestRpcService;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
import org.apache.hadoop.net.NetUtils;
import org.junit.Before;
import org.junit.After;
import org.junit.Test;
+import com.google.protobuf.BlockingService;
public class TestMultipleProtocolServer {
private static final String ADDRESS = "0.0.0.0";
@@ -173,9 +178,19 @@ public class TestMultipleProtocolServer {
// create a server with two handlers
server = RPC.getServer(Foo0.class,
new Foo0Impl(), ADDRESS, 0, 2, false, conf, null);
- server.addProtocol(Foo1.class, new Foo1Impl());
- server.addProtocol(Bar.class, new BarImpl());
- server.addProtocol(Mixin.class, new BarImpl());
+ server.addProtocol(RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
+ server.addProtocol(RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
+ server.addProtocol(RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
+
+
+ // Add Protobuf server
+ // Create server side implementation
+ PBServerImpl pbServerImpl =
+ new PBServerImpl();
+ BlockingService service = TestProtobufRpcProto
+ .newReflectiveBlockingService(pbServerImpl);
+ server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
+ service);
server.start();
addr = NetUtils.getConnectAddress(server);
}
@@ -251,5 +266,16 @@ public class TestMultipleProtocolServer {
public void testIncorrectServerCreation() throws IOException {
RPC.getServer(Foo1.class,
new Foo0Impl(), ADDRESS, 0, 2, false, conf, null);
+ }
+
+ // Now test a PB service - a server hosts both PB and Writable Rpcs.
+ @Test
+ public void testPBService() throws Exception {
+ // Set RPC engine to protobuf RPC engine
+ Configuration conf2 = new Configuration();
+ RPC.setProtocolEngine(conf2, TestRpcService.class,
+ ProtobufRpcEngine.class);
+ TestRpcService client = RPC.getProxy(TestRpcService.class, 0, addr, conf2);
+ TestProtoBufRpc.testProtoBufRpc(client);
}
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
index 95083ab2faf..ea48b98db4d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
@@ -21,14 +21,18 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto;
import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto;
import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
-import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpc2Proto;
+import org.apache.hadoop.net.NetUtils;
import org.junit.Assert;
import org.junit.Test;
+import org.junit.Before;
+import org.junit.After;
import com.google.protobuf.BlockingService;
import com.google.protobuf.RpcController;
@@ -42,8 +46,21 @@ import com.google.protobuf.ServiceException;
public class TestProtoBufRpc {
public final static String ADDRESS = "0.0.0.0";
public final static int PORT = 0;
+ private static InetSocketAddress addr;
+ private static Configuration conf;
+ private static RPC.Server server;
+
+ @ProtocolInfo(protocolName = "testProto", protocolVersion = 1)
+ public interface TestRpcService
+ extends TestProtobufRpcProto.BlockingInterface {
+ }
- public static class ServerImpl implements BlockingInterface {
+ @ProtocolInfo(protocolName = "testProto2", protocolVersion = 1)
+ public interface TestRpcService2 extends
+ TestProtobufRpc2Proto.BlockingInterface {
+ }
+
+ public static class PBServerImpl implements TestRpcService {
@Override
public EmptyResponseProto ping(RpcController unused,
@@ -64,40 +81,78 @@ public class TestProtoBufRpc {
throw new ServiceException("error", new RpcServerException("error"));
}
}
+
+ public static class PBServer2Impl implements TestRpcService2 {
- private static RPC.Server startRPCServer(Configuration conf)
- throws IOException {
+ @Override
+ public EmptyResponseProto ping2(RpcController unused,
+ EmptyRequestProto request) throws ServiceException {
+ return EmptyResponseProto.newBuilder().build();
+ }
+
+ @Override
+ public EchoResponseProto echo2(RpcController unused, EchoRequestProto request)
+ throws ServiceException {
+ return EchoResponseProto.newBuilder().setMessage(request.getMessage())
+ .build();
+ }
+ }
+
+ @Before
+ public void setUp() throws IOException { // Setup server for both protocols
+ conf = new Configuration();
// Set RPC engine to protobuf RPC engine
- RPC.setProtocolEngine(conf, BlockingService.class, ProtobufRpcEngine.class);
+ RPC.setProtocolEngine(conf, TestRpcService.class, ProtobufRpcEngine.class);
// Create server side implementation
- ServerImpl serverImpl = new ServerImpl();
+ PBServerImpl serverImpl = new PBServerImpl();
BlockingService service = TestProtobufRpcProto
.newReflectiveBlockingService(serverImpl);
- // Get RPC server for serer side implementation
- RPC.Server server = RPC.getServer(BlockingService.class, service, ADDRESS,
- PORT, conf);
+ // Get RPC server for server side implementation
+ server = RPC.getServer(TestRpcService.class, service, ADDRESS, PORT, conf);
+ addr = NetUtils.getConnectAddress(server);
+
+ // now the second protocol
+ PBServer2Impl server2Impl = new PBServer2Impl();
+ BlockingService service2 = TestProtobufRpc2Proto
+ .newReflectiveBlockingService(server2Impl);
+
+ server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService2.class,
+ service2);
server.start();
- return server;
+ }
+
+
+ @After
+ public void tearDown() throws Exception {
+ server.stop();
}
- private static BlockingInterface getClient(Configuration conf,
- InetSocketAddress addr) throws IOException {
+ private static TestRpcService getClient() throws IOException {
// Set RPC engine to protobuf RPC engine
- RPC.setProtocolEngine(conf, BlockingInterface.class,
+ RPC.setProtocolEngine(conf, TestRpcService.class,
ProtobufRpcEngine.class);
- BlockingInterface client = RPC.getProxy(BlockingInterface.class, 0, addr,
+ return RPC.getProxy(TestRpcService.class, 0, addr,
+ conf);
+ }
+
+ private static TestRpcService2 getClient2() throws IOException {
+ // Set RPC engine to protobuf RPC engine
+ RPC.setProtocolEngine(conf, TestRpcService2.class,
+ ProtobufRpcEngine.class);
+ return RPC.getProxy(TestRpcService2.class, 0, addr,
conf);
- return client;
}
@Test
public void testProtoBufRpc() throws Exception {
- Configuration conf = new Configuration();
- RPC.Server server = startRPCServer(conf);
- BlockingInterface client = getClient(conf, server.getListenerAddress());
-
+ TestRpcService client = getClient();
+ testProtoBufRpc(client);
+ }
+
+ // separated test out so that other tests can call it.
+ public static void testProtoBufRpc(TestRpcService client) throws Exception {
// Test ping method
EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build();
client.ping(null, emptyRequest);
@@ -108,16 +163,29 @@ public class TestProtoBufRpc {
EchoResponseProto echoResponse = client.echo(null, echoRequest);
Assert.assertEquals(echoResponse.getMessage(), "hello");
- // Test error method - it should be thrown as RemoteException
+ // Test error method - error should be thrown as RemoteException
try {
client.error(null, emptyRequest);
Assert.fail("Expected exception is not thrown");
} catch (ServiceException e) {
RemoteException re = (RemoteException)e.getCause();
- re.printStackTrace();
RpcServerException rse = (RpcServerException) re
.unwrapRemoteException(RpcServerException.class);
- rse.printStackTrace();
}
}
+
+ @Test
+ public void testProtoBufRpc2() throws Exception {
+ TestRpcService2 client = getClient2();
+
+ // Test ping method
+ EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build();
+ client.ping2(null, emptyRequest);
+
+ // Test echo method
+ EchoRequestProto echoRequest = EchoRequestProto.newBuilder()
+ .setMessage("hello").build();
+ EchoResponseProto echoResponse = client.echo2(null, echoRequest);
+ Assert.assertEquals(echoResponse.getMessage(), "hello");
+ }
}
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
index 85e60dde9fa..d38d8232005 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
@@ -31,6 +31,7 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.junit.After;
import org.junit.Test;
@@ -56,6 +57,8 @@ public class TestRPCCompatibility {
String echo(String value) throws IOException;
}
+
+ // TestProtocol2 is a compatible impl of TestProtocol1 - hence use its name
@ProtocolInfo(protocolName=
"org.apache.hadoop.ipc.TestRPCCompatibility$TestProtocol1")
public interface TestProtocol2 extends TestProtocol1 {
@@ -114,9 +117,11 @@ public class TestRPCCompatibility {
public void tearDown() throws IOException {
if (proxy != null) {
RPC.stopProxy(proxy.getProxy());
+ proxy = null;
}
if (server != null) {
server.stop();
+ server = null;
}
}
@@ -126,7 +131,7 @@ public class TestRPCCompatibility {
TestImpl1 impl = new TestImpl1();
server = RPC.getServer(TestProtocol1.class,
impl, ADDRESS, 0, 2, false, conf, null);
- server.addProtocol(TestProtocol0.class, impl);
+ server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
@@ -170,8 +175,10 @@ public class TestRPCCompatibility {
public int echo(int value) throws IOException, NumberFormatException {
if (serverInfo.isMethodSupported("echo", int.class)) {
+System.out.println("echo int is supported");
return -value; // use version 3 echo long
} else { // server is version 2
+System.out.println("echo int is NOT supported");
return Integer.parseInt(proxy2.echo(String.valueOf(value)));
}
}
@@ -191,7 +198,7 @@ public class TestRPCCompatibility {
TestImpl1 impl = new TestImpl1();
server = RPC.getServer(TestProtocol1.class,
impl, ADDRESS, 0, 2, false, conf, null);
- server.addProtocol(TestProtocol0.class, impl);
+ server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
@@ -207,11 +214,12 @@ public class TestRPCCompatibility {
@Test // equal version client and server
public void testVersion2ClientVersion2Server() throws Exception {
+ ProtocolSignature.resetCache();
// create a server with two handlers
TestImpl2 impl = new TestImpl2();
server = RPC.getServer(TestProtocol2.class,
impl, ADDRESS, 0, 2, false, conf, null);
- server.addProtocol(TestProtocol0.class, impl);
+ server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/protobuf/TestRpcServiceProtos.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/protobuf/TestRpcServiceProtos.java
index 214b04f6b90..8b192971abc 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/protobuf/TestRpcServiceProtos.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/protobuf/TestRpcServiceProtos.java
@@ -359,6 +359,292 @@ public final class TestRpcServiceProtos {
}
}
+ public static abstract class TestProtobufRpc2Proto
+ implements com.google.protobuf.Service {
+ protected TestProtobufRpc2Proto() {}
+
+ public interface Interface {
+ public abstract void ping2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto request,
+ com.google.protobuf.RpcCallback done);
+
+ public abstract void echo2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto request,
+ com.google.protobuf.RpcCallback done);
+
+ }
+
+ public static com.google.protobuf.Service newReflectiveService(
+ final Interface impl) {
+ return new TestProtobufRpc2Proto() {
+ @java.lang.Override
+ public void ping2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto request,
+ com.google.protobuf.RpcCallback done) {
+ impl.ping2(controller, request, done);
+ }
+
+ @java.lang.Override
+ public void echo2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto request,
+ com.google.protobuf.RpcCallback done) {
+ impl.echo2(controller, request, done);
+ }
+
+ };
+ }
+
+ public static com.google.protobuf.BlockingService
+ newReflectiveBlockingService(final BlockingInterface impl) {
+ return new com.google.protobuf.BlockingService() {
+ public final com.google.protobuf.Descriptors.ServiceDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+
+ public final com.google.protobuf.Message callBlockingMethod(
+ com.google.protobuf.Descriptors.MethodDescriptor method,
+ com.google.protobuf.RpcController controller,
+ com.google.protobuf.Message request)
+ throws com.google.protobuf.ServiceException {
+ if (method.getService() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "Service.callBlockingMethod() given method descriptor for " +
+ "wrong service type.");
+ }
+ switch(method.getIndex()) {
+ case 0:
+ return impl.ping2(controller, (org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto)request);
+ case 1:
+ return impl.echo2(controller, (org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto)request);
+ default:
+ throw new java.lang.AssertionError("Can't get here.");
+ }
+ }
+
+ public final com.google.protobuf.Message
+ getRequestPrototype(
+ com.google.protobuf.Descriptors.MethodDescriptor method) {
+ if (method.getService() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "Service.getRequestPrototype() given method " +
+ "descriptor for wrong service type.");
+ }
+ switch(method.getIndex()) {
+ case 0:
+ return org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto.getDefaultInstance();
+ case 1:
+ return org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto.getDefaultInstance();
+ default:
+ throw new java.lang.AssertionError("Can't get here.");
+ }
+ }
+
+ public final com.google.protobuf.Message
+ getResponsePrototype(
+ com.google.protobuf.Descriptors.MethodDescriptor method) {
+ if (method.getService() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "Service.getResponsePrototype() given method " +
+ "descriptor for wrong service type.");
+ }
+ switch(method.getIndex()) {
+ case 0:
+ return org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto.getDefaultInstance();
+ case 1:
+ return org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto.getDefaultInstance();
+ default:
+ throw new java.lang.AssertionError("Can't get here.");
+ }
+ }
+
+ };
+ }
+
+ public abstract void ping2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto request,
+ com.google.protobuf.RpcCallback done);
+
+ public abstract void echo2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto request,
+ com.google.protobuf.RpcCallback done);
+
+ public static final
+ com.google.protobuf.Descriptors.ServiceDescriptor
+ getDescriptor() {
+ return org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.getDescriptor().getServices().get(1);
+ }
+ public final com.google.protobuf.Descriptors.ServiceDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+
+ public final void callMethod(
+ com.google.protobuf.Descriptors.MethodDescriptor method,
+ com.google.protobuf.RpcController controller,
+ com.google.protobuf.Message request,
+ com.google.protobuf.RpcCallback<
+ com.google.protobuf.Message> done) {
+ if (method.getService() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "Service.callMethod() given method descriptor for wrong " +
+ "service type.");
+ }
+ switch(method.getIndex()) {
+ case 0:
+ this.ping2(controller, (org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto)request,
+ com.google.protobuf.RpcUtil.specializeCallback(
+ done));
+ return;
+ case 1:
+ this.echo2(controller, (org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto)request,
+ com.google.protobuf.RpcUtil.specializeCallback(
+ done));
+ return;
+ default:
+ throw new java.lang.AssertionError("Can't get here.");
+ }
+ }
+
+ public final com.google.protobuf.Message
+ getRequestPrototype(
+ com.google.protobuf.Descriptors.MethodDescriptor method) {
+ if (method.getService() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "Service.getRequestPrototype() given method " +
+ "descriptor for wrong service type.");
+ }
+ switch(method.getIndex()) {
+ case 0:
+ return org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto.getDefaultInstance();
+ case 1:
+ return org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto.getDefaultInstance();
+ default:
+ throw new java.lang.AssertionError("Can't get here.");
+ }
+ }
+
+ public final com.google.protobuf.Message
+ getResponsePrototype(
+ com.google.protobuf.Descriptors.MethodDescriptor method) {
+ if (method.getService() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "Service.getResponsePrototype() given method " +
+ "descriptor for wrong service type.");
+ }
+ switch(method.getIndex()) {
+ case 0:
+ return org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto.getDefaultInstance();
+ case 1:
+ return org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto.getDefaultInstance();
+ default:
+ throw new java.lang.AssertionError("Can't get here.");
+ }
+ }
+
+ public static Stub newStub(
+ com.google.protobuf.RpcChannel channel) {
+ return new Stub(channel);
+ }
+
+ public static final class Stub extends org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpc2Proto implements Interface {
+ private Stub(com.google.protobuf.RpcChannel channel) {
+ this.channel = channel;
+ }
+
+ private final com.google.protobuf.RpcChannel channel;
+
+ public com.google.protobuf.RpcChannel getChannel() {
+ return channel;
+ }
+
+ public void ping2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto request,
+ com.google.protobuf.RpcCallback done) {
+ channel.callMethod(
+ getDescriptor().getMethods().get(0),
+ controller,
+ request,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto.getDefaultInstance(),
+ com.google.protobuf.RpcUtil.generalizeCallback(
+ done,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto.class,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto.getDefaultInstance()));
+ }
+
+ public void echo2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto request,
+ com.google.protobuf.RpcCallback done) {
+ channel.callMethod(
+ getDescriptor().getMethods().get(1),
+ controller,
+ request,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto.getDefaultInstance(),
+ com.google.protobuf.RpcUtil.generalizeCallback(
+ done,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto.class,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto.getDefaultInstance()));
+ }
+ }
+
+ public static BlockingInterface newBlockingStub(
+ com.google.protobuf.BlockingRpcChannel channel) {
+ return new BlockingStub(channel);
+ }
+
+ public interface BlockingInterface {
+ public org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto ping2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto request)
+ throws com.google.protobuf.ServiceException;
+
+ public org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto echo2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto request)
+ throws com.google.protobuf.ServiceException;
+ }
+
+ private static final class BlockingStub implements BlockingInterface {
+ private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+ this.channel = channel;
+ }
+
+ private final com.google.protobuf.BlockingRpcChannel channel;
+
+ public org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto ping2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto request)
+ throws com.google.protobuf.ServiceException {
+ return (org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto) channel.callBlockingMethod(
+ getDescriptor().getMethods().get(0),
+ controller,
+ request,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto.getDefaultInstance());
+ }
+
+
+ public org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto echo2(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto request)
+ throws com.google.protobuf.ServiceException {
+ return (org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto) channel.callBlockingMethod(
+ getDescriptor().getMethods().get(1),
+ controller,
+ request,
+ org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto.getDefaultInstance());
+ }
+
+ }
+ }
+
public static com.google.protobuf.Descriptors.FileDescriptor
getDescriptor() {
@@ -373,8 +659,11 @@ public final class TestRpcServiceProtos {
"uestProto\032\023.EmptyResponseProto\022-\n\004echo\022\021" +
".EchoRequestProto\032\022.EchoResponseProto\0220\n" +
"\005error\022\022.EmptyRequestProto\032\023.EmptyRespon" +
- "seProtoB<\n\036org.apache.hadoop.ipc.protobu" +
- "fB\024TestRpcServiceProtos\210\001\001\240\001\001"
+ "seProto2y\n\025TestProtobufRpc2Proto\0220\n\005ping" +
+ "2\022\022.EmptyRequestProto\032\023.EmptyResponsePro" +
+ "to\022.\n\005echo2\022\021.EchoRequestProto\032\022.EchoRes" +
+ "ponseProtoB<\n\036org.apache.hadoop.ipc.prot" +
+ "obufB\024TestRpcServiceProtos\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
index 14ba0ae170d..3dcffe9d090 100644
--- a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
+++ b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
@@ -31,3 +31,8 @@ service TestProtobufRpcProto {
rpc echo(EchoRequestProto) returns (EchoResponseProto);
rpc error(EmptyRequestProto) returns (EmptyResponseProto);
}
+
+service TestProtobufRpc2Proto {
+ rpc ping2(EmptyRequestProto) returns (EmptyResponseProto);
+ rpc echo2(EchoRequestProto) returns (EchoResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index d78b79c12bd..7bafb1f44d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -68,6 +68,10 @@ Trunk (unreleased changes)
HDFS-2410. Further cleanup of hardcoded configuration keys and values.
(suresh)
+ HADOOP-7862 Hdfs changes to work with HADOOP 7862:
+ Move the support for multiple protocols to lower layer so that Writable,
+ PB and Avro can all use it (Sanjay)
+
OPTIMIZATIONS
HDFS-2477. Optimize computing the diff between a block report and the
namenode state. (Tomasz Nykiel via hairong)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index b2c974c28b1..d86ad92f36e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -142,6 +142,7 @@ import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.net.DNS;
@@ -394,7 +395,7 @@ public class DataNode extends Configured
InterDatanodeProtocolServerSideTranslatorR23
interDatanodeProtocolServerTranslator =
new InterDatanodeProtocolServerSideTranslatorR23(this);
- ipcServer.addProtocol(InterDatanodeWireProtocol.class,
+ ipcServer.addProtocol(RpcKind.RPC_WRITABLE, InterDatanodeWireProtocol.class,
interDatanodeProtocolServerTranslator);
// set service-level authorization security policy
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
index c26f84a1ed5..a833f7fc53d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
/**
@@ -211,7 +212,7 @@ public class BackupNode extends NameNode {
super(conf, nn);
JournalProtocolServerSideTranslatorR23 journalProtocolTranslator =
new JournalProtocolServerSideTranslatorR23(this);
- this.clientRpcServer.addProtocol(JournalWireProtocol.class,
+ this.clientRpcServer.addProtocol(RpcKind.RPC_WRITABLE, JournalWireProtocol.class,
journalProtocolTranslator);
nnRpcAddress = nn.nnRpcAddress;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index d6ba4175c12..f005f8b9d4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -90,6 +90,7 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Groups;
@@ -152,13 +153,16 @@ class NameNodeRpcServer implements NamenodeProtocols {
dnSocketAddr.getHostName(), dnSocketAddr.getPort(),
serviceHandlerCount,
false, conf, namesystem.getDelegationTokenSecretManager());
- this.serviceRpcServer.addProtocol(DatanodeProtocol.class, this);
- this.serviceRpcServer.addProtocol(NamenodeProtocol.class, this);
- this.serviceRpcServer.addProtocol(
+ this.serviceRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
+ DatanodeProtocol.class, this);
+ this.serviceRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
+ NamenodeProtocol.class, this);
+ this.serviceRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
RefreshAuthorizationPolicyProtocol.class, this);
- this.serviceRpcServer.addProtocol(
+ this.serviceRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
RefreshUserMappingsProtocol.class, this);
- this.serviceRpcServer.addProtocol(GetUserMappingsProtocol.class, this);
+ this.serviceRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
+ GetUserMappingsProtocol.class, this);
this.serviceRpcServer.addProtocol(HAServiceProtocol.class, this);
this.serviceRPCAddress = this.serviceRpcServer.getListenerAddress();
@@ -174,12 +178,16 @@ class NameNodeRpcServer implements NamenodeProtocols {
clientProtocolServerTranslator, socAddr.getHostName(),
socAddr.getPort(), handlerCount, false, conf,
namesystem.getDelegationTokenSecretManager());
- this.clientRpcServer.addProtocol(DatanodeProtocol.class, this);
- this.clientRpcServer.addProtocol(NamenodeProtocol.class, this);
- this.clientRpcServer.addProtocol(
+ this.clientRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
+ DatanodeProtocol.class, this);
+ this.clientRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
+ NamenodeProtocol.class, this);
+ this.clientRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
RefreshAuthorizationPolicyProtocol.class, this);
- this.clientRpcServer.addProtocol(RefreshUserMappingsProtocol.class, this);
- this.clientRpcServer.addProtocol(GetUserMappingsProtocol.class, this);
+ this.clientRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
+ RefreshUserMappingsProtocol.class, this);
+ this.clientRpcServer.addProtocol(RpcKind.RPC_WRITABLE,
+ GetUserMappingsProtocol.class, this);
this.clientRpcServer.addProtocol(HAServiceProtocol.class, this);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index af12badddb4..be57bd4e399 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.mockito.internal.stubbing.answers.ThrowsException;
import org.mockito.invocation.InvocationOnMock;
@@ -97,7 +98,7 @@ public class TestDFSClientRetries extends TestCase {
}
@Override
- public Writable call(String protocol, Writable param, long receiveTime)
+ public Writable call(RpcKind rpcKind, String protocol, Writable param, long receiveTime)
throws IOException {
if (sleep) {
// sleep a bit
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
index d50376aaff2..5f48651c890 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
@@ -28,6 +28,7 @@ import java.net.SocketTimeoutException;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils;
@@ -81,7 +82,7 @@ public class TestInterDatanodeProtocol {
}
@Override
- public Writable call(String protocol, Writable param, long receiveTime)
+ public Writable call(RpcKind rpcKind, String protocol, Writable param, long receiveTime)
throws IOException {
if (sleep) {
// sleep a bit
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 70a2001f38b..fbdcba77da4 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -35,11 +35,15 @@ Trunk (unreleased changes)
uri with no authority. (John George via jitendra)
MAPREDUCE-3169. Create a new MiniMRCluster equivalent which only provides
- client APIs cross MR1 and MR2. (Ahmed via tucu)
+ client APIs cross MR1 and MR2 (Ahmed via tucu)
MAPREDUCE-3415. improve MiniMRYarnCluster & DistributedShell JAR resolution.
(tucu)
+ HADOOP-7862 MR changes to work with HADOOP 7862:
+ Move the support for multiple protocols to lower layer so that Writable,
+ PB and Avro can all use it (Sanjay)
+
BUG FIXES
MAPREDUCE-3412. Fix 'ant docs'. (amarrk)
@@ -68,6 +72,8 @@ Trunk (unreleased changes)
MAPREDUCE-3477. Hadoop site documentation cannot be built anymore. (jeagles via tucu)
+ MAPREDUCE-3500. MRJobConfig creates an LD_LIBRARY_PATH using the platform ARCH. (tucu)
+
Release 0.23.1 - Unreleased
INCOMPATIBLE CHANGES
@@ -129,6 +135,12 @@ Release 0.23.1 - Unreleased
MAPREDUCE-3448. TestCombineOutputCollector javac unchecked warning on mocked
generics (Jonathan Eagles via mahadev)
+ MAPREDUCE-3169 amendment. Deprecate MiniMRCluster. (Ahmed Radwan via
+ sseth)
+
+ MAPREDUCE-3369. Migrate MR1 tests to run on MR2 using the new interfaces
+ introduced in MAPREDUCE-3169. (Ahmed Radwan via tomwhite)
+
OPTIMIZATIONS
BUG FIXES
@@ -211,6 +223,26 @@ Release 0.23.1 - Unreleased
MAPREDUCE-3452. fifoscheduler web ui page always shows 0% used for the queue.
(Jonathan Eagles via mahadev)
+ MAPREDUCE-3443. JobClient and Job should function in the context of the
+ UGI which created them. (Mahadev Konar via sseth)
+
+ MAPREDUCE-3460. MR AM can hang if containers are allocated on a node
+ blacklisted by the AM. (Hitesh Shah and Robert Joseph Evans via sseth)
+
+ MAPREDUCE-3453. RM web ui application details page shows RM cluster about
+ information. (Jonathan Eagles via sseth)
+
+ MAPREDUCE-3479. JobClient#getJob cannot find local jobs. (tomwhite)
+
+ MAPREDUCE-3456. $HADOOP_PREFIX/bin/yarn should set defaults for
+ $HADOOP_*_HOME (Eric Payne via mahadev)
+
+ MAPREDUCE-3458. Fix findbugs warnings in hadoop-examples. (Devaraj K
+ via mahadev)
+
+ MAPREDUCE-3485. DISKS_FAILED -101 error code should be defined in same location as
+ ABORTED_CONTAINER_EXIT_STATUS. (Ravi Gummadi via mahadev)
+
Release 0.23.0 - 2011-11-01
INCOMPATIBLE CHANGES
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index e8588e5cd0e..81a5a75b503 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -87,7 +87,7 @@ public class RMContainerAllocator extends RMContainerRequestor
}
/*
- Vocabulory Used:
+ Vocabulary Used:
pending -> requests which are NOT yet sent to RM
scheduled -> requests which are sent to RM but not yet assigned
assigned -> requests which are assigned to a container
@@ -565,6 +565,7 @@ public class RMContainerAllocator extends RMContainerRequestor
if (event.getEarlierAttemptFailed()) {
earlierFailedMaps.add(event.getAttemptID());
request = new ContainerRequest(event, PRIORITY_FAST_FAIL_MAP);
+ LOG.info("Added "+event.getAttemptID()+" to list of failed maps");
} else {
for (String host : event.getHosts()) {
LinkedList list = mapsHostMapping.get(host);
@@ -603,7 +604,9 @@ public class RMContainerAllocator extends RMContainerRequestor
containersAllocated += allocatedContainers.size();
while (it.hasNext()) {
Container allocated = it.next();
- LOG.info("Assigning container " + allocated);
+ LOG.info("Assigning container " + allocated.getId() +
+ " with priority " + allocated.getPriority() +
+ " to NM " + allocated.getNodeId());
// check if allocated container meets memory requirements
// and whether we have any scheduled tasks that need
@@ -645,7 +648,8 @@ public class RMContainerAllocator extends RMContainerRequestor
// we need to request for a new container
// and release the current one
LOG.info("Got allocated container on a blacklisted "
- + " host. Releasing container " + allocated);
+ + " host "+allocated.getNodeId().getHost()
+ +". Releasing container " + allocated);
// find the request matching this allocated container
// and replace it with a new one
@@ -727,10 +731,20 @@ public class RMContainerAllocator extends RMContainerRequestor
}
private ContainerRequest getContainerReqToReplace(Container allocated) {
+ LOG.info("Finding containerReq for allocated container: " + allocated);
Priority priority = allocated.getPriority();
ContainerRequest toBeReplaced = null;
- if (PRIORITY_FAST_FAIL_MAP.equals(priority)
- || PRIORITY_MAP.equals(priority)) {
+ if (PRIORITY_FAST_FAIL_MAP.equals(priority)) {
+ LOG.info("Replacing FAST_FAIL_MAP container " + allocated.getId());
+ Iterator iter = earlierFailedMaps.iterator();
+ while (toBeReplaced == null && iter.hasNext()) {
+ toBeReplaced = maps.get(iter.next());
+ }
+ LOG.info("Found replacement: " + toBeReplaced);
+ return toBeReplaced;
+ }
+ else if (PRIORITY_MAP.equals(priority)) {
+ LOG.info("Replacing MAP container " + allocated.getId());
// allocated container was for a map
String host = allocated.getNodeId().getHost();
LinkedList list = mapsHostMapping.get(host);
@@ -749,6 +763,7 @@ public class RMContainerAllocator extends RMContainerRequestor
TaskAttemptId tId = reduces.keySet().iterator().next();
toBeReplaced = reduces.remove(tId);
}
+ LOG.info("Found replacement: " + toBeReplaced);
return toBeReplaced;
}
@@ -758,7 +773,7 @@ public class RMContainerAllocator extends RMContainerRequestor
//try to assign to earlierFailedMaps if present
ContainerRequest assigned = null;
while (assigned == null && earlierFailedMaps.size() > 0) {
- TaskAttemptId tId = earlierFailedMaps.removeFirst();
+ TaskAttemptId tId = earlierFailedMaps.removeFirst();
if (maps.containsKey(tId)) {
assigned = maps.remove(tId);
JobCounterUpdateEvent jce =
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
index ba3c73219dd..6c03c6690cf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
@@ -105,6 +105,13 @@ public abstract class RMContainerRequestor extends RMCommunicator {
this.priority = priority;
}
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("AttemptId[").append(attemptID).append("]");
+ sb.append("Capability[").append(capability).append("]");
+ sb.append("Priority[").append(priority).append("]");
+ return sb.toString();
+ }
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
index 9dd877b3301..812393c1b58 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
@@ -580,6 +580,135 @@ public class TestRMContainerAllocator {
}
}
+ @Test
+ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception {
+ LOG.info("Running testBlackListedNodesWithSchedulingToThatNode");
+
+ Configuration conf = new Configuration();
+ conf.setBoolean(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, true);
+ conf.setInt(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, 1);
+
+ MyResourceManager rm = new MyResourceManager(conf);
+ rm.start();
+ DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext()
+ .getDispatcher();
+
+ // Submit the application
+ RMApp app = rm.submitApp(1024);
+ dispatcher.await();
+
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
+ amNodeManager.nodeHeartbeat(true);
+ dispatcher.await();
+
+ ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
+ .getAppAttemptId();
+ rm.sendAMLaunched(appAttemptId);
+ dispatcher.await();
+
+ JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0);
+ Job mockJob = mock(Job.class);
+ when(mockJob.getReport()).thenReturn(
+ MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
+ 0, 0, 0, 0, 0, 0, "jobfile", null));
+ MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
+ appAttemptId, mockJob);
+
+ // add resources to scheduler
+ MockNM nodeManager1 = rm.registerNode("h1:1234", 10240);
+ MockNM nodeManager3 = rm.registerNode("h3:1234", 10240);
+ dispatcher.await();
+
+ LOG.info("Requesting 1 Containers _1 on H1");
+ // create the container request
+ ContainerRequestEvent event1 = createReq(jobId, 1, 1024,
+ new String[] { "h1" });
+ allocator.sendRequest(event1);
+
+ LOG.info("RM Heartbeat (to send the container requests)");
+ // this tells the scheduler about the requests
+ // as nodes are not added, no allocations
+ List assigned = allocator.schedule();
+ dispatcher.await();
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
+
+ LOG.info("h1 Heartbeat (To actually schedule the containers)");
+ // update resources in scheduler
+ nodeManager1.nodeHeartbeat(true); // Node heartbeat
+ dispatcher.await();
+
+ LOG.info("RM Heartbeat (To process the scheduled containers)");
+ assigned = allocator.schedule();
+ dispatcher.await();
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
+
+ LOG.info("Failing container _1 on H1 (should blacklist the node)");
+ // Send events to blacklist nodes h1 and h2
+ ContainerFailedEvent f1 = createFailEvent(jobId, 1, "h1", false);
+ allocator.sendFailure(f1);
+
+ //At this stage, a request should be created for a fast fail map
+ //Create a FAST_FAIL request for a previously failed map.
+ ContainerRequestEvent event1f = createReq(jobId, 1, 1024,
+ new String[] { "h1" }, true, false);
+ allocator.sendRequest(event1f);
+
+ //Update the Scheduler with the new requests.
+ assigned = allocator.schedule();
+ dispatcher.await();
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
+
+ // send another request with different resource and priority
+ ContainerRequestEvent event3 = createReq(jobId, 3, 1024,
+ new String[] { "h1", "h3" });
+ allocator.sendRequest(event3);
+
+ //Allocator is aware of prio:5 container, and prio:20 (h1+h3) container.
+ //RM is only aware of the prio:5 container
+
+ LOG.info("h1 Heartbeat (To actually schedule the containers)");
+ // update resources in scheduler
+ nodeManager1.nodeHeartbeat(true); // Node heartbeat
+ dispatcher.await();
+
+ LOG.info("RM Heartbeat (To process the scheduled containers)");
+ assigned = allocator.schedule();
+ dispatcher.await();
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
+
+ //RMContainerAllocator gets assigned a p:5 on a blacklisted node.
+
+ //Send a release for the p:5 container + another request.
+ LOG.info("RM Heartbeat (To process the re-scheduled containers)");
+ assigned = allocator.schedule();
+ dispatcher.await();
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
+
+ //Hearbeat from H3 to schedule on this host.
+ LOG.info("h3 Heartbeat (To re-schedule the containers)");
+ nodeManager3.nodeHeartbeat(true); // Node heartbeat
+ dispatcher.await();
+
+ LOG.info("RM Heartbeat (To process the re-scheduled containers for H3)");
+ assigned = allocator.schedule();
+ dispatcher.await();
+
+ // For debugging
+ for (TaskAttemptContainerAssignedEvent assig : assigned) {
+ LOG.info(assig.getTaskAttemptID() +
+ " assgined to " + assig.getContainer().getId() +
+ " with priority " + assig.getContainer().getPriority());
+ }
+
+ Assert.assertEquals("No of assignments must be 2", 2, assigned.size());
+
+ // validate that all containers are assigned to h3
+ for (TaskAttemptContainerAssignedEvent assig : assigned) {
+ Assert.assertEquals("Assigned container " + assig.getContainer().getId()
+ + " host not correct", "h3", assig.getContainer().getNodeId().getHost());
+ }
+ }
+
private static class MyFifoScheduler extends FifoScheduler {
public MyFifoScheduler(RMContext rmContext) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClientGetJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClientGetJob.java
new file mode 100644
index 00000000000..5ba5f25a63b
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClientGetJob.java
@@ -0,0 +1,62 @@
+/**
+ * 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.mapred;
+
+import static junit.framework.Assert.assertNotNull;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+public class TestJobClientGetJob {
+
+ private static Path TEST_ROOT_DIR =
+ new Path(System.getProperty("test.build.data","/tmp"));
+
+ private Path createTempFile(String filename, String contents)
+ throws IOException {
+ Path path = new Path(TEST_ROOT_DIR, filename);
+ Configuration conf = new Configuration();
+ FSDataOutputStream os = FileSystem.getLocal(conf).create(path);
+ os.writeBytes(contents);
+ os.close();
+ return path;
+ }
+
+ @SuppressWarnings("deprecation")
+ @Test
+ public void testGetRunningJobFromJobClient() throws Exception {
+ JobConf conf = new JobConf();
+ conf.set("mapreduce.framework.name", "local");
+ FileInputFormat.addInputPath(conf, createTempFile("in", "hello"));
+ FileOutputFormat.setOutputPath(conf,
+ new Path(TEST_ROOT_DIR, getClass().getSimpleName()));
+ JobClient jc = new JobClient(conf);
+ RunningJob runningJob = jc.submitJob(conf);
+ assertNotNull("Running job", runningJob);
+ // Check that the running job can be retrieved by ID
+ RunningJob newRunningJob = jc.getJob(runningJob.getID());
+ assertNotNull("New running job", newRunningJob);
+ }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
index 0505e33ce54..1b0b453ddd0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
@@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URL;
+import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.hadoop.mapreduce.util.ConfigUtil;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -421,6 +423,11 @@ public class JobClient extends CLI {
}
Cluster cluster;
+ /**
+ * Ugi of the client. We store this ugi when the client is created and
+ * then make sure that the same ugi is used to run the various protocols.
+ */
+ UserGroupInformation clientUgi;
/**
* Create a job client.
@@ -458,6 +465,7 @@ public class JobClient extends CLI {
public void init(JobConf conf) throws IOException {
setConf(conf);
cluster = new Cluster(conf);
+ clientUgi = UserGroupInformation.getCurrentUser();
}
@InterfaceAudience.Private
@@ -487,8 +495,7 @@ public class JobClient extends CLI {
@Override
public boolean isManaged(Token> token) throws IOException {
return true;
- }
-
+ }
}
/**
@@ -500,6 +507,7 @@ public class JobClient extends CLI {
public JobClient(InetSocketAddress jobTrackAddr,
Configuration conf) throws IOException {
cluster = new Cluster(jobTrackAddr, conf);
+ clientUgi = UserGroupInformation.getCurrentUser();
}
/**
@@ -562,21 +570,38 @@ public class JobClient extends CLI {
* @throws FileNotFoundException
* @throws IOException
*/
- public RunningJob submitJob(JobConf conf) throws FileNotFoundException,
+ public RunningJob submitJob(final JobConf conf) throws FileNotFoundException,
IOException {
try {
conf.setBooleanIfUnset("mapred.mapper.new-api", false);
conf.setBooleanIfUnset("mapred.reducer.new-api", false);
- Job job = Job.getInstance(conf);
- job.submit();
+ Job job = clientUgi.doAs(new PrivilegedExceptionAction () {
+ @Override
+ public Job run() throws IOException, ClassNotFoundException,
+ InterruptedException {
+ Job job = Job.getInstance(conf);
+ job.submit();
+ return job;
+ }
+ });
+ // update our Cluster instance with the one created by Job for submission
+ // (we can't pass our Cluster instance to Job, since Job wraps the config
+ // instance, and the two configs would then diverge)
+ cluster = job.getCluster();
return new NetworkedJob(job);
} catch (InterruptedException ie) {
throw new IOException("interrupted", ie);
- } catch (ClassNotFoundException cnfe) {
- throw new IOException("class not found", cnfe);
}
}
+ private Job getJobUsingCluster(final JobID jobid) throws IOException,
+ InterruptedException {
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ public Job run() throws IOException, InterruptedException {
+ return cluster.getJob(jobid);
+ }
+ });
+ }
/**
* Get an {@link RunningJob} object to track an ongoing job. Returns
* null if the id does not correspond to any known job.
@@ -586,9 +611,10 @@ public class JobClient extends CLI {
* jobid
doesn't correspond to any known job.
* @throws IOException
*/
- public RunningJob getJob(JobID jobid) throws IOException {
+ public RunningJob getJob(final JobID jobid) throws IOException {
try {
- Job job = cluster.getJob(jobid);
+
+ Job job = getJobUsingCluster(jobid);
if (job != null) {
JobStatus status = JobStatus.downgrade(job.getStatus());
if (status != null) {
@@ -621,9 +647,10 @@ public class JobClient extends CLI {
return getTaskReports(jobId, TaskType.MAP);
}
- private TaskReport[] getTaskReports(JobID jobId, TaskType type) throws IOException {
+ private TaskReport[] getTaskReports(final JobID jobId, TaskType type) throws
+ IOException {
try {
- Job j = cluster.getJob(jobId);
+ Job j = getJobUsingCluster(jobId);
if(j == null) {
return EMPTY_TASK_REPORTS;
}
@@ -688,10 +715,11 @@ public class JobClient extends CLI {
* @param state the state of the task
* (pending/running/completed/failed/killed)
*/
- public void displayTasks(JobID jobId, String type, String state)
+ public void displayTasks(final JobID jobId, String type, String state)
throws IOException {
try {
- super.displayTasks(cluster.getJob(jobId), type, state);
+ Job job = getJobUsingCluster(jobId);
+ super.displayTasks(job, type, state);
} catch (InterruptedException ie) {
throw new IOException(ie);
}
@@ -706,15 +734,20 @@ public class JobClient extends CLI {
*/
public ClusterStatus getClusterStatus() throws IOException {
try {
- ClusterMetrics metrics = cluster.getClusterStatus();
- return new ClusterStatus(metrics.getTaskTrackerCount(),
- metrics.getBlackListedTaskTrackerCount(), cluster.getTaskTrackerExpiryInterval(),
- metrics.getOccupiedMapSlots(),
- metrics.getOccupiedReduceSlots(), metrics.getMapSlotCapacity(),
- metrics.getReduceSlotCapacity(),
- cluster.getJobTrackerStatus(),
- metrics.getDecommissionedTaskTrackerCount());
- } catch (InterruptedException ie) {
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ public ClusterStatus run() throws IOException, InterruptedException {
+ ClusterMetrics metrics = cluster.getClusterStatus();
+ return new ClusterStatus(metrics.getTaskTrackerCount(),
+ metrics.getBlackListedTaskTrackerCount(), cluster.getTaskTrackerExpiryInterval(),
+ metrics.getOccupiedMapSlots(),
+ metrics.getOccupiedReduceSlots(), metrics.getMapSlotCapacity(),
+ metrics.getReduceSlotCapacity(),
+ cluster.getJobTrackerStatus(),
+ metrics.getDecommissionedTaskTrackerCount());
+ }
+ });
+ }
+ catch (InterruptedException ie) {
throw new IOException(ie);
}
}
@@ -750,13 +783,17 @@ public class JobClient extends CLI {
*/
public ClusterStatus getClusterStatus(boolean detailed) throws IOException {
try {
- ClusterMetrics metrics = cluster.getClusterStatus();
- return new ClusterStatus(arrayToStringList(cluster.getActiveTaskTrackers()),
- arrayToBlackListInfo(cluster.getBlackListedTaskTrackers()),
- cluster.getTaskTrackerExpiryInterval(), metrics.getOccupiedMapSlots(),
- metrics.getOccupiedReduceSlots(), metrics.getMapSlotCapacity(),
- metrics.getReduceSlotCapacity(),
- cluster.getJobTrackerStatus());
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ public ClusterStatus run() throws IOException, InterruptedException {
+ ClusterMetrics metrics = cluster.getClusterStatus();
+ return new ClusterStatus(arrayToStringList(cluster.getActiveTaskTrackers()),
+ arrayToBlackListInfo(cluster.getBlackListedTaskTrackers()),
+ cluster.getTaskTrackerExpiryInterval(), metrics.getOccupiedMapSlots(),
+ metrics.getOccupiedReduceSlots(), metrics.getMapSlotCapacity(),
+ metrics.getReduceSlotCapacity(),
+ cluster.getJobTrackerStatus());
+ }
+ });
} catch (InterruptedException ie) {
throw new IOException(ie);
}
@@ -787,7 +824,14 @@ public class JobClient extends CLI {
*/
public JobStatus[] getAllJobs() throws IOException {
try {
- org.apache.hadoop.mapreduce.JobStatus[] jobs = cluster.getAllJobStatuses();
+ org.apache.hadoop.mapreduce.JobStatus[] jobs =
+ clientUgi.doAs(new PrivilegedExceptionAction<
+ org.apache.hadoop.mapreduce.JobStatus[]> () {
+ public org.apache.hadoop.mapreduce.JobStatus[] run()
+ throws IOException, InterruptedException {
+ return cluster.getAllJobStatuses();
+ }
+ });
JobStatus[] stats = new JobStatus[jobs.length];
for (int i = 0; i < jobs.length; i++) {
stats[i] = JobStatus.downgrade(jobs[i]);
@@ -909,7 +953,12 @@ public class JobClient extends CLI {
*/
public int getDefaultMaps() throws IOException {
try {
- return cluster.getClusterStatus().getMapSlotCapacity();
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public Integer run() throws IOException, InterruptedException {
+ return cluster.getClusterStatus().getMapSlotCapacity();
+ }
+ });
} catch (InterruptedException ie) {
throw new IOException(ie);
}
@@ -923,7 +972,12 @@ public class JobClient extends CLI {
*/
public int getDefaultReduces() throws IOException {
try {
- return cluster.getClusterStatus().getReduceSlotCapacity();
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public Integer run() throws IOException, InterruptedException {
+ return cluster.getClusterStatus().getReduceSlotCapacity();
+ }
+ });
} catch (InterruptedException ie) {
throw new IOException(ie);
}
@@ -936,8 +990,13 @@ public class JobClient extends CLI {
*/
public Path getSystemDir() {
try {
- return cluster.getSystemDir();
- } catch (IOException ioe) {
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public Path run() throws IOException, InterruptedException {
+ return cluster.getSystemDir();
+ }
+ });
+ } catch (IOException ioe) {
return null;
} catch (InterruptedException ie) {
return null;
@@ -962,7 +1021,11 @@ public class JobClient extends CLI {
*/
public JobQueueInfo[] getRootQueues() throws IOException {
try {
- return getJobQueueInfoArray(cluster.getRootQueues());
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ public JobQueueInfo[] run() throws IOException, InterruptedException {
+ return getJobQueueInfoArray(cluster.getRootQueues());
+ }
+ });
} catch (InterruptedException ie) {
throw new IOException(ie);
}
@@ -976,9 +1039,13 @@ public class JobClient extends CLI {
* @return the array of immediate children JobQueueInfo objects
* @throws IOException
*/
- public JobQueueInfo[] getChildQueues(String queueName) throws IOException {
+ public JobQueueInfo[] getChildQueues(final String queueName) throws IOException {
try {
- return getJobQueueInfoArray(cluster.getChildQueues(queueName));
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ public JobQueueInfo[] run() throws IOException, InterruptedException {
+ return getJobQueueInfoArray(cluster.getChildQueues(queueName));
+ }
+ });
} catch (InterruptedException ie) {
throw new IOException(ie);
}
@@ -993,7 +1060,11 @@ public class JobClient extends CLI {
*/
public JobQueueInfo[] getQueues() throws IOException {
try {
- return getJobQueueInfoArray(cluster.getQueues());
+ return clientUgi.doAs(new PrivilegedExceptionAction() {
+ public JobQueueInfo[] run() throws IOException, InterruptedException {
+ return getJobQueueInfoArray(cluster.getQueues());
+ }
+ });
} catch (InterruptedException ie) {
throw new IOException(ie);
}
@@ -1007,9 +1078,14 @@ public class JobClient extends CLI {
* @throws IOException
*/
- public JobStatus[] getJobsFromQueue(String queueName) throws IOException {
+ public JobStatus[] getJobsFromQueue(final String queueName) throws IOException {
try {
- QueueInfo queue = cluster.getQueue(queueName);
+ QueueInfo queue = clientUgi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public QueueInfo run() throws IOException, InterruptedException {
+ return cluster.getQueue(queueName);
+ }
+ });
if (queue == null) {
return null;
}
@@ -1032,9 +1108,14 @@ public class JobClient extends CLI {
* @return Queue information associated to particular queue.
* @throws IOException
*/
- public JobQueueInfo getQueueInfo(String queueName) throws IOException {
+ public JobQueueInfo getQueueInfo(final String queueName) throws IOException {
try {
- QueueInfo queueInfo = cluster.getQueue(queueName);
+ QueueInfo queueInfo = clientUgi.doAs(new
+ PrivilegedExceptionAction() {
+ public QueueInfo run() throws IOException, InterruptedException {
+ return cluster.getQueue(queueName);
+ }
+ });
if (queueInfo != null) {
return new JobQueueInfo(queueInfo);
}
@@ -1052,7 +1133,14 @@ public class JobClient extends CLI {
public QueueAclsInfo[] getQueueAclsForCurrentUser() throws IOException {
try {
org.apache.hadoop.mapreduce.QueueAclsInfo[] acls =
- cluster.getQueueAclsForCurrentUser();
+ clientUgi.doAs(new
+ PrivilegedExceptionAction
+ () {
+ public org.apache.hadoop.mapreduce.QueueAclsInfo[] run()
+ throws IOException, InterruptedException {
+ return cluster.getQueueAclsForCurrentUser();
+ }
+ });
QueueAclsInfo[] ret = new QueueAclsInfo[acls.length];
for (int i = 0 ; i < acls.length; i++ ) {
ret[i] = QueueAclsInfo.downgrade(acls[i]);
@@ -1070,8 +1158,14 @@ public class JobClient extends CLI {
* @throws IOException
*/
public Token
- getDelegationToken(Text renewer) throws IOException, InterruptedException {
- return cluster.getDelegationToken(renewer);
+ getDelegationToken(final Text renewer) throws IOException, InterruptedException {
+ return clientUgi.doAs(new
+ PrivilegedExceptionAction>() {
+ public Token run() throws IOException,
+ InterruptedException {
+ return cluster.getDelegationToken(renewer);
+ }
+ });
}
/**
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
index 530aee19990..5e92baa8b64 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
@@ -30,6 +30,7 @@ import java.net.URL;
import java.net.URLConnection;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
+import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -315,7 +316,12 @@ public class Job extends JobContextImpl implements JobContext {
* @throws IOException
*/
synchronized void updateStatus() throws IOException, InterruptedException {
- this.status = cluster.getClient().getJobStatus(status.getJobID());
+ this.status = ugi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public JobStatus run() throws IOException, InterruptedException {
+ return cluster.getClient().getJobStatus(status.getJobID());
+ }
+ });
if (this.status == null) {
throw new IOException("Job status not available ");
}
@@ -430,6 +436,11 @@ public class Job extends JobContextImpl implements JobContext {
updateStatus();
return status.isRetired();
}
+
+ @Private
+ public Cluster getCluster() {
+ return cluster;
+ }
/** Only for mocks in unit tests. */
@Private
@@ -476,8 +487,16 @@ public class Job extends JobContextImpl implements JobContext {
InterruptedException {
int failCount = 1;
TaskCompletionEvent lastEvent = null;
- for (TaskCompletionEvent event : cluster.getClient().getTaskCompletionEvents(
- status.getJobID(), 0, 10)) {
+ TaskCompletionEvent[] events = ugi.doAs(new
+ PrivilegedExceptionAction() {
+ @Override
+ public TaskCompletionEvent[] run() throws IOException,
+ InterruptedException {
+ return cluster.getClient().getTaskCompletionEvents(
+ status.getJobID(), 0, 10);
+ }
+ });
+ for (TaskCompletionEvent event : events) {
if (event.getStatus().equals(TaskCompletionEvent.Status.FAILED)) {
failCount++;
lastEvent = event;
@@ -500,7 +519,12 @@ public class Job extends JobContextImpl implements JobContext {
public TaskReport[] getTaskReports(TaskType type)
throws IOException, InterruptedException {
ensureState(JobState.RUNNING);
- return cluster.getClient().getTaskReports(getJobID(), type);
+ final TaskType tmpType = type;
+ return ugi.doAs(new PrivilegedExceptionAction() {
+ public TaskReport[] run() throws IOException, InterruptedException {
+ return cluster.getClient().getTaskReports(getJobID(), tmpType);
+ }
+ });
}
/**
@@ -603,7 +627,14 @@ public class Job extends JobContextImpl implements JobContext {
org.apache.hadoop.mapred.JobPriority.valueOf(priority.name()));
} else {
ensureState(JobState.RUNNING);
- cluster.getClient().setJobPriority(getJobID(), priority.toString());
+ final JobPriority tmpPriority = priority;
+ ugi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public Object run() throws IOException, InterruptedException {
+ cluster.getClient().setJobPriority(getJobID(), tmpPriority.toString());
+ return null;
+ }
+ });
}
}
@@ -615,12 +646,17 @@ public class Job extends JobContextImpl implements JobContext {
* @return an array of {@link TaskCompletionEvent}s
* @throws IOException
*/
- public TaskCompletionEvent[] getTaskCompletionEvents(int startFrom,
- int numEvents) throws IOException, InterruptedException {
+ public TaskCompletionEvent[] getTaskCompletionEvents(final int startFrom,
+ final int numEvents) throws IOException, InterruptedException {
ensureState(JobState.RUNNING);
- return cluster.getClient().getTaskCompletionEvents(getJobID(),
- startFrom, numEvents);
- }
+ return ugi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public TaskCompletionEvent[] run() throws IOException, InterruptedException {
+ return cluster.getClient().getTaskCompletionEvents(getJobID(),
+ startFrom, numEvents);
+ }
+ });
+ }
/**
* Kill indicated task attempt.
@@ -628,10 +664,14 @@ public class Job extends JobContextImpl implements JobContext {
* @param taskId the id of the task to be terminated.
* @throws IOException
*/
- public boolean killTask(TaskAttemptID taskId)
+ public boolean killTask(final TaskAttemptID taskId)
throws IOException, InterruptedException {
ensureState(JobState.RUNNING);
- return cluster.getClient().killTask(taskId, false);
+ return ugi.doAs(new PrivilegedExceptionAction() {
+ public Boolean run() throws IOException, InterruptedException {
+ return cluster.getClient().killTask(taskId, false);
+ }
+ });
}
/**
@@ -640,10 +680,15 @@ public class Job extends JobContextImpl implements JobContext {
* @param taskId the id of the task to be terminated.
* @throws IOException
*/
- public boolean failTask(TaskAttemptID taskId)
+ public boolean failTask(final TaskAttemptID taskId)
throws IOException, InterruptedException {
ensureState(JobState.RUNNING);
- return cluster.getClient().killTask(taskId, true);
+ return ugi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public Boolean run() throws IOException, InterruptedException {
+ return cluster.getClient().killTask(taskId, true);
+ }
+ });
}
/**
@@ -656,7 +701,12 @@ public class Job extends JobContextImpl implements JobContext {
public Counters getCounters()
throws IOException, InterruptedException {
ensureState(JobState.RUNNING);
- return cluster.getClient().getJobCounters(getJobID());
+ return ugi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public Counters run() throws IOException, InterruptedException {
+ return cluster.getClient().getJobCounters(getJobID());
+ }
+ });
}
/**
@@ -665,10 +715,15 @@ public class Job extends JobContextImpl implements JobContext {
* @return the list of diagnostic messages for the task
* @throws IOException
*/
- public String[] getTaskDiagnostics(TaskAttemptID taskid)
+ public String[] getTaskDiagnostics(final TaskAttemptID taskid)
throws IOException, InterruptedException {
ensureState(JobState.RUNNING);
- return cluster.getClient().getTaskDiagnostics(taskid);
+ return ugi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public String[] run() throws IOException, InterruptedException {
+ return cluster.getClient().getTaskDiagnostics(taskid);
+ }
+ });
}
/**
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 769d842c607..dd189e2eafa 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -17,7 +17,6 @@
*/
package org.apache.hadoop.mapreduce;
-import org.apache.hadoop.util.PlatformName;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -439,7 +438,7 @@ public interface MRJobConfig {
"mapreduce.admin.user.env";
public static final String DEFAULT_MAPRED_ADMIN_USER_ENV =
- "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native/" + PlatformName.getPlatformName();
+ "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native";
public static final String WORKDIR = "work";
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
index 99b7e8826e7..71ea84bb8c5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
@@ -22,7 +22,7 @@ import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
-import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
@@ -156,6 +156,8 @@ public class ClientServiceDelegate {
application = rm.getApplicationReport(appId);
continue;
}
+ UserGroupInformation newUgi = UserGroupInformation.createRemoteUser(
+ UserGroupInformation.getCurrentUser().getUserName());
serviceAddr = application.getHost() + ":" + application.getRpcPort();
if (UserGroupInformation.isSecurityEnabled()) {
String clientTokenEncoded = application.getClientToken();
@@ -167,11 +169,17 @@ public class ClientServiceDelegate {
.getHost(), application.getRpcPort());
clientToken.setService(new Text(addr.getAddress().getHostAddress()
+ ":" + addr.getPort()));
- UserGroupInformation.getCurrentUser().addToken(clientToken);
+ newUgi.addToken(clientToken);
}
LOG.info("The url to track the job: " + application.getTrackingUrl());
LOG.debug("Connecting to " + serviceAddr);
- realProxy = instantiateAMProxy(serviceAddr);
+ final String tempStr = serviceAddr;
+ realProxy = newUgi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public MRClientProtocol run() throws IOException {
+ return instantiateAMProxy(tempStr);
+ }
+ });
return realProxy;
} catch (IOException e) {
//possibly the AM has crashed
@@ -243,17 +251,11 @@ public class ClientServiceDelegate {
MRClientProtocol instantiateAMProxy(final String serviceAddr)
throws IOException {
- UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
LOG.trace("Connecting to ApplicationMaster at: " + serviceAddr);
- MRClientProtocol proxy = currentUser
- .doAs(new PrivilegedAction() {
- @Override
- public MRClientProtocol run() {
- YarnRPC rpc = YarnRPC.create(conf);
- return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
+ YarnRPC rpc = YarnRPC.create(conf);
+ MRClientProtocol proxy =
+ (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
NetUtils.createSocketAddr(serviceAddr), conf);
- }
- });
LOG.trace("Connected to ApplicationMaster at: " + serviceAddr);
return proxy;
}
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/conf/TestNoDefaultsJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
similarity index 87%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
index 4daf90ddce0..d91754d71d7 100644
--- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
@@ -59,19 +59,6 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
JobConf conf = new JobConf(false);
- //seeding JT and NN info into non-defaults (empty jobconf)
- String jobTrackerAddress = createJobConf().get(JTConfig.JT_IPC_ADDRESS);
- if (jobTrackerAddress == null) {
- jobTrackerAddress = "local";
- }
- conf.set(JTConfig.JT_IPC_ADDRESS, jobTrackerAddress);
- if (jobTrackerAddress == "local") {
- conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.LOCAL_FRAMEWORK_NAME);
- }
- else {
- conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.CLASSIC_FRAMEWORK_NAME);
- }
-
conf.set("fs.default.name", createJobConf().get("fs.default.name"));
conf.setJobName("mr");
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
new file mode 100644
index 00000000000..c102e8f8626
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
@@ -0,0 +1,214 @@
+/**
+ * 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.mapred;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.MRConfig;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Abstract Test case class to run MR in local or cluster mode and in local FS
+ * or DFS.
+ *
+ * The Hadoop instance is started and stopped on each test method.
+ *
+ * If using DFS the filesystem is reformated at each start (test method).
+ *
+ * Job Configurations should be created using a configuration returned by the
+ * 'createJobConf()' method.
+ */
+public abstract class HadoopTestCase extends TestCase {
+ public static final int LOCAL_MR = 1;
+ public static final int CLUSTER_MR = 2;
+ public static final int LOCAL_FS = 4;
+ public static final int DFS_FS = 8;
+
+ private boolean localMR;
+ private boolean localFS;
+
+ private int taskTrackers;
+ private int dataNodes;
+
+ /**
+ * Creates a testcase for local or cluster MR using DFS.
+ *
+ * The DFS will be formatted regardless if there was one or not before in the
+ * given location.
+ *
+ * @param mrMode indicates if the MR should be local (LOCAL_MR) or cluster
+ * (CLUSTER_MR)
+ * @param fsMode indicates if the FS should be local (LOCAL_FS) or DFS (DFS_FS)
+ *
+ * local FS when using relative PATHs)
+ *
+ * @param taskTrackers number of task trackers to start when using cluster
+ *
+ * @param dataNodes number of data nodes to start when using DFS
+ *
+ * @throws IOException thrown if the base directory cannot be set.
+ */
+ public HadoopTestCase(int mrMode, int fsMode, int taskTrackers, int dataNodes)
+ throws IOException {
+ if (mrMode != LOCAL_MR && mrMode != CLUSTER_MR) {
+ throw new IllegalArgumentException(
+ "Invalid MapRed mode, must be LOCAL_MR or CLUSTER_MR");
+ }
+ if (fsMode != LOCAL_FS && fsMode != DFS_FS) {
+ throw new IllegalArgumentException(
+ "Invalid FileSystem mode, must be LOCAL_FS or DFS_FS");
+ }
+ if (taskTrackers < 1) {
+ throw new IllegalArgumentException(
+ "Invalid taskTrackers value, must be greater than 0");
+ }
+ if (dataNodes < 1) {
+ throw new IllegalArgumentException(
+ "Invalid dataNodes value, must be greater than 0");
+ }
+ localMR = (mrMode == LOCAL_MR);
+ localFS = (fsMode == LOCAL_FS);
+ /*
+ JobConf conf = new JobConf();
+ fsRoot = conf.get("hadoop.tmp.dir");
+
+ if (fsRoot == null) {
+ throw new IllegalArgumentException(
+ "hadoop.tmp.dir is not defined");
+ }
+
+ fsRoot = fsRoot.replace(' ', '+') + "/fs";
+
+ File file = new File(fsRoot);
+ if (!file.exists()) {
+ if (!file.mkdirs()) {
+ throw new RuntimeException("Could not create FS base path: " + file);
+ }
+ }
+ */
+ this.taskTrackers = taskTrackers;
+ this.dataNodes = dataNodes;
+ }
+
+ /**
+ * Indicates if the MR is running in local or cluster mode.
+ *
+ * @return returns TRUE if the MR is running locally, FALSE if running in
+ * cluster mode.
+ */
+ public boolean isLocalMR() {
+ return localMR;
+ }
+
+ /**
+ * Indicates if the filesystem is local or DFS.
+ *
+ * @return returns TRUE if the filesystem is local, FALSE if it is DFS.
+ */
+ public boolean isLocalFS() {
+ return localFS;
+ }
+
+
+ private MiniDFSCluster dfsCluster = null;
+ private MiniMRCluster mrCluster = null;
+ private FileSystem fileSystem = null;
+
+ /**
+ * Creates Hadoop instance based on constructor configuration before
+ * a test case is run.
+ *
+ * @throws Exception
+ */
+ protected void setUp() throws Exception {
+ super.setUp();
+ if (localFS) {
+ fileSystem = FileSystem.getLocal(new JobConf());
+ }
+ else {
+ dfsCluster = new MiniDFSCluster(new JobConf(), dataNodes, true, null);
+ fileSystem = dfsCluster.getFileSystem();
+ }
+ if (localMR) {
+ }
+ else {
+ //noinspection deprecation
+ mrCluster = new MiniMRCluster(taskTrackers, fileSystem.getUri().toString(), 1);
+ }
+ }
+
+ /**
+ * Destroys Hadoop instance based on constructor configuration after
+ * a test case is run.
+ *
+ * @throws Exception
+ */
+ protected void tearDown() throws Exception {
+ try {
+ if (mrCluster != null) {
+ mrCluster.shutdown();
+ }
+ }
+ catch (Exception ex) {
+ System.out.println(ex);
+ }
+ try {
+ if (dfsCluster != null) {
+ dfsCluster.shutdown();
+ }
+ }
+ catch (Exception ex) {
+ System.out.println(ex);
+ }
+ super.tearDown();
+ }
+
+ /**
+ * Returns the Filesystem in use.
+ *
+ * TestCases should use this Filesystem as it
+ * is properly configured with the workingDir for relative PATHs.
+ *
+ * @return the filesystem used by Hadoop.
+ */
+ protected FileSystem getFileSystem() {
+ return fileSystem;
+ }
+
+ /**
+ * Returns a job configuration preconfigured to run against the Hadoop
+ * managed by the testcase.
+ * @return configuration that works on the testcase Hadoop instance
+ */
+ protected JobConf createJobConf() {
+ if (localMR) {
+ JobConf conf = new JobConf();
+ conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.LOCAL_FRAMEWORK_NAME);
+ return conf;
+ }
+ else {
+ return mrCluster.createJobConf();
+ }
+ }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java
index a73b959ec3b..627bae3c1ee 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRCluster.java
@@ -34,7 +34,11 @@ import org.apache.hadoop.security.UserGroupInformation;
* Due to major differences between MR1 and MR2, a number of methods are either
* unimplemented/unsupported or were re-implemented to provide wrappers around
* MR2 functionality.
+ *
+ * @deprecated Use {@link org.apache.hadoop.mapred.MiniMRClientClusterFactory}
+ * instead
*/
+@Deprecated
public class MiniMRCluster {
private static final Log LOG = LogFactory.getLog(MiniMRCluster.class);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
new file mode 100644
index 00000000000..026edfbddb0
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -0,0 +1,224 @@
+/**
+ * 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.mapred;
+
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.ServletHolder;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.ServletException;
+import java.io.IOException;
+import java.io.DataOutputStream;
+
+/**
+ * Base class to test Job end notification in local and cluster mode.
+ *
+ * Starts up hadoop on Local or Cluster mode (by extending of the
+ * HadoopTestCase class) and it starts a servlet engine that hosts
+ * a servlet that will receive the notification of job finalization.
+ *
+ * The notification servlet returns a HTTP 400 the first time is called
+ * and a HTTP 200 the second time, thus testing retry.
+ *
+ * In both cases local file system is used (this is irrelevant for
+ * the tested functionality)
+ *
+ *
+ */
+public abstract class NotificationTestCase extends HadoopTestCase {
+
+ protected NotificationTestCase(int mode) throws IOException {
+ super(mode, HadoopTestCase.LOCAL_FS, 1, 1);
+ }
+
+ private int port;
+ private String contextPath = "/notification";
+ private String servletPath = "/mapred";
+ private Server webServer;
+
+ private void startHttpServer() throws Exception {
+
+ // Create the webServer
+ if (webServer != null) {
+ webServer.stop();
+ webServer = null;
+ }
+ webServer = new Server(0);
+
+ Context context = new Context(webServer, contextPath);
+
+ // create servlet handler
+ context.addServlet(new ServletHolder(new NotificationServlet()),
+ servletPath);
+
+ // Start webServer
+ webServer.start();
+ port = webServer.getConnectors()[0].getLocalPort();
+
+ }
+
+ private void stopHttpServer() throws Exception {
+ if (webServer != null) {
+ webServer.stop();
+ webServer.destroy();
+ webServer = null;
+ }
+ }
+
+ public static class NotificationServlet extends HttpServlet {
+ public static int counter = 0;
+ private static final long serialVersionUID = 1L;
+
+ protected void doGet(HttpServletRequest req, HttpServletResponse res)
+ throws ServletException, IOException {
+ switch (counter) {
+ case 0:
+ {
+ assertTrue(req.getQueryString().contains("SUCCEEDED"));
+ }
+ break;
+ case 2:
+ {
+ assertTrue(req.getQueryString().contains("KILLED"));
+ }
+ break;
+ case 4:
+ {
+ assertTrue(req.getQueryString().contains("FAILED"));
+ }
+ break;
+ }
+ if (counter % 2 == 0) {
+ res.sendError(HttpServletResponse.SC_BAD_REQUEST, "forcing error");
+ }
+ else {
+ res.setStatus(HttpServletResponse.SC_OK);
+ }
+ counter++;
+ }
+ }
+
+ private String getNotificationUrlTemplate() {
+ return "http://localhost:" + port + contextPath + servletPath +
+ "?jobId=$jobId&jobStatus=$jobStatus";
+ }
+
+ protected JobConf createJobConf() {
+ JobConf conf = super.createJobConf();
+ conf.setJobEndNotificationURI(getNotificationUrlTemplate());
+ conf.setInt(JobContext.MR_JOB_END_RETRY_ATTEMPTS, 3);
+ conf.setInt(JobContext.MR_JOB_END_RETRY_INTERVAL, 200);
+ return conf;
+ }
+
+
+ protected void setUp() throws Exception {
+ super.setUp();
+ startHttpServer();
+ }
+
+ protected void tearDown() throws Exception {
+ stopHttpServer();
+ super.tearDown();
+ }
+
+ public void testMR() throws Exception {
+ System.out.println(launchWordCount(this.createJobConf(),
+ "a b c d e f g h", 1, 1));
+ Thread.sleep(2000);
+ assertEquals(2, NotificationServlet.counter);
+
+ Path inDir = new Path("notificationjob/input");
+ Path outDir = new Path("notificationjob/output");
+
+ // Hack for local FS that does not have the concept of a 'mounting point'
+ if (isLocalFS()) {
+ String localPathRoot = System.getProperty("test.build.data","/tmp")
+ .toString().replace(' ', '+');;
+ inDir = new Path(localPathRoot, inDir);
+ outDir = new Path(localPathRoot, outDir);
+ }
+
+ // run a job with KILLED status
+ System.out.println(UtilsForTests.runJobKill(this.createJobConf(), inDir,
+ outDir).getID());
+ Thread.sleep(2000);
+ assertEquals(4, NotificationServlet.counter);
+
+ // run a job with FAILED status
+ System.out.println(UtilsForTests.runJobFail(this.createJobConf(), inDir,
+ outDir).getID());
+ Thread.sleep(2000);
+ assertEquals(6, NotificationServlet.counter);
+ }
+
+ private String launchWordCount(JobConf conf,
+ String input,
+ int numMaps,
+ int numReduces) throws IOException {
+ Path inDir = new Path("testing/wc/input");
+ Path outDir = new Path("testing/wc/output");
+
+ // Hack for local FS that does not have the concept of a 'mounting point'
+ if (isLocalFS()) {
+ String localPathRoot = System.getProperty("test.build.data","/tmp")
+ .toString().replace(' ', '+');;
+ inDir = new Path(localPathRoot, inDir);
+ outDir = new Path(localPathRoot, outDir);
+ }
+
+ FileSystem fs = FileSystem.get(conf);
+ fs.delete(outDir, true);
+ if (!fs.mkdirs(inDir)) {
+ throw new IOException("Mkdirs failed to create " + inDir.toString());
+ }
+ {
+ DataOutputStream file = fs.create(new Path(inDir, "part-0"));
+ file.writeBytes(input);
+ file.close();
+ }
+ conf.setJobName("wordcount");
+ conf.setInputFormat(TextInputFormat.class);
+
+ // the keys are words (strings)
+ conf.setOutputKeyClass(Text.class);
+ // the values are counts (ints)
+ conf.setOutputValueClass(IntWritable.class);
+
+ conf.setMapperClass(WordCount.MapClass.class);
+ conf.setCombinerClass(WordCount.Reduce.class);
+ conf.setReducerClass(WordCount.Reduce.class);
+
+ FileInputFormat.setInputPaths(conf, inDir);
+ FileOutputFormat.setOutputPath(conf, outDir);
+ conf.setNumMapTasks(numMaps);
+ conf.setNumReduceTasks(numReduces);
+ JobClient.runJob(conf);
+ return MapReduceTestUtil.readOutput(outDir, conf);
+ }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SortValidator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SortValidator.java
new file mode 100644
index 00000000000..381d42e238d
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SortValidator.java
@@ -0,0 +1,584 @@
+/**
+ * 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.mapred;
+
+import java.io.*;
+import java.net.URI;
+import java.util.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapred.lib.HashPartitioner;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.fs.*;
+
+/**
+ * A set of utilities to validate the sort of the map-reduce framework.
+ * This utility program has 2 main parts:
+ * 1. Checking the records' statistics
+ * a) Validates the no. of bytes and records in sort's input & output.
+ * b) Validates the xor of the md5's of each key/value pair.
+ * c) Ensures same key/value is present in both input and output.
+ * 2. Check individual records to ensure each record is present in both
+ * the input and the output of the sort (expensive on large data-sets).
+ *
+ * To run: bin/hadoop jar build/hadoop-examples.jar sortvalidate
+ * [-m maps ] [-r reduces ] [-deep]
+ * -sortInput sort-in-dir -sortOutput sort-out-dir
+ */
+public class SortValidator extends Configured implements Tool {
+
+ static private final IntWritable sortInput = new IntWritable(1);
+ static private final IntWritable sortOutput = new IntWritable(2);
+ static public String SORT_REDUCES =
+ "mapreduce.sortvalidator.sort.reduce.tasks";
+ static public String MAPS_PER_HOST = "mapreduce.sortvalidator.mapsperhost";
+ static public String REDUCES_PER_HOST =
+ "mapreduce.sortvalidator.reducesperhost";
+ static void printUsage() {
+ System.err.println("sortvalidate [-m ] [-r ] [-deep] " +
+ "-sortInput -sortOutput ");
+ System.exit(1);
+ }
+
+ static private IntWritable deduceInputFile(JobConf job) {
+ Path[] inputPaths = FileInputFormat.getInputPaths(job);
+ Path inputFile = new Path(job.get(JobContext.MAP_INPUT_FILE));
+
+ // value == one for sort-input; value == two for sort-output
+ return (inputFile.getParent().equals(inputPaths[0])) ?
+ sortInput : sortOutput;
+ }
+
+ static private byte[] pair(BytesWritable a, BytesWritable b) {
+ byte[] pairData = new byte[a.getLength()+ b.getLength()];
+ System.arraycopy(a.getBytes(), 0, pairData, 0, a.getLength());
+ System.arraycopy(b.getBytes(), 0, pairData, a.getLength(), b.getLength());
+ return pairData;
+ }
+
+ private static final PathFilter sortPathsFilter = new PathFilter() {
+ public boolean accept(Path path) {
+ return (path.getName().startsWith("part-"));
+ }
+ };
+
+ /**
+ * A simple map-reduce job which checks consistency of the
+ * MapReduce framework's sort by checking:
+ * a) Records are sorted correctly
+ * b) Keys are partitioned correctly
+ * c) The input and output have same no. of bytes and records.
+ * d) The input and output have the correct 'checksum' by xor'ing
+ * the md5 of each record.
+ *
+ */
+ public static class RecordStatsChecker {
+
+ /**
+ * Generic way to get raw data from a {@link Writable}.
+ */
+ static class Raw {
+ /**
+ * Get raw data bytes from a {@link Writable}
+ * @param writable {@link Writable} object from whom to get the raw data
+ * @return raw data of the writable
+ */
+ public byte[] getRawBytes(Writable writable) {
+ return writable.toString().getBytes();
+ }
+
+ /**
+ * Get number of raw data bytes of the {@link Writable}
+ * @param writable {@link Writable} object from whom to get the raw data
+ * length
+ * @return number of raw data bytes
+ */
+ public int getRawBytesLength(Writable writable) {
+ return writable.toString().getBytes().length;
+ }
+ }
+
+ /**
+ * Specialization of {@link Raw} for {@link BytesWritable}.
+ */
+ static class RawBytesWritable extends Raw {
+ public byte[] getRawBytes(Writable bw) {
+ return ((BytesWritable)bw).getBytes();
+ }
+ public int getRawBytesLength(Writable bw) {
+ return ((BytesWritable)bw).getLength();
+ }
+ }
+
+ /**
+ * Specialization of {@link Raw} for {@link Text}.
+ */
+ static class RawText extends Raw {
+ public byte[] getRawBytes(Writable text) {
+ return ((Text)text).getBytes();
+ }
+ public int getRawBytesLength(Writable text) {
+ return ((Text)text).getLength();
+ }
+ }
+
+ private static Raw createRaw(Class rawClass) {
+ if (rawClass == Text.class) {
+ return new RawText();
+ } else if (rawClass == BytesWritable.class) {
+ System.err.println("Returning " + RawBytesWritable.class);
+ return new RawBytesWritable();
+ }
+ return new Raw();
+ }
+
+ public static class RecordStatsWritable implements Writable {
+ private long bytes = 0;
+ private long records = 0;
+ private int checksum = 0;
+
+ public RecordStatsWritable() {}
+
+ public RecordStatsWritable(long bytes, long records, int checksum) {
+ this.bytes = bytes;
+ this.records = records;
+ this.checksum = checksum;
+ }
+
+ public void write(DataOutput out) throws IOException {
+ WritableUtils.writeVLong(out, bytes);
+ WritableUtils.writeVLong(out, records);
+ WritableUtils.writeVInt(out, checksum);
+ }
+
+ public void readFields(DataInput in) throws IOException {
+ bytes = WritableUtils.readVLong(in);
+ records = WritableUtils.readVLong(in);
+ checksum = WritableUtils.readVInt(in);
+ }
+
+ public long getBytes() { return bytes; }
+ public long getRecords() { return records; }
+ public int getChecksum() { return checksum; }
+ }
+
+ public static class Map extends MapReduceBase
+ implements Mapper {
+
+ private IntWritable key = null;
+ private WritableComparable prevKey = null;
+ private Class extends WritableComparable> keyClass;
+ private Partitioner partitioner = null;
+ private int partition = -1;
+ private int noSortReducers = -1;
+ private long recordId = -1;
+
+ private Raw rawKey;
+ private Raw rawValue;
+
+ public void configure(JobConf job) {
+ // 'key' == sortInput for sort-input; key == sortOutput for sort-output
+ key = deduceInputFile(job);
+
+ if (key == sortOutput) {
+ partitioner = new HashPartitioner();
+
+ // Figure the 'current' partition and no. of reduces of the 'sort'
+ try {
+ URI inputURI = new URI(job.get(JobContext.MAP_INPUT_FILE));
+ String inputFile = inputURI.getPath();
+ // part file is of the form part-r-xxxxx
+ partition = Integer.valueOf(inputFile.substring(
+ inputFile.lastIndexOf("part") + 7)).intValue();
+ noSortReducers = job.getInt(SORT_REDUCES, -1);
+ } catch (Exception e) {
+ System.err.println("Caught: " + e);
+ System.exit(-1);
+ }
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public void map(WritableComparable key, Writable value,
+ OutputCollector output,
+ Reporter reporter) throws IOException {
+ // Set up rawKey and rawValue on the first call to 'map'
+ if (recordId == -1) {
+ rawKey = createRaw(key.getClass());
+ rawValue = createRaw(value.getClass());
+ }
+ ++recordId;
+
+ if (this.key == sortOutput) {
+ // Check if keys are 'sorted' if this
+ // record is from sort's output
+ if (prevKey == null) {
+ prevKey = key;
+ keyClass = prevKey.getClass();
+ } else {
+ // Sanity check
+ if (keyClass != key.getClass()) {
+ throw new IOException("Type mismatch in key: expected " +
+ keyClass.getName() + ", received " +
+ key.getClass().getName());
+ }
+
+ // Check if they were sorted correctly
+ if (prevKey.compareTo(key) > 0) {
+ throw new IOException("The 'map-reduce' framework wrongly" +
+ " classifed (" + prevKey + ") > (" +
+ key + ") "+ "for record# " + recordId);
+ }
+ prevKey = key;
+ }
+
+ // Check if the sorted output is 'partitioned' right
+ int keyPartition =
+ partitioner.getPartition(key, value, noSortReducers);
+ if (partition != keyPartition) {
+ throw new IOException("Partitions do not match for record# " +
+ recordId + " ! - '" + partition + "' v/s '" +
+ keyPartition + "'");
+ }
+ }
+
+ // Construct the record-stats and output (this.key, record-stats)
+ byte[] keyBytes = rawKey.getRawBytes(key);
+ int keyBytesLen = rawKey.getRawBytesLength(key);
+ byte[] valueBytes = rawValue.getRawBytes(value);
+ int valueBytesLen = rawValue.getRawBytesLength(value);
+
+ int keyValueChecksum =
+ (WritableComparator.hashBytes(keyBytes, keyBytesLen) ^
+ WritableComparator.hashBytes(valueBytes, valueBytesLen));
+
+ output.collect(this.key,
+ new RecordStatsWritable((keyBytesLen+valueBytesLen),
+ 1, keyValueChecksum)
+ );
+ }
+
+ }
+
+ public static class Reduce extends MapReduceBase
+ implements Reducer {
+
+ public void reduce(IntWritable key, Iterator values,
+ OutputCollector output,
+ Reporter reporter) throws IOException {
+ long bytes = 0;
+ long records = 0;
+ int xor = 0;
+ while (values.hasNext()) {
+ RecordStatsWritable stats = values.next();
+ bytes += stats.getBytes();
+ records += stats.getRecords();
+ xor ^= stats.getChecksum();
+ }
+
+ output.collect(key, new RecordStatsWritable(bytes, records, xor));
+ }
+ }
+
+ public static class NonSplitableSequenceFileInputFormat
+ extends SequenceFileInputFormat {
+ protected boolean isSplitable(FileSystem fs, Path filename) {
+ return false;
+ }
+ }
+
+ static void checkRecords(Configuration defaults,
+ Path sortInput, Path sortOutput) throws IOException {
+ FileSystem inputfs = sortInput.getFileSystem(defaults);
+ FileSystem outputfs = sortOutput.getFileSystem(defaults);
+ FileSystem defaultfs = FileSystem.get(defaults);
+ JobConf jobConf = new JobConf(defaults, RecordStatsChecker.class);
+ jobConf.setJobName("sortvalidate-recordstats-checker");
+
+ int noSortReduceTasks =
+ outputfs.listStatus(sortOutput, sortPathsFilter).length;
+ jobConf.setInt(SORT_REDUCES, noSortReduceTasks);
+ int noSortInputpaths = inputfs.listStatus(sortInput).length;
+
+ jobConf.setInputFormat(NonSplitableSequenceFileInputFormat.class);
+ jobConf.setOutputFormat(SequenceFileOutputFormat.class);
+
+ jobConf.setOutputKeyClass(IntWritable.class);
+ jobConf.setOutputValueClass(RecordStatsChecker.RecordStatsWritable.class);
+
+ jobConf.setMapperClass(Map.class);
+ jobConf.setCombinerClass(Reduce.class);
+ jobConf.setReducerClass(Reduce.class);
+
+ jobConf.setNumMapTasks(noSortReduceTasks);
+ jobConf.setNumReduceTasks(1);
+
+ FileInputFormat.setInputPaths(jobConf, sortInput);
+ FileInputFormat.addInputPath(jobConf, sortOutput);
+ Path outputPath = new Path("/tmp/sortvalidate/recordstatschecker");
+ if (defaultfs.exists(outputPath)) {
+ defaultfs.delete(outputPath, true);
+ }
+ FileOutputFormat.setOutputPath(jobConf, outputPath);
+
+ // Uncomment to run locally in a single process
+ //job_conf.set(JTConfig.JT, "local");
+ Path[] inputPaths = FileInputFormat.getInputPaths(jobConf);
+ System.out.println("\nSortValidator.RecordStatsChecker: Validate sort " +
+ "from " + inputPaths[0] + " (" +
+ noSortInputpaths + " files), " +
+ inputPaths[1] + " (" +
+ noSortReduceTasks +
+ " files) into " +
+ FileOutputFormat.getOutputPath(jobConf) +
+ " with 1 reducer.");
+ Date startTime = new Date();
+ System.out.println("Job started: " + startTime);
+ JobClient.runJob(jobConf);
+ Date end_time = new Date();
+ System.out.println("Job ended: " + end_time);
+ System.out.println("The job took " +
+ (end_time.getTime() - startTime.getTime()) /1000 + " seconds.");
+
+ // Check to ensure that the statistics of the
+ // framework's sort-input and sort-output match
+ SequenceFile.Reader stats = new SequenceFile.Reader(defaultfs,
+ new Path(outputPath, "part-00000"), defaults);
+ IntWritable k1 = new IntWritable();
+ IntWritable k2 = new IntWritable();
+ RecordStatsWritable v1 = new RecordStatsWritable();
+ RecordStatsWritable v2 = new RecordStatsWritable();
+ if (!stats.next(k1, v1)) {
+ throw new IOException("Failed to read record #1 from reduce's output");
+ }
+ if (!stats.next(k2, v2)) {
+ throw new IOException("Failed to read record #2 from reduce's output");
+ }
+
+ if ((v1.getBytes() != v2.getBytes()) || (v1.getRecords() != v2.getRecords()) ||
+ v1.getChecksum() != v2.getChecksum()) {
+ throw new IOException("(" +
+ v1.getBytes() + ", " + v1.getRecords() + ", " + v1.getChecksum() + ") v/s (" +
+ v2.getBytes() + ", " + v2.getRecords() + ", " + v2.getChecksum() + ")");
+ }
+ }
+
+ }
+
+ /**
+ * A simple map-reduce task to check if the input and the output
+ * of the framework's sort is consistent by ensuring each record
+ * is present in both the input and the output.
+ *
+ */
+ public static class RecordChecker {
+
+ public static class Map extends MapReduceBase
+ implements Mapper {
+
+ private IntWritable value = null;
+
+ public void configure(JobConf job) {
+ // value == one for sort-input; value == two for sort-output
+ value = deduceInputFile(job);
+ }
+
+ public void map(BytesWritable key,
+ BytesWritable value,
+ OutputCollector output,
+ Reporter reporter) throws IOException {
+ // newKey = (key, value)
+ BytesWritable keyValue = new BytesWritable(pair(key, value));
+
+ // output (newKey, value)
+ output.collect(keyValue, this.value);
+ }
+ }
+
+ public static class Reduce extends MapReduceBase
+ implements Reducer {
+
+ public void reduce(BytesWritable key, Iterator values,
+ OutputCollector output,
+ Reporter reporter) throws IOException {
+ int ones = 0;
+ int twos = 0;
+ while (values.hasNext()) {
+ IntWritable count = values.next();
+ if (count.equals(sortInput)) {
+ ++ones;
+ } else if (count.equals(sortOutput)) {
+ ++twos;
+ } else {
+ throw new IOException("Invalid 'value' of " + count.get() +
+ " for (key,value): " + key.toString());
+ }
+ }
+
+ // Check to ensure there are equal no. of ones and twos
+ if (ones != twos) {
+ throw new IOException("Illegal ('one', 'two'): (" + ones + ", " + twos +
+ ") for (key, value): " + key.toString());
+ }
+ }
+ }
+
+ static void checkRecords(Configuration defaults, int noMaps, int noReduces,
+ Path sortInput, Path sortOutput) throws IOException {
+ JobConf jobConf = new JobConf(defaults, RecordChecker.class);
+ jobConf.setJobName("sortvalidate-record-checker");
+
+ jobConf.setInputFormat(SequenceFileInputFormat.class);
+ jobConf.setOutputFormat(SequenceFileOutputFormat.class);
+
+ jobConf.setOutputKeyClass(BytesWritable.class);
+ jobConf.setOutputValueClass(IntWritable.class);
+
+ jobConf.setMapperClass(Map.class);
+ jobConf.setReducerClass(Reduce.class);
+
+ JobClient client = new JobClient(jobConf);
+ ClusterStatus cluster = client.getClusterStatus();
+ if (noMaps == -1) {
+ noMaps = cluster.getTaskTrackers() *
+ jobConf.getInt(MAPS_PER_HOST, 10);
+ }
+ if (noReduces == -1) {
+ noReduces = (int) (cluster.getMaxReduceTasks() * 0.9);
+ String sortReduces = jobConf.get(REDUCES_PER_HOST);
+ if (sortReduces != null) {
+ noReduces = cluster.getTaskTrackers() *
+ Integer.parseInt(sortReduces);
+ }
+ }
+ jobConf.setNumMapTasks(noMaps);
+ jobConf.setNumReduceTasks(noReduces);
+
+ FileInputFormat.setInputPaths(jobConf, sortInput);
+ FileInputFormat.addInputPath(jobConf, sortOutput);
+ Path outputPath = new Path("/tmp/sortvalidate/recordchecker");
+ FileSystem fs = FileSystem.get(defaults);
+ if (fs.exists(outputPath)) {
+ fs.delete(outputPath, true);
+ }
+ FileOutputFormat.setOutputPath(jobConf, outputPath);
+
+ // Uncomment to run locally in a single process
+ //job_conf.set(JTConfig.JT, "local");
+ Path[] inputPaths = FileInputFormat.getInputPaths(jobConf);
+ System.out.println("\nSortValidator.RecordChecker: Running on " +
+ cluster.getTaskTrackers() +
+ " nodes to validate sort from " +
+ inputPaths[0] + ", " +
+ inputPaths[1] + " into " +
+ FileOutputFormat.getOutputPath(jobConf) +
+ " with " + noReduces + " reduces.");
+ Date startTime = new Date();
+ System.out.println("Job started: " + startTime);
+ JobClient.runJob(jobConf);
+ Date end_time = new Date();
+ System.out.println("Job ended: " + end_time);
+ System.out.println("The job took " +
+ (end_time.getTime() - startTime.getTime()) /1000 + " seconds.");
+ }
+ }
+
+
+ /**
+ * The main driver for sort-validator program.
+ * Invoke this method to submit the map/reduce job.
+ * @throws IOException When there is communication problems with the
+ * job tracker.
+ */
+ public int run(String[] args) throws Exception {
+ Configuration defaults = getConf();
+
+ int noMaps = -1, noReduces = -1;
+ Path sortInput = null, sortOutput = null;
+ boolean deepTest = false;
+ for(int i=0; i < args.length; ++i) {
+ try {
+ if ("-m".equals(args[i])) {
+ noMaps = Integer.parseInt(args[++i]);
+ } else if ("-r".equals(args[i])) {
+ noReduces = Integer.parseInt(args[++i]);
+ } else if ("-sortInput".equals(args[i])){
+ sortInput = new Path(args[++i]);
+ } else if ("-sortOutput".equals(args[i])){
+ sortOutput = new Path(args[++i]);
+ } else if ("-deep".equals(args[i])) {
+ deepTest = true;
+ } else {
+ printUsage();
+ return -1;
+ }
+ } catch (NumberFormatException except) {
+ System.err.println("ERROR: Integer expected instead of " + args[i]);
+ printUsage();
+ return -1;
+ } catch (ArrayIndexOutOfBoundsException except) {
+ System.err.println("ERROR: Required parameter missing from " +
+ args[i-1]);
+ printUsage();
+ return -1;
+ }
+ }
+
+ // Sanity check
+ if (sortInput == null || sortOutput == null) {
+ printUsage();
+ return -2;
+ }
+
+ // Check if the records are consistent and sorted correctly
+ RecordStatsChecker.checkRecords(defaults, sortInput, sortOutput);
+
+ // Check if the same records are present in sort's inputs & outputs
+ if (deepTest) {
+ RecordChecker.checkRecords(defaults, noMaps, noReduces, sortInput,
+ sortOutput);
+ }
+
+ System.out.println("\nSUCCESS! Validated the MapReduce framework's 'sort'" +
+ " successfully.");
+
+ return 0;
+ }
+
+ public static void main(String[] args) throws Exception {
+ int res = ToolRunner.run(new Configuration(), new SortValidator(), args);
+ System.exit(res);
+ }
+}
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestFileOutputFormat.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestJobCounters.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobCounters.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestJobCounters.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobCounters.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestLazyOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestLazyOutput.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestLocalMRNotification.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalMRNotification.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestLocalMRNotification.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalMRNotification.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
similarity index 93%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
index 5e510094ced..1b93377fd14 100644
--- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
@@ -48,7 +48,6 @@ public class TestSpecialCharactersInOutputPath extends TestCase {
private static final String OUTPUT_FILENAME = "result[0]";
public static boolean launchJob(URI fileSys,
- String jobTracker,
JobConf conf,
int numMaps,
int numReduces) throws IOException {
@@ -68,8 +67,6 @@ public class TestSpecialCharactersInOutputPath extends TestCase {
// use WordCount example
FileSystem.setDefaultUri(conf, fileSys);
- conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.CLASSIC_FRAMEWORK_NAME);
- conf.set(JTConfig.JT_IPC_ADDRESS, jobTracker);
conf.setJobName("foo");
conf.setInputFormat(TextInputFormat.class);
@@ -113,11 +110,9 @@ public class TestSpecialCharactersInOutputPath extends TestCase {
fileSys = dfs.getFileSystem();
namenode = fileSys.getUri().toString();
mr = new MiniMRCluster(taskTrackers, namenode, 2);
- final String jobTrackerName = "localhost:" + mr.getJobTrackerPort();
JobConf jobConf = new JobConf();
boolean result;
- result = launchJob(fileSys.getUri(), jobTrackerName, jobConf,
- 3, 1);
+ result = launchJob(fileSys.getUri(), jobConf, 3, 1);
assertTrue(result);
} finally {
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestTaskCommit.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
new file mode 100644
index 00000000000..17995603d06
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
@@ -0,0 +1,787 @@
+/**
+ * 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.mapred;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.DataOutputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Enumeration;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.util.StringUtils;
+
+import org.apache.commons.logging.Log;
+
+/**
+ * Utilities used in unit test.
+ *
+ */
+public class UtilsForTests {
+
+ static final Log LOG = LogFactory.getLog(UtilsForTests.class);
+
+ final static long KB = 1024L * 1;
+ final static long MB = 1024L * KB;
+ final static long GB = 1024L * MB;
+ final static long TB = 1024L * GB;
+ final static long PB = 1024L * TB;
+ final static Object waitLock = new Object();
+
+ static DecimalFormat dfm = new DecimalFormat("####.000");
+ static DecimalFormat ifm = new DecimalFormat("###,###,###,###,###");
+
+ public static String dfmt(double d) {
+ return dfm.format(d);
+ }
+
+ public static String ifmt(double d) {
+ return ifm.format(d);
+ }
+
+ public static String formatBytes(long numBytes) {
+ StringBuffer buf = new StringBuffer();
+ boolean bDetails = true;
+ double num = numBytes;
+
+ if (numBytes < KB) {
+ buf.append(numBytes + " B");
+ bDetails = false;
+ } else if (numBytes < MB) {
+ buf.append(dfmt(num / KB) + " KB");
+ } else if (numBytes < GB) {
+ buf.append(dfmt(num / MB) + " MB");
+ } else if (numBytes < TB) {
+ buf.append(dfmt(num / GB) + " GB");
+ } else if (numBytes < PB) {
+ buf.append(dfmt(num / TB) + " TB");
+ } else {
+ buf.append(dfmt(num / PB) + " PB");
+ }
+ if (bDetails) {
+ buf.append(" (" + ifmt(numBytes) + " bytes)");
+ }
+ return buf.toString();
+ }
+
+ public static String formatBytes2(long numBytes) {
+ StringBuffer buf = new StringBuffer();
+ long u = 0;
+ if (numBytes >= TB) {
+ u = numBytes / TB;
+ numBytes -= u * TB;
+ buf.append(u + " TB ");
+ }
+ if (numBytes >= GB) {
+ u = numBytes / GB;
+ numBytes -= u * GB;
+ buf.append(u + " GB ");
+ }
+ if (numBytes >= MB) {
+ u = numBytes / MB;
+ numBytes -= u * MB;
+ buf.append(u + " MB ");
+ }
+ if (numBytes >= KB) {
+ u = numBytes / KB;
+ numBytes -= u * KB;
+ buf.append(u + " KB ");
+ }
+ buf.append(u + " B"); //even if zero
+ return buf.toString();
+ }
+
+ static final String regexpSpecials = "[]()?*+|.!^-\\~@";
+
+ public static String regexpEscape(String plain) {
+ StringBuffer buf = new StringBuffer();
+ char[] ch = plain.toCharArray();
+ int csup = ch.length;
+ for (int c = 0; c < csup; c++) {
+ if (regexpSpecials.indexOf(ch[c]) != -1) {
+ buf.append("\\");
+ }
+ buf.append(ch[c]);
+ }
+ return buf.toString();
+ }
+
+ public static String safeGetCanonicalPath(File f) {
+ try {
+ String s = f.getCanonicalPath();
+ return (s == null) ? f.toString() : s;
+ } catch (IOException io) {
+ return f.toString();
+ }
+ }
+
+ public static String slurp(File f) throws IOException {
+ int len = (int) f.length();
+ byte[] buf = new byte[len];
+ FileInputStream in = new FileInputStream(f);
+ String contents = null;
+ try {
+ in.read(buf, 0, len);
+ contents = new String(buf, "UTF-8");
+ } finally {
+ in.close();
+ }
+ return contents;
+ }
+
+ public static String slurpHadoop(Path p, FileSystem fs) throws IOException {
+ int len = (int) fs.getFileStatus(p).getLen();
+ byte[] buf = new byte[len];
+ InputStream in = fs.open(p);
+ String contents = null;
+ try {
+ in.read(buf, 0, len);
+ contents = new String(buf, "UTF-8");
+ } finally {
+ in.close();
+ }
+ return contents;
+ }
+
+ public static String rjustify(String s, int width) {
+ if (s == null) s = "null";
+ if (width > s.length()) {
+ s = getSpace(width - s.length()) + s;
+ }
+ return s;
+ }
+
+ public static String ljustify(String s, int width) {
+ if (s == null) s = "null";
+ if (width > s.length()) {
+ s = s + getSpace(width - s.length());
+ }
+ return s;
+ }
+
+ static char[] space;
+ static {
+ space = new char[300];
+ Arrays.fill(space, '\u0020');
+ }
+
+ public static String getSpace(int len) {
+ if (len > space.length) {
+ space = new char[Math.max(len, 2 * space.length)];
+ Arrays.fill(space, '\u0020');
+ }
+ return new String(space, 0, len);
+ }
+
+ /**
+ * Gets job status from the jobtracker given the jobclient and the job id
+ */
+ static JobStatus getJobStatus(JobClient jc, JobID id) throws IOException {
+ JobStatus[] statuses = jc.getAllJobs();
+ for (JobStatus jobStatus : statuses) {
+ if (jobStatus.getJobID().equals(id)) {
+ return jobStatus;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * A utility that waits for specified amount of time
+ */
+ public static void waitFor(long duration) {
+ try {
+ synchronized (waitLock) {
+ waitLock.wait(duration);
+ }
+ } catch (InterruptedException ie) {}
+ }
+
+ /**
+ * Wait for the jobtracker to be RUNNING.
+ */
+ static void waitForJobTracker(JobClient jobClient) {
+ while (true) {
+ try {
+ ClusterStatus status = jobClient.getClusterStatus();
+ while (status.getJobTrackerStatus() != JobTrackerStatus.RUNNING) {
+ waitFor(100);
+ status = jobClient.getClusterStatus();
+ }
+ break; // means that the jt is ready
+ } catch (IOException ioe) {}
+ }
+ }
+
+ /**
+ * Waits until all the jobs at the jobtracker complete.
+ */
+ static void waitTillDone(JobClient jobClient) throws IOException {
+ // Wait for the last job to complete
+ while (true) {
+ boolean shouldWait = false;
+ for (JobStatus jobStatuses : jobClient.getAllJobs()) {
+ if (jobStatuses.getRunState() != JobStatus.SUCCEEDED
+ && jobStatuses.getRunState() != JobStatus.FAILED
+ && jobStatuses.getRunState() != JobStatus.KILLED) {
+ shouldWait = true;
+ break;
+ }
+ }
+ if (shouldWait) {
+ waitFor(100);
+ } else {
+ break;
+ }
+ }
+ }
+
+ /**
+ * Configure a waiting job
+ */
+ static void configureWaitingJobConf(JobConf jobConf, Path inDir,
+ Path outputPath, int numMaps, int numRed,
+ String jobName, String mapSignalFilename,
+ String redSignalFilename)
+ throws IOException {
+ jobConf.setJobName(jobName);
+ jobConf.setInputFormat(NonSplitableSequenceFileInputFormat.class);
+ jobConf.setOutputFormat(SequenceFileOutputFormat.class);
+ FileInputFormat.setInputPaths(jobConf, inDir);
+ FileOutputFormat.setOutputPath(jobConf, outputPath);
+ jobConf.setMapperClass(UtilsForTests.HalfWaitingMapper.class);
+ jobConf.setReducerClass(IdentityReducer.class);
+ jobConf.setOutputKeyClass(BytesWritable.class);
+ jobConf.setOutputValueClass(BytesWritable.class);
+ jobConf.setInputFormat(RandomInputFormat.class);
+ jobConf.setNumMapTasks(numMaps);
+ jobConf.setNumReduceTasks(numRed);
+ jobConf.setJar("build/test/mapred/testjar/testjob.jar");
+ jobConf.set(getTaskSignalParameter(true), mapSignalFilename);
+ jobConf.set(getTaskSignalParameter(false), redSignalFilename);
+ }
+
+ /**
+ * Commonly used map and reduce classes
+ */
+
+ /**
+ * Map is a Mapper that just waits for a file to be created on the dfs. The
+ * file creation is a signal to the mappers and hence acts as a waiting job.
+ */
+
+ static class WaitingMapper
+ extends MapReduceBase
+ implements Mapper {
+
+ FileSystem fs = null;
+ Path signal;
+ int id = 0;
+ int totalMaps = 0;
+
+ /**
+ * Checks if the map task needs to wait. By default all the maps will wait.
+ * This method needs to be overridden to make a custom waiting mapper.
+ */
+ public boolean shouldWait(int id) {
+ return true;
+ }
+
+ /**
+ * Returns a signal file on which the map task should wait. By default all
+ * the maps wait on a single file passed as test.mapred.map.waiting.target.
+ * This method needs to be overridden to make a custom waiting mapper
+ */
+ public Path getSignalFile(int id) {
+ return signal;
+ }
+
+ /** The waiting function. The map exits once it gets a signal. Here the
+ * signal is the file existence.
+ */
+ public void map(WritableComparable key, Writable val,
+ OutputCollector output,
+ Reporter reporter)
+ throws IOException {
+ if (shouldWait(id)) {
+ if (fs != null) {
+ while (!fs.exists(getSignalFile(id))) {
+ try {
+ reporter.progress();
+ synchronized (this) {
+ this.wait(1000); // wait for 1 sec
+ }
+ } catch (InterruptedException ie) {
+ System.out.println("Interrupted while the map was waiting for "
+ + " the signal.");
+ break;
+ }
+ }
+ } else {
+ throw new IOException("Could not get the DFS!!");
+ }
+ }
+ }
+
+ public void configure(JobConf conf) {
+ try {
+ String taskId = conf.get(JobContext.TASK_ATTEMPT_ID);
+ id = Integer.parseInt(taskId.split("_")[4]);
+ totalMaps = Integer.parseInt(conf.get(JobContext.NUM_MAPS));
+ fs = FileSystem.get(conf);
+ signal = new Path(conf.get(getTaskSignalParameter(true)));
+ } catch (IOException ioe) {
+ System.out.println("Got an exception while obtaining the filesystem");
+ }
+ }
+ }
+
+ /** Only the later half of the maps wait for the signal while the rest
+ * complete immediately.
+ */
+ static class HalfWaitingMapper extends WaitingMapper {
+ @Override
+ public boolean shouldWait(int id) {
+ return id >= (totalMaps / 2);
+ }
+ }
+
+ /**
+ * Reduce that just waits for a file to be created on the dfs. The
+ * file creation is a signal to the reduce.
+ */
+
+ static class WaitingReducer extends MapReduceBase
+ implements Reducer {
+
+ FileSystem fs = null;
+ Path signal;
+
+ /** The waiting function. The reduce exits once it gets a signal. Here the
+ * signal is the file existence.
+ */
+ public void reduce(WritableComparable key, Iterator val,
+ OutputCollector output,
+ Reporter reporter)
+ throws IOException {
+ if (fs != null) {
+ while (!fs.exists(signal)) {
+ try {
+ reporter.progress();
+ synchronized (this) {
+ this.wait(1000); // wait for 1 sec
+ }
+ } catch (InterruptedException ie) {
+ System.out.println("Interrupted while the map was waiting for the"
+ + " signal.");
+ break;
+ }
+ }
+ } else {
+ throw new IOException("Could not get the DFS!!");
+ }
+ }
+
+ public void configure(JobConf conf) {
+ try {
+ fs = FileSystem.get(conf);
+ signal = new Path(conf.get(getTaskSignalParameter(false)));
+ } catch (IOException ioe) {
+ System.out.println("Got an exception while obtaining the filesystem");
+ }
+ }
+ }
+
+ static String getTaskSignalParameter(boolean isMap) {
+ return isMap
+ ? "test.mapred.map.waiting.target"
+ : "test.mapred.reduce.waiting.target";
+ }
+
+ /**
+ * Signal the maps/reduces to start.
+ */
+ static void signalTasks(MiniDFSCluster dfs, FileSystem fileSys,
+ String mapSignalFile,
+ String reduceSignalFile, int replication)
+ throws IOException {
+ writeFile(dfs.getNameNode(), fileSys.getConf(), new Path(mapSignalFile),
+ (short)replication);
+ writeFile(dfs.getNameNode(), fileSys.getConf(), new Path(reduceSignalFile),
+ (short)replication);
+ }
+
+ /**
+ * Signal the maps/reduces to start.
+ */
+ static void signalTasks(MiniDFSCluster dfs, FileSystem fileSys,
+ boolean isMap, String mapSignalFile,
+ String reduceSignalFile)
+ throws IOException {
+ // signal the maps to complete
+ writeFile(dfs.getNameNode(), fileSys.getConf(),
+ isMap
+ ? new Path(mapSignalFile)
+ : new Path(reduceSignalFile), (short)1);
+ }
+
+ static String getSignalFile(Path dir) {
+ return (new Path(dir, "signal")).toString();
+ }
+
+ static String getMapSignalFile(Path dir) {
+ return (new Path(dir, "map-signal")).toString();
+ }
+
+ static String getReduceSignalFile(Path dir) {
+ return (new Path(dir, "reduce-signal")).toString();
+ }
+
+ static void writeFile(NameNode namenode, Configuration conf, Path name,
+ short replication) throws IOException {
+ FileSystem fileSys = FileSystem.get(conf);
+ SequenceFile.Writer writer =
+ SequenceFile.createWriter(fileSys, conf, name,
+ BytesWritable.class, BytesWritable.class,
+ CompressionType.NONE);
+ writer.append(new BytesWritable(), new BytesWritable());
+ writer.close();
+ fileSys.setReplication(name, replication);
+ DFSTestUtil.waitReplication(fileSys, name, replication);
+ }
+
+ // Input formats
+ /**
+ * A custom input format that creates virtual inputs of a single string
+ * for each map.
+ */
+ public static class RandomInputFormat implements InputFormat {
+
+ public InputSplit[] getSplits(JobConf job,
+ int numSplits) throws IOException {
+ InputSplit[] result = new InputSplit[numSplits];
+ Path outDir = FileOutputFormat.getOutputPath(job);
+ for(int i=0; i < result.length; ++i) {
+ result[i] = new FileSplit(new Path(outDir, "dummy-split-" + i),
+ 0, 1, (String[])null);
+ }
+ return result;
+ }
+
+ static class RandomRecordReader implements RecordReader {
+ Path name;
+ public RandomRecordReader(Path p) {
+ name = p;
+ }
+ public boolean next(Text key, Text value) {
+ if (name != null) {
+ key.set(name.getName());
+ name = null;
+ return true;
+ }
+ return false;
+ }
+ public Text createKey() {
+ return new Text();
+ }
+ public Text createValue() {
+ return new Text();
+ }
+ public long getPos() {
+ return 0;
+ }
+ public void close() {}
+ public float getProgress() {
+ return 0.0f;
+ }
+ }
+
+ public RecordReader getRecordReader(InputSplit split,
+ JobConf job,
+ Reporter reporter)
+ throws IOException {
+ return new RandomRecordReader(((FileSplit) split).getPath());
+ }
+ }
+
+ // Start a job and return its RunningJob object
+ static RunningJob runJob(JobConf conf, Path inDir, Path outDir)
+ throws IOException {
+ return runJob(conf, inDir, outDir, conf.getNumMapTasks(), conf.getNumReduceTasks());
+ }
+
+ // Start a job and return its RunningJob object
+ static RunningJob runJob(JobConf conf, Path inDir, Path outDir, int numMaps,
+ int numReds) throws IOException {
+
+ String input = "The quick brown fox\n" + "has many silly\n"
+ + "red fox sox\n";
+
+ // submit the job and wait for it to complete
+ return runJob(conf, inDir, outDir, numMaps, numReds, input);
+ }
+
+ // Start a job with the specified input and return its RunningJob object
+ static RunningJob runJob(JobConf conf, Path inDir, Path outDir, int numMaps,
+ int numReds, String input) throws IOException {
+ FileSystem fs = FileSystem.get(conf);
+ if (fs.exists(outDir)) {
+ fs.delete(outDir, true);
+ }
+ if (!fs.exists(inDir)) {
+ fs.mkdirs(inDir);
+ }
+
+ for (int i = 0; i < numMaps; ++i) {
+ DataOutputStream file = fs.create(new Path(inDir, "part-" + i));
+ file.writeBytes(input);
+ file.close();
+ }
+
+ conf.setInputFormat(TextInputFormat.class);
+ conf.setOutputKeyClass(LongWritable.class);
+ conf.setOutputValueClass(Text.class);
+
+ FileInputFormat.setInputPaths(conf, inDir);
+ FileOutputFormat.setOutputPath(conf, outDir);
+ conf.setNumMapTasks(numMaps);
+ conf.setNumReduceTasks(numReds);
+
+ JobClient jobClient = new JobClient(conf);
+ RunningJob job = jobClient.submitJob(conf);
+
+ return job;
+ }
+
+ // Run a job that will be succeeded and wait until it completes
+ public static RunningJob runJobSucceed(JobConf conf, Path inDir, Path outDir)
+ throws IOException {
+ conf.setJobName("test-job-succeed");
+ conf.setMapperClass(IdentityMapper.class);
+ conf.setReducerClass(IdentityReducer.class);
+
+ RunningJob job = UtilsForTests.runJob(conf, inDir, outDir);
+ while (!job.isComplete()) {
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ break;
+ }
+ }
+
+ return job;
+ }
+
+ // Run a job that will be failed and wait until it completes
+ public static RunningJob runJobFail(JobConf conf, Path inDir, Path outDir)
+ throws IOException {
+ conf.setJobName("test-job-fail");
+ conf.setMapperClass(FailMapper.class);
+ conf.setReducerClass(IdentityReducer.class);
+ conf.setMaxMapAttempts(1);
+
+ RunningJob job = UtilsForTests.runJob(conf, inDir, outDir);
+ while (!job.isComplete()) {
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ break;
+ }
+ }
+
+ return job;
+ }
+
+ // Run a job that will be killed and wait until it completes
+ public static RunningJob runJobKill(JobConf conf, Path inDir, Path outDir)
+ throws IOException {
+
+ conf.setJobName("test-job-kill");
+ conf.setMapperClass(KillMapper.class);
+ conf.setReducerClass(IdentityReducer.class);
+
+ RunningJob job = UtilsForTests.runJob(conf, inDir, outDir);
+ while (job.getJobState() != JobStatus.RUNNING) {
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ break;
+ }
+ }
+ job.killJob();
+ while (job.cleanupProgress() == 0.0f) {
+ try {
+ Thread.sleep(10);
+ } catch (InterruptedException ie) {
+ break;
+ }
+ }
+
+ return job;
+ }
+
+ /**
+ * Cleans up files/dirs inline. CleanupQueue deletes in a separate thread
+ * asynchronously.
+ */
+ public static class InlineCleanupQueue extends CleanupQueue {
+ List stalePaths = new ArrayList();
+
+ public InlineCleanupQueue() {
+ // do nothing
+ }
+
+ @Override
+ public void addToQueue(PathDeletionContext... contexts) {
+ // delete paths in-line
+ for (PathDeletionContext context : contexts) {
+ try {
+ if (!deletePath(context)) {
+ LOG.warn("Stale path " + context.fullPath);
+ stalePaths.add(context.fullPath);
+ }
+ } catch (IOException e) {
+ LOG.warn("Caught exception while deleting path "
+ + context.fullPath);
+ LOG.info(StringUtils.stringifyException(e));
+ stalePaths.add(context.fullPath);
+ }
+ }
+ }
+ }
+
+ static class FakeClock extends Clock {
+ long time = 0;
+
+ public void advance(long millis) {
+ time += millis;
+ }
+
+ @Override
+ long getTime() {
+ return time;
+ }
+ }
+ // Mapper that fails
+ static class FailMapper extends MapReduceBase implements
+ Mapper {
+
+ public void map(WritableComparable key, Writable value,
+ OutputCollector out, Reporter reporter)
+ throws IOException {
+ //NOTE- the next line is required for the TestDebugScript test to succeed
+ System.err.println("failing map");
+ throw new RuntimeException("failing map");
+ }
+ }
+
+ // Mapper that sleeps for a long time.
+ // Used for running a job that will be killed
+ static class KillMapper extends MapReduceBase implements
+ Mapper {
+
+ public void map(WritableComparable key, Writable value,
+ OutputCollector out, Reporter reporter)
+ throws IOException {
+
+ try {
+ Thread.sleep(1000000);
+ } catch (InterruptedException e) {
+ // Do nothing
+ }
+ }
+ }
+
+ static void setUpConfigFile(Properties confProps, File configFile)
+ throws IOException {
+ Configuration config = new Configuration(false);
+ FileOutputStream fos = new FileOutputStream(configFile);
+
+ for (Enumeration> e = confProps.propertyNames(); e.hasMoreElements();) {
+ String key = (String) e.nextElement();
+ config.set(key, confProps.getProperty(key));
+ }
+
+ config.writeXml(fos);
+ fos.close();
+ }
+
+ /**
+ * This creates a file in the dfs
+ * @param dfs FileSystem Local File System where file needs to be picked
+ * @param URIPATH Path dfs path where file needs to be copied
+ * @param permission FsPermission File permission
+ * @return returns the DataOutputStream
+ */
+ public static DataOutputStream
+ createTmpFileDFS(FileSystem dfs, Path URIPATH,
+ FsPermission permission, String input) throws Exception {
+ //Creating the path with the file
+ DataOutputStream file =
+ FileSystem.create(dfs, URIPATH, permission);
+ file.writeBytes(input);
+ file.close();
+ return file;
+ }
+
+ /**
+ * This formats the long tasktracker name to just the FQDN
+ * @param taskTrackerLong String The long format of the tasktracker string
+ * @return String The FQDN of the tasktracker
+ * @throws Exception
+ */
+ public static String getFQDNofTT (String taskTrackerLong) throws Exception {
+ //Getting the exact FQDN of the tasktracker from the tasktracker string.
+ String[] firstSplit = taskTrackerLong.split("_");
+ String tmpOutput = firstSplit[1];
+ String[] secondSplit = tmpOutput.split(":");
+ String tmpTaskTracker = secondSplit[0];
+ return tmpTaskTracker;
+ }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/WordCount.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/WordCount.java
new file mode 100644
index 00000000000..60d29001924
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/WordCount.java
@@ -0,0 +1,159 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * This is an example Hadoop Map/Reduce application.
+ * It reads the text input files, breaks each line into words
+ * and counts them. The output is a locally sorted list of words and the
+ * count of how often they occurred.
+ *
+ * To run: bin/hadoop jar build/hadoop-examples.jar wordcount
+ * [-m maps ] [-r reduces ] in-dir out-dir
+ */
+public class WordCount extends Configured implements Tool {
+
+ /**
+ * Counts the words in each line.
+ * For each line of input, break the line into words and emit them as
+ * (word , 1 ).
+ */
+ public static class MapClass extends MapReduceBase
+ implements Mapper {
+
+ private final static IntWritable one = new IntWritable(1);
+ private Text word = new Text();
+
+ public void map(LongWritable key, Text value,
+ OutputCollector output,
+ Reporter reporter) throws IOException {
+ String line = value.toString();
+ StringTokenizer itr = new StringTokenizer(line);
+ while (itr.hasMoreTokens()) {
+ word.set(itr.nextToken());
+ output.collect(word, one);
+ }
+ }
+ }
+
+ /**
+ * A reducer class that just emits the sum of the input values.
+ */
+ public static class Reduce extends MapReduceBase
+ implements Reducer {
+
+ public void reduce(Text key, Iterator values,
+ OutputCollector output,
+ Reporter reporter) throws IOException {
+ int sum = 0;
+ while (values.hasNext()) {
+ sum += values.next().get();
+ }
+ output.collect(key, new IntWritable(sum));
+ }
+ }
+
+ static int printUsage() {
+ System.out.println("wordcount [-m ] [-r ] ");
+ ToolRunner.printGenericCommandUsage(System.out);
+ return -1;
+ }
+
+ /**
+ * The main driver for word count map/reduce program.
+ * Invoke this method to submit the map/reduce job.
+ * @throws IOException When there is communication problems with the
+ * job tracker.
+ */
+ public int run(String[] args) throws Exception {
+ JobConf conf = new JobConf(getConf(), WordCount.class);
+ conf.setJobName("wordcount");
+
+ // the keys are words (strings)
+ conf.setOutputKeyClass(Text.class);
+ // the values are counts (ints)
+ conf.setOutputValueClass(IntWritable.class);
+
+ conf.setMapperClass(MapClass.class);
+ conf.setCombinerClass(Reduce.class);
+ conf.setReducerClass(Reduce.class);
+
+ List other_args = new ArrayList();
+ for(int i=0; i < args.length; ++i) {
+ try {
+ if ("-m".equals(args[i])) {
+ conf.setNumMapTasks(Integer.parseInt(args[++i]));
+ } else if ("-r".equals(args[i])) {
+ conf.setNumReduceTasks(Integer.parseInt(args[++i]));
+ } else {
+ other_args.add(args[i]);
+ }
+ } catch (NumberFormatException except) {
+ System.out.println("ERROR: Integer expected instead of " + args[i]);
+ return printUsage();
+ } catch (ArrayIndexOutOfBoundsException except) {
+ System.out.println("ERROR: Required parameter missing from " +
+ args[i-1]);
+ return printUsage();
+ }
+ }
+ // Make sure there are exactly 2 parameters left.
+ if (other_args.size() != 2) {
+ System.out.println("ERROR: Wrong number of parameters: " +
+ other_args.size() + " instead of 2.");
+ return printUsage();
+ }
+ FileInputFormat.setInputPaths(conf, other_args.get(0));
+ FileOutputFormat.setOutputPath(conf, new Path(other_args.get(1)));
+
+ JobClient.runJob(conf);
+ return 0;
+ }
+
+
+ public static void main(String[] args) throws Exception {
+ int res = ToolRunner.run(new Configuration(), new WordCount(), args);
+ System.exit(res);
+ }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java
new file mode 100644
index 00000000000..d160de5db61
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java
@@ -0,0 +1,154 @@
+/**
+ * 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.mapred.jobcontrol;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Utility methods used in various Job Control unit tests.
+ */
+public class JobControlTestUtils {
+
+ static private Random rand = new Random();
+
+ private static NumberFormat idFormat = NumberFormat.getInstance();
+
+ static {
+ idFormat.setMinimumIntegerDigits(4);
+ idFormat.setGroupingUsed(false);
+ }
+
+ /**
+ * Cleans the data from the passed Path in the passed FileSystem.
+ *
+ * @param fs FileSystem to delete data from.
+ * @param dirPath Path to be deleted.
+ * @throws IOException If an error occurs cleaning the data.
+ */
+ static void cleanData(FileSystem fs, Path dirPath) throws IOException {
+ fs.delete(dirPath, true);
+ }
+
+ /**
+ * Generates a string of random digits.
+ *
+ * @return A random string.
+ */
+ private static String generateRandomWord() {
+ return idFormat.format(rand.nextLong());
+ }
+
+ /**
+ * Generates a line of random text.
+ *
+ * @return A line of random text.
+ */
+ private static String generateRandomLine() {
+ long r = rand.nextLong() % 7;
+ long n = r + 20;
+ StringBuffer sb = new StringBuffer();
+ for (int i = 0; i < n; i++) {
+ sb.append(generateRandomWord()).append(" ");
+ }
+ sb.append("\n");
+ return sb.toString();
+ }
+
+ /**
+ * Generates data that can be used for Job Control tests.
+ *
+ * @param fs FileSystem to create data in.
+ * @param dirPath Path to create the data in.
+ * @throws IOException If an error occurs creating the data.
+ */
+ static void generateData(FileSystem fs, Path dirPath) throws IOException {
+ FSDataOutputStream out = fs.create(new Path(dirPath, "data.txt"));
+ for (int i = 0; i < 10000; i++) {
+ String line = generateRandomLine();
+ out.write(line.getBytes("UTF-8"));
+ }
+ out.close();
+ }
+
+ /**
+ * Creates a simple copy job.
+ *
+ * @param indirs List of input directories.
+ * @param outdir Output directory.
+ * @return JobConf initialised for a simple copy job.
+ * @throws Exception If an error occurs creating job configuration.
+ */
+ static JobConf createCopyJob(List indirs, Path outdir) throws Exception {
+
+ Configuration defaults = new Configuration();
+ JobConf theJob = new JobConf(defaults, TestJobControl.class);
+ theJob.setJobName("DataMoveJob");
+
+ FileInputFormat.setInputPaths(theJob, indirs.toArray(new Path[0]));
+ theJob.setMapperClass(DataCopy.class);
+ FileOutputFormat.setOutputPath(theJob, outdir);
+ theJob.setOutputKeyClass(Text.class);
+ theJob.setOutputValueClass(Text.class);
+ theJob.setReducerClass(DataCopy.class);
+ theJob.setNumMapTasks(12);
+ theJob.setNumReduceTasks(4);
+ return theJob;
+ }
+
+ /**
+ * Simple Mapper and Reducer implementation which copies data it reads in.
+ */
+ public static class DataCopy extends MapReduceBase implements
+ Mapper, Reducer {
+ public void map(LongWritable key, Text value, OutputCollector output,
+ Reporter reporter) throws IOException {
+ output.collect(new Text(key.toString()), value);
+ }
+
+ public void reduce(Text key, Iterator values,
+ OutputCollector output, Reporter reporter)
+ throws IOException {
+ Text dumbKey = new Text("");
+ while (values.hasNext()) {
+ Text data = values.next();
+ output.collect(dumbKey, data);
+ }
+ }
+ }
+
+}
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/jobcontrol/TestJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestJobControl.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/jobcontrol/TestJobControl.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestJobControl.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/lib/TestChainMapReduce.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestChild.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
similarity index 100%
rename from hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
rename to hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/hadoop-mapreduce-examples/dev-support/findbugs-exclude.xml
new file mode 100644
index 00000000000..244adafebdc
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/dev-support/findbugs-exclude.xml
@@ -0,0 +1,63 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
index 48824df0829..60557ddc738 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
@@ -27,6 +27,10 @@
Apache Hadoop MapReduce Examples
jar
+
+ ${basedir}
+
+
org.apache.hadoop
@@ -58,6 +62,18 @@
+
+
+ org.codehaus.mojo
+ findbugs-maven-plugin
+
+ true
+ true
+ ${mr.examples.basedir}/dev-support/findbugs-exclude.xml
+ Max
+
+
+
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Join.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Join.java
index 86a13ad0ca2..2063d04655a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Join.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Join.java
@@ -19,7 +19,9 @@
package org.apache.hadoop.examples;
import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@@ -29,9 +31,14 @@ import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapred.ClusterStatus;
import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.join.*;
+import org.apache.hadoop.mapreduce.lib.join.CompositeInputFormat;
+import org.apache.hadoop.mapreduce.lib.join.TupleWritable;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
import org.apache.hadoop.util.Tool;
@@ -52,7 +59,7 @@ import org.apache.hadoop.util.ToolRunner;
* [in-dir ]* in-dir out-dir
*/
public class Join extends Configured implements Tool {
- public static String REDUCES_PER_HOST = "mapreduce.join.reduces_per_host";
+ public static final String REDUCES_PER_HOST = "mapreduce.join.reduces_per_host";
static int printUsage() {
System.out.println("join [-r ] " +
"[-inFormat ] " +
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java
index 3b8759a9774..5e636b901e7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java
@@ -69,7 +69,7 @@ public class Pentomino {
}
public int[] getRotations() {
- return rotations;
+ return rotations.clone();
}
public boolean getFlippable() {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java
index ec99c19bc44..9880d54003e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java
@@ -70,7 +70,7 @@ public class TeraGen extends Configured implements Tool {
public static enum Counters {CHECKSUM}
- public static String NUM_ROWS = "mapreduce.terasort.num-rows";
+ public static final String NUM_ROWS = "mapreduce.terasort.num-rows";
/**
* An input format that assigns ranges of longs to each mapper.
*/
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java
index a381aba913f..4ef0033012f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java
@@ -156,10 +156,10 @@ public class TeraInputFormat extends FileInputFormat {
* them and picks N-1 keys to generate N equally sized partitions.
* @param job the job to sample
* @param partFile where to write the output file to
- * @throws IOException if something goes wrong
+ * @throws Throwable if something goes wrong
*/
public static void writePartitionFile(final JobContext job,
- Path partFile) throws IOException, InterruptedException {
+ Path partFile) throws Throwable {
long t1 = System.currentTimeMillis();
Configuration conf = job.getConfiguration();
final TeraInputFormat inFormat = new TeraInputFormat();
@@ -174,11 +174,12 @@ public class TeraInputFormat extends FileInputFormat {
final long recordsPerSample = sampleSize / samples;
final int sampleStep = splits.size() / samples;
Thread[] samplerReader = new Thread[samples];
+ SamplerThreadGroup threadGroup = new SamplerThreadGroup("Sampler Reader Thread Group");
// take N samples from different parts of the input
for(int i=0; i < samples; ++i) {
final int idx = i;
samplerReader[i] =
- new Thread ("Sampler Reader " + idx) {
+ new Thread (threadGroup,"Sampler Reader " + idx) {
{
setDaemon(true);
}
@@ -201,7 +202,7 @@ public class TeraInputFormat extends FileInputFormat {
} catch (IOException ie){
System.err.println("Got an exception while reading splits " +
StringUtils.stringifyException(ie));
- System.exit(-1);
+ throw new RuntimeException(ie);
} catch (InterruptedException e) {
}
@@ -215,6 +216,9 @@ public class TeraInputFormat extends FileInputFormat {
for (int i = 0; i < samples; i++) {
try {
samplerReader[i].join();
+ if(threadGroup.getThrowable() != null){
+ throw threadGroup.getThrowable();
+ }
} catch (InterruptedException e) {
}
}
@@ -225,6 +229,25 @@ public class TeraInputFormat extends FileInputFormat {
long t3 = System.currentTimeMillis();
System.out.println("Computing parititions took " + (t3 - t2) + "ms");
}
+
+ static class SamplerThreadGroup extends ThreadGroup{
+
+ private Throwable throwable;
+
+ public SamplerThreadGroup(String s) {
+ super(s);
+ }
+
+ @Override
+ public void uncaughtException(Thread thread, Throwable throwable) {
+ this.throwable = throwable;
+ }
+
+ public Throwable getThrowable() {
+ return this.throwable;
+ }
+
+ }
static class TeraRecordReader extends RecordReader {
private FSDataInputStream in;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
index 2ce8155b476..7daa3016c21 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -300,7 +299,12 @@ public class TeraSort extends Configured implements Tool {
TeraInputFormat.PARTITION_FILENAME);
URI partitionUri = new URI(partitionFile.toString() +
"#" + TeraInputFormat.PARTITION_FILENAME);
- TeraInputFormat.writePartitionFile(job, partitionFile);
+ try {
+ TeraInputFormat.writePartitionFile(job, partitionFile);
+ } catch (Throwable e) {
+ LOG.error(e.getMessage());
+ return -1;
+ }
job.addCacheFile(partitionUri);
job.createSymlink();
long end = System.currentTimeMillis();
diff --git a/hadoop-mapreduce-project/hadoop-yarn/bin/yarn b/hadoop-mapreduce-project/hadoop-yarn/bin/yarn
index b11c94b37ee..b8e23a97f5e 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/bin/yarn
+++ b/hadoop-mapreduce-project/hadoop-yarn/bin/yarn
@@ -148,9 +148,13 @@ IFS=
# add hadoop-common libs to CLASSPATH
if [ ! -d "$HADOOP_COMMON_HOME" ]; then
- echo No HADOOP_COMMON_HOME set.
- echo Please specify it either in yarn-env.sh or in the environment.
- exit 1
+ if [ -d "$HADOOP_PREFIX" ]; then
+ export HADOOP_COMMON_HOME=$HADOOP_PREFIX
+ else
+ echo No HADOOP_COMMON_HOME set.
+ echo Please specify it either in yarn-env.sh or in the environment.
+ exit 1
+ fi
fi
CLASSPATH=${CLASSPATH}:$HADOOP_COMMON_HOME/share/hadoop/common'/*'
@@ -158,9 +162,13 @@ CLASSPATH=${CLASSPATH}:$HADOOP_COMMON_HOME/share/hadoop/common/lib'/*'
# add hadoop-hdfs libs to CLASSPATH
if [ ! -d "$HADOOP_HDFS_HOME" ]; then
- echo No HADOOP_HDFS_HOME set.
- echo Please specify it either in yarn-env.sh or in the environment.
- exit 1
+ if [ -d "$HADOOP_PREFIX" ]; then
+ export HADOOP_HDFS_HOME=$HADOOP_PREFIX
+ else
+ echo No HADOOP_HDFS_HOME set.
+ echo Please specify it either in yarn-env.sh or in the environment.
+ exit 1
+ fi
fi
CLASSPATH=${CLASSPATH}:$HADOOP_HDFS_HOME/share/hadoop/hdfs'/*'
CLASSPATH=${CLASSPATH}:$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib'/*'
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
index e2dfc82c4b9..1ef35ac289d 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
@@ -73,6 +73,11 @@ public interface ContainerStatus {
* Container killed by the framework, either due to being released by
* the application or being 'lost' due to node failures etc. have a special
* exit code of {@literal -100}.
+ *
+ * When threshold number of the nodemanager-local-directories or
+ * threshold number of the nodemanager-log-directories become bad, then
+ * container is not launched and is exited with exit status of
+ * {@literal -101}.
*
* @return exit status for the container
*/
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
index 39b15e0cefd..ef655100f00 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
@@ -340,6 +340,21 @@ public class ContainerPBImpl extends ProtoBase implements Contai
return ((ContainerStatusPBImpl)t).getProto();
}
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Container: [");
+ sb.append("ContainerId: ").append(getId()).append(", ");
+ sb.append("NodeId: ").append(getNodeId()).append(", ");
+ sb.append("NodeHttpAddress: ").append(getNodeHttpAddress()).append(", ");
+ sb.append("Resource: ").append(getResource()).append(", ");
+ sb.append("Priority: ").append(getPriority()).append(", ");
+ sb.append("State: ").append(getState()).append(", ");
+ sb.append("Token: ").append(getContainerToken()).append(", ");
+ sb.append("Status: ").append(getContainerStatus());
+ sb.append("]");
+ return sb.toString();
+ }
+
//TODO Comparator
@Override
public int compareTo(Container other) {
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index d4b8f9fc56c..fa4ffa1656a 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -431,6 +431,7 @@ public class YarnConfiguration extends Configuration {
public static final int INVALID_CONTAINER_EXIT_STATUS = -1000;
public static final int ABORTED_CONTAINER_EXIT_STATUS = -100;
+ public static final int DISKS_FAILED = -101;
////////////////////////////////
// Web Proxy Configs
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java
index ad8938f378d..5e84c3e501d 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java
@@ -40,6 +40,7 @@ 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.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
@@ -309,8 +310,8 @@ public class ProtoOverHadoopRpcEngine implements RpcEngine {
}
@Override
- public Writable call(String protocol, Writable writableRequest,
- long receiveTime) throws IOException {
+ 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();
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
index e6a47da89c9..9cffde1a65d 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
@@ -122,8 +122,7 @@ public abstract class ContainerExecutor implements Configurable {
public enum ExitCode {
FORCE_KILLED(137),
- TERMINATED(143),
- DISKS_FAILED(-101);
+ TERMINATED(143);
private final int code;
private ExitCode(int exitCode) {
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
index 15de5d2749b..821d4a042b3 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
@@ -181,7 +181,7 @@ public class ContainerLaunch implements Callable {
List logDirs = dirsHandler.getLogDirs();
if (!dirsHandler.areDisksHealthy()) {
- ret = ExitCode.DISKS_FAILED.getExitCode();
+ ret = YarnConfiguration.DISKS_FAILED;
throw new IOException("Most of the disks failed. "
+ dirsHandler.getDisksHealthReport());
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
new file mode 100644
index 00000000000..cb9541fd7ce
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
@@ -0,0 +1,38 @@
+/**
+* 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.webapp;
+
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.InfoBlock;
+
+import com.google.inject.Inject;
+
+public class AppBlock extends HtmlBlock {
+
+ @Inject
+ AppBlock(ResourceManager rm, ViewContext ctx) {
+ super(ctx);
+ }
+
+ @Override
+ protected void render(Block html) {
+ html._(InfoBlock.class);
+ }
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
new file mode 100644
index 00000000000..a55c62f4b59
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
@@ -0,0 +1,32 @@
+/**
+* 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.webapp;
+
+import org.apache.hadoop.yarn.webapp.SubView;
+
+public class AppPage extends RmView {
+
+ @Override protected void preHead(Page.HTML<_> html) {
+ commonPreHead(html);
+ }
+
+ @Override protected Class extends SubView> content() {
+ return AppBlock.class;
+ }
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
index 700fdb3e3d2..49410e500a8 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
@@ -125,7 +125,7 @@ public class RmController extends Controller {
} else {
info._("AM container logs:", "AM not yet registered with RM");
}
- render(AboutPage.class);
+ render(AppPage.class);
}
public void nodes() {
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm
index 079c54b48b0..98e19b2c513 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm
@@ -403,9 +403,10 @@ Hadoop MapReduce Next Generation - Cluster Setup
the health of the local disks (specifically checks nodemanager-local-dirs
and nodemanager-log-dirs) and after reaching the threshold of number of
bad directories based on the value set for the config property
- yarn.nodemanager.disk-health-checker.min-healthy-disks. The boot disk is
- either raided or a failure in the boot disk is identified by the health
- checker script.
+ yarn.nodemanager.disk-health-checker.min-healthy-disks, the whole node is
+ marked unhealthy and this info is sent to resource manager also. The boot
+ disk is either raided or a failure in the boot disk is identified by the
+ health checker script.
* {Slaves file}
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index 5ccf9f2c296..b54af34be9f 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -349,7 +349,7 @@
UNTAR='tar xfBp -'
LIB_DIR="${BUILD_DIR}/native/target/usr/local/lib"
if [ -d $${LIB_DIR} ] ; then
- TARGET_DIR="${BUILD_DIR}/${project.artifactId}-${project.version}/lib"
+ TARGET_DIR="${BUILD_DIR}/${project.artifactId}-${project.version}/lib/native"
mkdir -p $${TARGET_DIR}
cd $${LIB_DIR}
$$TAR lib* | (cd $${TARGET_DIR}/; $$UNTAR)