diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh index 1ad507a326..4b2d115580 100755 --- a/dev-support/test-patch.sh +++ b/dev-support/test-patch.sh @@ -396,7 +396,7 @@ checkJavadocWarnings () { echo "There appear to be $javadocWarnings javadoc warnings generated by the patched build." ### if current warnings greater than OK_JAVADOC_WARNINGS - if [[ $javadocWarnings > $OK_JAVADOC_WARNINGS ]] ; then + if [[ $javadocWarnings -gt $OK_JAVADOC_WARNINGS ]] ; then JIRA_COMMENT="$JIRA_COMMENT -1 javadoc. The javadoc tool appears to have generated `expr $(($javadocWarnings-$OK_JAVADOC_WARNINGS))` warning messages." @@ -573,7 +573,7 @@ $JIRA_COMMENT_FOOTER" done ### if current warnings greater than OK_FINDBUGS_WARNINGS - if [[ $findbugsWarnings > $OK_FINDBUGS_WARNINGS ]] ; then + if [[ $findbugsWarnings -gt $OK_FINDBUGS_WARNINGS ]] ; then JIRA_COMMENT="$JIRA_COMMENT -1 findbugs. The patch appears to introduce `expr $(($findbugsWarnings-$OK_FINDBUGS_WARNINGS))` new Findbugs (version ${findbugs_version}) warnings." diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index be5490c4a8..50d2673d28 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -55,6 +55,11 @@ Trunk (unreleased changes) HADOOP-7792. Add verifyToken method to AbstractDelegationTokenSecretManager. (jitendra) + HADOOP-7776 Make the Ipc-Header in a RPC-Payload an explicit header (sanjay) + + HADOOP-7688. Add servlet handler check in HttpServer.start(). + (Uma Maheswara Rao G via szetszwo) + BUGS HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required @@ -95,6 +100,20 @@ Trunk (unreleased changes) HADOOP-7773. Add support for protocol buffer based RPC engine. (suresh) +Release 0.23.1 - Unreleased + + INCOMPATIBLE CHANGES + + NEW FEATURES + + IMPROVEMENTS + + HADOOP-7801. HADOOP_PREFIX cannot be overriden. (Bruno Mahé via tomwhite) + + OPTIMIZATIONS + + BUG FIXES + Release 0.23.0 - 2011-11-01 INCOMPATIBLE CHANGES @@ -798,6 +817,12 @@ Release 0.23.0 - 2011-11-01 HADOOP-7778. FindBugs warning in Token.getKind(). (tomwhite) + HADOOP-7798. Add support gpg signatures for maven release artifacts. + (cutting via acmurthy) + + HADOOP-7797. Fix top-level pom.xml to refer to correct staging maven + repository. (omalley via acmurthy) + Release 0.22.0 - Unreleased INCOMPATIBLE CHANGES 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 5f248fdf53..8bf9aecef4 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 @@ -27,7 +27,9 @@ this="$common_bin/$script" # the root of the Hadoop installation # See HADOOP-6255 for directory structure layout -export HADOOP_PREFIX=`dirname "$this"`/.. +HADOOP_DEFAULT_PREFIX=`dirname "$this"`/.. +HADOOP_PREFIX=${HADOOP_PREFIX:-$HADOOP_DEFAULT_PREFIX} +export HADOOP_PREFIX #check to see if the conf dir is given as an optional argument if [ $# -gt 1 ] diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java index ccc72edacf..e529e789ed 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java @@ -707,6 +707,14 @@ public void start() throws IOException { listener.setPort((oriPort += 1)); } } + // Make sure there is no handler failures. + Handler[] handlers = webServer.getHandlers(); + for (int i = 0; i < handlers.length; i++) { + if (handlers[i].isFailed()) { + throw new IOException( + "Problem in starting http server. Server handlers failed"); + } + } } catch (IOException e) { throw e; } catch (InterruptedException e) { 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 58cf810186..55e8a23d48 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 @@ -48,6 +48,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.RpcPayloadHeader.*; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; @@ -152,16 +153,20 @@ synchronized boolean isZeroReference() { return refCount==0; } - /** A call waiting for a value. */ + /** + * Class that represents an RPC call + */ private class Call { - int id; // call id - Writable param; // parameter - Writable value; // value, null if error - IOException error; // exception, null if value - boolean done; // true when call is done + final int id; // call id + final Writable rpcRequest; // the serialized rpc request - RpcPayload + Writable rpcResponse; // null if rpc has error + IOException error; // exception, null if success + final RpcKind rpcKind; // Rpc EngineKind + boolean done; // true when call is done - protected Call(Writable param) { - this.param = param; + protected Call(RpcKind rpcKind, Writable param) { + this.rpcKind = rpcKind; + this.rpcRequest = param; synchronized (Client.this) { this.id = counter++; } @@ -187,15 +192,15 @@ public synchronized void setException(IOException error) { /** Set the return value when there is no error. * Notify the caller the call is done. * - * @param value return value of the call. + * @param rpcResponse return value of the rpc call. */ - public synchronized void setValue(Writable value) { - this.value = value; + public synchronized void setRpcResponse(Writable rpcResponse) { + this.rpcResponse = rpcResponse; callComplete(); } - public synchronized Writable getValue() { - return value; + public synchronized Writable getRpcResult() { + return rpcResponse; } } @@ -727,6 +732,7 @@ private synchronized boolean waitForWork() { } } + @SuppressWarnings("unused") public InetSocketAddress getRemoteAddress() { return server; } @@ -787,8 +793,10 @@ public void sendParam(Call call) { //for serializing the //data to be written d = new DataOutputBuffer(); - d.writeInt(call.id); - call.param.write(d); + RpcPayloadHeader header = new RpcPayloadHeader( + call.rpcKind, RpcPayloadOperation.RPC_FINAL_PAYLOAD, call.id); + header.write(d); + call.rpcRequest.write(d); byte[] data = d.getData(); int dataLength = d.getLength(); out.writeInt(dataLength); //first put the data length @@ -825,7 +833,7 @@ private void receiveResponse() { if (state == Status.SUCCESS.state) { Writable value = ReflectionUtils.newInstance(valueClass, conf); value.readFields(in); // read value - call.setValue(value); + call.setRpcResponse(value); calls.remove(id); } else if (state == Status.ERROR.state) { call.setException(new RemoteException(WritableUtils.readString(in), @@ -909,7 +917,7 @@ private class ParallelCall extends Call { private int index; public ParallelCall(Writable param, ParallelResults results, int index) { - super(param); + super(RpcKind.RPC_WRITABLE, param); this.results = results; this.index = index; } @@ -933,7 +941,7 @@ public ParallelResults(int size) { /** Collect a result. */ public synchronized void callComplete(ParallelCall call) { - values[call.index] = call.getValue(); // store the value + values[call.index] = call.getRpcResult(); // store the value count++; // count it if (count == size) // if all values are in notify(); // then notify waiting caller @@ -993,15 +1001,23 @@ public void stop() { } } + /** + * Same as {@link #call(RpcKind, Writable, ConnectionId)} for Writable + */ + public Writable call(Writable param, InetSocketAddress address) + throws InterruptedException, IOException { + return call(RpcKind.RPC_WRITABLE, 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(Writable, ConnectionId)} instead + * @deprecated Use {@link #call(RpcKind, Writable, ConnectionId)} instead */ @Deprecated - public Writable call(Writable param, InetSocketAddress address) + public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress address) throws InterruptedException, IOException { - return call(param, address, null); + return call(rpcKind, param, address, null); } /** Make a call, passing param, to the IPC server running at @@ -1009,15 +1025,15 @@ public Writable call(Writable param, InetSocketAddress address) * the value. * Throws exceptions if there are network problems or if the remote code * threw an exception. - * @deprecated Use {@link #call(Writable, ConnectionId)} instead + * @deprecated Use {@link #call(RpcKind, Writable, ConnectionId)} instead */ @Deprecated - public Writable call(Writable param, InetSocketAddress addr, + public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, UserGroupInformation ticket) throws InterruptedException, IOException { ConnectionId remoteId = ConnectionId.getConnectionId(addr, null, ticket, 0, conf); - return call(param, remoteId); + return call(rpcKind, param, remoteId); } /** Make a call, passing param, to the IPC server running at @@ -1026,18 +1042,33 @@ public Writable call(Writable param, InetSocketAddress addr, * timeout, returning the value. * Throws exceptions if there are network problems or if the remote code * threw an exception. - * @deprecated Use {@link #call(Writable, ConnectionId)} instead + * @deprecated Use {@link #call(RpcKind, Writable, ConnectionId)} instead */ @Deprecated - public Writable call(Writable param, InetSocketAddress addr, + public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, Class protocol, UserGroupInformation ticket, int rpcTimeout) throws InterruptedException, IOException { ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol, ticket, rpcTimeout, conf); - return call(param, remoteId); + return call(rpcKind, param, remoteId); } + + /** + * Same as {@link #call(RpcKind, Writable, InetSocketAddress, + * Class, UserGroupInformation, int, Configuration)} + * except that rpcKind is writable. + */ + public Writable call(Writable param, InetSocketAddress addr, + Class protocol, UserGroupInformation ticket, + int rpcTimeout, Configuration conf) + throws InterruptedException, IOException { + ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol, + ticket, rpcTimeout, conf); + return call(RpcKind.RPC_WRITABLE, param, remoteId); + } + /** * Make a call, passing param, to the IPC server running at * address which is servicing the protocol protocol, @@ -1046,22 +1077,31 @@ public Writable call(Writable param, InetSocketAddress addr, * value. Throws exceptions if there are network problems or if the remote * code threw an exception. */ - public Writable call(Writable param, InetSocketAddress addr, + public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, Class protocol, UserGroupInformation ticket, int rpcTimeout, Configuration conf) throws InterruptedException, IOException { ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol, ticket, rpcTimeout, conf); - return call(param, remoteId); + return call(rpcKind, param, remoteId); + } + + /** + * Same as {link {@link #call(RpcKind, Writable, ConnectionId)} + * except the rpcKind is RPC_WRITABLE + */ + public Writable call(Writable param, ConnectionId remoteId) + throws InterruptedException, IOException { + return call(RpcKind.RPC_WRITABLE, param, remoteId); } /** Make a call, passing param, to the IPC server defined by * remoteId, returning the value. * Throws exceptions if there are network problems or if the remote code * threw an exception. */ - public Writable call(Writable param, ConnectionId remoteId) + public Writable call(RpcKind rpcKind, Writable param, ConnectionId remoteId) throws InterruptedException, IOException { - Call call = new Call(param); + Call call = new Call(rpcKind, param); Connection connection = getConnection(remoteId, call); connection.sendParam(call); // send the parameter boolean interrupted = false; @@ -1093,7 +1133,7 @@ public Writable call(Writable param, ConnectionId remoteId) call.error); } } else { - return call.value; + return call.rpcResponse; } } } 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 aec56a9d57..dad94227ca 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.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; +import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind; import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcExceptionProto; import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcRequestProto; import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcResponseProto; @@ -139,7 +140,7 @@ public Object invoke(Object proxy, Method method, Object[] args) HadoopRpcRequestProto rpcRequest = constructRpcRequest(method, args); RpcResponseWritable val = null; try { - val = (RpcResponseWritable) client.call( + val = (RpcResponseWritable) client.call(RpcKind.RPC_PROTOCOL_BUFFER, new RpcRequestWritable(rpcRequest), remoteId); } catch (Exception e) { RpcClientException ce = new RpcClientException("Client exception", e); 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 new file mode 100644 index 0000000000..1b62f0caa7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java @@ -0,0 +1,118 @@ +package org.apache.hadoop.ipc; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.Writable; + +/** + * This is the rpc payload header. It is sent with every rpc call + *
+ * The format of RPC call is as follows:
+ * +---------------------------------------------------+
+ * |  Rpc length in bytes (header + payload length)    |
+ * +---------------------------------------------------+
+ * |      Rpc Header       |       Rpc Payload         |
+ * +---------------------------------------------------+
+ * 
+ * The format of Rpc Header is:
+ * +----------------------------------+
+ * |  RpcKind (1 bytes)               |      
+ * +----------------------------------+
+ * |  RpcPayloadOperation (1 bytes)   |      
+ * +----------------------------------+
+ * |  Call ID (4 bytes)               |      
+ * +----------------------------------+
+ * 
+ * {@link RpcKind} determines the type of serialization used for Rpc Payload.
+ * 
+ *

+ * Note this header does NOT have its own version number, + * it used the version number from the connection header. + */ +public class RpcPayloadHeader implements Writable { + public enum RpcPayloadOperation { + RPC_FINAL_PAYLOAD ((short)1), + RPC_CONTINUATION_PAYLOAD ((short)2), // not implemented yet + RPC_CLOSE_CONNECTION ((short)3); // close the rpc connection + + private final short code; + private static final short FIRST_INDEX = RPC_FINAL_PAYLOAD.code; + RpcPayloadOperation(short val) { + this.code = val; + } + + public void write(DataOutput out) throws IOException { + out.writeByte(code); + } + + static RpcPayloadOperation readFields(DataInput in) throws IOException { + short inValue = in.readByte(); + return RpcPayloadOperation.values()[inValue - FIRST_INDEX]; + } + } + + 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); + + private final short value; + private static final short FIRST_INDEX = RPC_BUILTIN.value; + RpcKind(short val) { + this.value = val; + } + + public void write(DataOutput out) throws IOException { + out.writeByte(value); + } + + static RpcKind readFields(DataInput in) throws IOException { + short inValue = in.readByte(); + return RpcKind.values()[inValue - FIRST_INDEX]; + } + } + + private RpcKind kind; + private RpcPayloadOperation operation; + private int callId; + + public RpcPayloadHeader() { + kind = RpcKind.RPC_WRITABLE; + operation = RpcPayloadOperation.RPC_CLOSE_CONNECTION; + } + + public RpcPayloadHeader(RpcKind kind, RpcPayloadOperation op, int callId) { + this.kind = kind; + this.operation = op; + this.callId = callId; + } + + int getCallId() { + return callId; + } + + RpcKind getkind() { + return kind; + } + + RpcPayloadOperation getOperation() { + return operation; + } + + @Override + public void write(DataOutput out) throws IOException { + kind.write(out); + operation.write(out); + out.writeInt(callId); + } + + @Override + public void readFields(DataInput in) throws IOException { + kind = RpcKind.readFields(in); + operation = RpcPayloadOperation.readFields(in); + this.callId = in.readInt(); + } +} 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 78c0bc76a1..c04833f232 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 @@ -61,11 +61,13 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.ipc.RpcPayloadHeader.*; 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.VersionMismatch; +import org.apache.hadoop.ipc.RpcPayloadHeader.RpcPayloadOperation; import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics; import org.apache.hadoop.ipc.metrics.RpcMetrics; import org.apache.hadoop.net.NetUtils; @@ -124,7 +126,8 @@ public abstract class Server { // 4 : Introduced SASL security layer // 5 : Introduced use of {@link ArrayPrimitiveWritable$Internal} // in ObjectWritable to efficiently transmit arrays of primitives - public static final byte CURRENT_VERSION = 5; + // 6 : Made RPC payload header explicit + public static final byte CURRENT_VERSION = 6; /** * Initial and max size of response buffer @@ -277,28 +280,33 @@ public ServiceAuthorizationManager getServiceAuthorizationManager() { /** A call queued for handling. */ private static class Call { - private int id; // the client's call id - private Writable param; // the parameter passed - private Connection connection; // connection to client - private long timestamp; // the time received when response is null - // the time served when response is not null - private ByteBuffer response; // the response for this call + private final int callId; // the client's call id + private final Writable rpcRequest; // Serialized Rpc request from client + private final Connection connection; // connection to client + private long timestamp; // time received when response is null + // time served when response is not null + private ByteBuffer rpcResponse; // the response for this call + private final RpcKind rpcKind; - public Call(int id, Writable param, Connection connection) { - this.id = id; - this.param = param; + public Call(int id, Writable param, Connection connection) { + this( id, param, connection, RpcKind.RPC_BUILTIN ); + } + public Call(int id, Writable param, Connection connection, RpcKind kind) { + this.callId = id; + this.rpcRequest = param; this.connection = connection; this.timestamp = System.currentTimeMillis(); - this.response = null; + this.rpcResponse = null; + this.rpcKind = kind; } @Override public String toString() { - return param.toString() + " from " + connection.toString(); + return rpcRequest.toString() + " from " + connection.toString(); } public void setResponse(ByteBuffer response) { - this.response = response; + this.rpcResponse = response; } } @@ -795,17 +803,17 @@ private boolean processResponse(LinkedList responseQueue, call = responseQueue.removeFirst(); SocketChannel channel = call.connection.channel; if (LOG.isDebugEnabled()) { - LOG.debug(getName() + ": responding to #" + call.id + " from " + + LOG.debug(getName() + ": responding to #" + call.callId + " from " + call.connection); } // // Send as much data as we can in the non-blocking fashion // - int numBytes = channelWrite(channel, call.response); + int numBytes = channelWrite(channel, call.rpcResponse); if (numBytes < 0) { return true; } - if (!call.response.hasRemaining()) { + if (!call.rpcResponse.hasRemaining()) { call.connection.decRpcCount(); if (numElements == 1) { // last call fully processes. done = true; // no more data for this channel. @@ -813,7 +821,7 @@ private boolean processResponse(LinkedList responseQueue, done = false; // more calls pending to be sent. } if (LOG.isDebugEnabled()) { - LOG.debug(getName() + ": responding to #" + call.id + " from " + + LOG.debug(getName() + ": responding to #" + call.callId + " from " + call.connection + " Wrote " + numBytes + " bytes."); } } else { @@ -841,7 +849,7 @@ private boolean processResponse(LinkedList responseQueue, } } if (LOG.isDebugEnabled()) { - LOG.debug(getName() + ": responding to #" + call.id + " from " + + LOG.debug(getName() + ": responding to #" + call.callId + " from " + call.connection + " Wrote partial " + numBytes + " bytes."); } @@ -1408,18 +1416,24 @@ private void processOneRpc(byte[] buf) throws IOException, private void processData(byte[] buf) throws IOException, InterruptedException { DataInputStream dis = new DataInputStream(new ByteArrayInputStream(buf)); - int id = dis.readInt(); // try to read an id + RpcPayloadHeader header = new RpcPayloadHeader(); + header.readFields(dis); // Read the RpcPayload header if (LOG.isDebugEnabled()) - LOG.debug(" got #" + id); - Writable param; - try { - param = ReflectionUtils.newInstance(paramClass, conf);//read param - param.readFields(dis); + LOG.debug(" got #" + header.getCallId()); + if (header.getOperation() != RpcPayloadOperation.RPC_FINAL_PAYLOAD) { + throw new IOException("IPC Server does not implement operation" + + header.getOperation()); + } + Writable rpcRequest; + try { //Read the rpc request + rpcRequest = ReflectionUtils.newInstance(paramClass, conf); + rpcRequest.readFields(dis); } catch (Throwable t) { LOG.warn("Unable to read call parameters for client " + getHostAddress(), t); - final Call readParamsFailedCall = new Call(id, null, this); + final Call readParamsFailedCall = + new Call(header.getCallId(), null, this); ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream(); setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null, @@ -1429,7 +1443,7 @@ private void processData(byte[] buf) throws IOException, InterruptedException { return; } - Call call = new Call(id, param, this); + Call call = new Call(header.getCallId(), rpcRequest, this, header.getkind()); callQueue.put(call); // queue the call; maybe blocked here incRpcCount(); // Increment the rpc count } @@ -1493,8 +1507,8 @@ public void run() { final Call call = callQueue.take(); // pop the queue; maybe blocked here if (LOG.isDebugEnabled()) - LOG.debug(getName() + ": has #" + call.id + " from " + - call.connection); + LOG.debug(getName() + ": has Call#" + call.callId + + "for RpcKind " + call.rpcKind + " from " + call.connection); String errorClass = null; String error = null; @@ -1505,7 +1519,7 @@ public void run() { // 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.param, + value = call(call.connection.protocolName, call.rpcRequest, call.timestamp); } else { value = @@ -1515,7 +1529,7 @@ public void run() { public Writable run() throws Exception { // make the call return call(call.connection.protocolName, - call.param, call.timestamp); + call.rpcRequest, call.timestamp); } } @@ -1657,7 +1671,7 @@ private void setupResponse(ByteArrayOutputStream response, throws IOException { response.reset(); DataOutputStream out = new DataOutputStream(response); - out.writeInt(call.id); // write call id + out.writeInt(call.callId); // write call id out.writeInt(status.state); // write status if (status == Status.SUCCESS) { 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 314bfac582..48842237d4 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 @@ -39,6 +39,7 @@ import org.apache.commons.logging.*; import org.apache.hadoop.io.*; +import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind; import org.apache.hadoop.ipc.VersionedProtocol; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager; @@ -242,7 +243,7 @@ public Object invoke(Object proxy, Method method, Object[] args) } ObjectWritable value = (ObjectWritable) - client.call(new Invocation(method, args), remoteId); + client.call(RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId); if (LOG.isDebugEnabled()) { long callTime = System.currentTimeMillis() - startTime; LOG.debug("Call: " + method.getName() + " " + callTime); diff --git a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-datanode b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-datanode index 81795879cf..658f6d7574 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-datanode +++ b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-datanode @@ -75,7 +75,7 @@ check_privsep_dir() { } export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" -export HADOOP_PREFIX="/usr" +export HADOOP_PREFIX=${HADOOP_PREFIX:-/usr} case "$1" in start) diff --git a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-jobtracker b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-jobtracker index c483d76a17..8958da35de 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-jobtracker +++ b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-jobtracker @@ -67,7 +67,7 @@ check_privsep_dir() { } export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" -export HADOOP_PREFIX="/usr" +export HADOOP_PREFIX=${HADOOP_PREFIX:-/usr} case "$1" in start) diff --git a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-namenode b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-namenode index e2608769d4..6e5485cc8a 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-namenode +++ b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-namenode @@ -71,7 +71,7 @@ format() { } export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" -export HADOOP_PREFIX="/usr" +export HADOOP_PREFIX=${HADOOP_PREFIX:-/usr} case "$1" in start) diff --git a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-tasktracker b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-tasktracker index 983030bbb6..ca36cd6f45 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-tasktracker +++ b/hadoop-common-project/hadoop-common/src/main/packages/deb/init.d/hadoop-tasktracker @@ -67,7 +67,7 @@ check_privsep_dir() { } export PATH="${PATH:+$PATH:}/usr/sbin:/usr/bin" -export HADOOP_PREFIX="/usr" +export HADOOP_PREFIX=${HADOOP_PREFIX:-/usr} case "$1" in start) diff --git a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh index db1f5b9e45..de47ce2716 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh +++ b/hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh @@ -19,7 +19,9 @@ bin=`dirname "$0"` bin=`cd "$bin"; pwd` this="${BASH_SOURCE-$0}" -export HADOOP_PREFIX=`dirname "$this"`/.. +HADOOP_DEFAULT_PREFIX=`dirname "$this"`/.. +HADOOP_PREFIX=${HADOOP_PREFIX:-$HADOOP_DEFAULT_PREFIX} +export HADOOP_PREFIX usage() { echo " diff --git a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-jobtracker b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-jobtracker index 5c2272b8fd..9e85c66ee9 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-jobtracker +++ b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-jobtracker @@ -27,7 +27,7 @@ source /etc/default/hadoop-env.sh RETVAL=0 PIDFILE="${HADOOP_PID_DIR}/hadoop-mapred-jobtracker.pid" desc="Hadoop jobtracker daemon" -export HADOOP_PREFIX="/usr" +export HADOOP_PREFIX=${HADOOP_PREFIX:-/usr} start() { echo -n $"Starting $desc (hadoop-jobtracker): " diff --git a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-namenode b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-namenode index 4e7f74b046..3067f03ef5 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-namenode +++ b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-namenode @@ -27,7 +27,7 @@ source /etc/default/hadoop-env.sh RETVAL=0 PIDFILE="${HADOOP_PID_DIR}/hadoop-hdfs-namenode.pid" desc="Hadoop namenode daemon" -export HADOOP_PREFIX="/usr" +export HADOOP_PREFIX=${HADOOP_PREFIX:-/usr} start() { echo -n $"Starting $desc (hadoop-namenode): " diff --git a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-tasktracker b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-tasktracker index 2d06c5a9c9..9a57f56275 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-tasktracker +++ b/hadoop-common-project/hadoop-common/src/main/packages/rpm/init.d/hadoop-tasktracker @@ -27,7 +27,7 @@ source /etc/default/hadoop-env.sh RETVAL=0 PIDFILE="${HADOOP_PID_DIR}/hadoop-mapred-tasktracker.pid" desc="Hadoop tasktracker daemon" -export HADOOP_PREFIX="/usr" +export HADOOP_PREFIX=${HADOOP_PREFIX:-/usr} start() { echo -n $"Starting $desc (hadoop-tasktracker): " diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java index 6cd21beb1b..7bf608767e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java @@ -45,7 +45,7 @@ public class TestServletFilter extends HttpServerFunctionalTest { static public class SimpleFilter implements Filter { private FilterConfig filterConfig = null; - public void init(FilterConfig filterConfig) { + public void init(FilterConfig filterConfig) throws ServletException { this.filterConfig = filterConfig; } @@ -137,4 +137,36 @@ public void testServletFilter() throws Exception { http.stop(); } } + + static public class ErrorFilter extends SimpleFilter { + @Override + public void init(FilterConfig arg0) throws ServletException { + throw new ServletException("Throwing the exception from Filter init"); + } + + /** Configuration for the filter */ + static public class Initializer extends FilterInitializer { + public Initializer() { + } + + public void initFilter(FilterContainer container, Configuration conf) { + container.addFilter("simple", ErrorFilter.class.getName(), null); + } + } + } + + @Test + public void testServletFilterWhenInitThrowsException() throws Exception { + Configuration conf = new Configuration(); + // start a http server with CountingFilter + conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY, + ErrorFilter.Initializer.class.getName()); + HttpServer http = createTestServer(conf); + try { + http.start(); + fail("expecting exception"); + } catch (IOException e) { + assertTrue( e.getMessage().contains("Problem in starting http server. Server handlers failed")); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 2a7f753295..f9ed841355 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -52,6 +52,18 @@ Trunk (unreleased changes) HDFS-2334. Add Closeable to JournalManager. (Ivan Kelly via jitendra) + + OPTIMIZATIONS + HDFS-2477. Optimize computing the diff between a block report and the + namenode state. (Tomasz Nykiel via hairong) + + HDFS-2495. Increase granularity of write operations in ReplicationMonitor + thus reducing contention for write lock. (Tomasz Nykiel via hairong) + + HDFS-2476. More CPU efficient data structure for under-replicated, + over-replicated, and invalidated blocks. + (Tomasz Nykiel via todd) + BUG FIXES HDFS-2287. TestParallelRead has a small off-by-one bug. (todd) @@ -88,7 +100,25 @@ Trunk (unreleased changes) HDFS-2526. (Client)NamenodeProtocolTranslatorR23 do not need to keep a reference to rpcProxyWithoutRetry (atm) - HDFS-2416. distcp with a webhdfs uri on a secure cluster fails. (jitendra) +Release 0.23.1 - UNRELEASED + + INCOMPATIBLE CHANGES + + NEW FEATURES + + IMPROVEMENTS + + OPTIMIZATIONS + + HDFS-2130. Switch default checksum to CRC32C. (todd) + + HDFS-2533. Remove needless synchronization on some FSDataSet methods. + (todd) + + HDFS-2129. Simplify BlockReader to not inherit from FSInputChecker. + (todd) + + BUG FIXES Release 0.23.0 - 2011-11-01 @@ -416,6 +446,9 @@ Release 0.23.0 - 2011-11-01 HDFS-2385. Support renew and cancel delegation tokens in webhdfs. (szetszwo) + HDFS-2539. Support doAs and GETHOMEDIRECTORY in webhdfs. + (szetszwo) + IMPROVEMENTS HDFS-1875. MiniDFSCluster hard-codes dfs.datanode.address to localhost @@ -1246,6 +1279,22 @@ Release 0.23.0 - 2011-11-01 HDFS-2065. Add null checks in DFSClient.getFileChecksum(..). (Uma Maheswara Rao G via szetszwo) + HDFS-2416. distcp with a webhdfs uri on a secure cluster fails. (jitendra) + + HDFS-2527. WebHdfs: remove the use of "Range" header in Open; use ugi + username if renewer parameter is null in GetDelegationToken; response OK + when setting replication for non-files; rename GETFILEBLOCKLOCATIONS to + GET_BLOCK_LOCATIONS and state that it is a private unstable API; replace + isDirectory and isSymlink with enum {FILE, DIRECTORY, SYMLINK} in + HdfsFileStatus JSON object. (szetszwo) + + HDFS-2528. Webhdfs: set delegation kind to WEBHDFS and add a HDFS token + when http requests are redirected to datanode. (szetszwo) + + HDFS-2540. Webhdfs: change "Expect: 100-continue" to two-step write; change + "HdfsFileStatus" and "localName" respectively to "FileStatus" and + "pathSuffix" in JSON response. (szetszwo) + BREAKDOWN OF HDFS-1073 SUBTASKS HDFS-1521. Persist transaction ID on disk between NN restarts. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java index 3ebbeec68b..dfab7309b5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java @@ -20,14 +20,11 @@ import java.io.IOException; import java.net.Socket; -import org.apache.hadoop.fs.PositionedReadable; -import org.apache.hadoop.fs.Seekable; - /** * A BlockReader is responsible for reading a single block * from a single datanode. */ -public interface BlockReader extends Seekable, PositionedReadable { +public interface BlockReader { /* same interface as inputStream java.io.InputStream#read() * used by DFSInputStream#read() @@ -43,16 +40,21 @@ public interface BlockReader extends Seekable, PositionedReadable { */ long skip(long n) throws IOException; - /** - * Read a single byte, returning -1 at enf of stream. - */ - int read() throws IOException; - void close() throws IOException; /** - * kind of like readFully(). Only reads as much as possible. - * And allows use of protected readFully(). + * Read exactly the given amount of data, throwing an exception + * if EOF is reached before that amount + */ + void readFully(byte[] buf, int readOffset, int amtToRead) throws IOException; + + /** + * Similar to {@link #readFully(byte[], int, int)} except that it will + * not throw an exception on EOF. However, it differs from the simple + * {@link #read(byte[], int, int)} call in that it is guaranteed to + * read the data if it is available. In other words, if this call + * does not throw an exception, then either the buffer has been + * filled or the next call will return EOF. */ int readAll(byte[] buf, int offset, int len) throws IOException; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java index 52c6cc42ce..855d7ece79 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java @@ -22,6 +22,8 @@ import java.net.Socket; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSClient.Conf; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.security.token.Token; @@ -32,17 +34,26 @@ */ @InterfaceAudience.Private public class BlockReaderFactory { - public static BlockReader newBlockReader(Socket sock, String file, + /** + * @see #newBlockReader(Conf, Socket, String, ExtendedBlock, Token, long, long, int, boolean, String) + */ + public static BlockReader newBlockReader( + Configuration conf, + Socket sock, String file, ExtendedBlock block, Token blockToken, - long startOffset, long len, int bufferSize) throws IOException { - return newBlockReader(sock, file, block, blockToken, startOffset, + long startOffset, long len) throws IOException { + int bufferSize = conf.getInt(DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY, + DFSConfigKeys.IO_FILE_BUFFER_SIZE_DEFAULT); + return newBlockReader(new Conf(conf), + sock, file, block, blockToken, startOffset, len, bufferSize, true, ""); } /** * Create a new BlockReader specifically to satisfy a read. * This method also sends the OP_READ_BLOCK request. - * + * + * @param conf the DFSClient configuration * @param sock An established Socket to the DN. The BlockReader will not close it normally * @param file File location * @param block The block object @@ -54,7 +65,9 @@ public static BlockReader newBlockReader(Socket sock, String file, * @param clientName Client name * @return New BlockReader instance, or null on error. */ + @SuppressWarnings("deprecation") public static BlockReader newBlockReader( + Conf conf, Socket sock, String file, ExtendedBlock block, Token blockToken, @@ -62,8 +75,13 @@ public static BlockReader newBlockReader( int bufferSize, boolean verifyChecksum, String clientName) throws IOException { - return RemoteBlockReader.newBlockReader( - sock, file, block, blockToken, startOffset, len, bufferSize, verifyChecksum, clientName); + if (conf.useLegacyBlockReader) { + return RemoteBlockReader.newBlockReader( + sock, file, block, blockToken, startOffset, len, bufferSize, verifyChecksum, clientName); + } else { + return RemoteBlockReader2.newBlockReader( + sock, file, block, blockToken, startOffset, len, bufferSize, verifyChecksum, clientName); + } } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java index 80da377f34..36c7c1dec0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java @@ -18,17 +18,13 @@ package org.apache.hadoop.hdfs; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.net.HttpURLConnection; -import java.net.MalformedURLException; import java.net.URL; -import java.util.StringTokenizer; import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.hdfs.server.namenode.StreamFile; -import org.apache.hadoop.hdfs.web.resources.OffsetParam; /** * To support HTTP byte streams, a new connection to an HTTP server needs to be @@ -37,16 +33,14 @@ * is made on the successive read(). The normal input stream functions are * connected to the currently active input stream. */ -public class ByteRangeInputStream extends FSInputStream { +public abstract class ByteRangeInputStream extends FSInputStream { /** * This class wraps a URL and provides method to open connection. * It can be overridden to change how a connection is opened. */ - public static class URLOpener { + public static abstract class URLOpener { protected URL url; - /** The url with offset parameter */ - protected URL offsetUrl; public URLOpener(URL u) { url = u; @@ -60,52 +54,9 @@ public URL getURL() { return url; } - protected HttpURLConnection openConnection() throws IOException { - return (HttpURLConnection)offsetUrl.openConnection(); - } + protected abstract HttpURLConnection openConnection() throws IOException; - private HttpURLConnection openConnection(final long offset) throws IOException { - offsetUrl = offset == 0L? url: new URL(url + "&" + new OffsetParam(offset)); - final HttpURLConnection conn = openConnection(); - conn.setRequestMethod("GET"); - if (offset != 0L) { - conn.setRequestProperty("Range", "bytes=" + offset + "-"); - } - return conn; - } - } - - static private final String OFFSET_PARAM_PREFIX = OffsetParam.NAME + "="; - - /** Remove offset parameter, if there is any, from the url */ - static URL removeOffsetParam(final URL url) throws MalformedURLException { - String query = url.getQuery(); - if (query == null) { - return url; - } - final String lower = query.toLowerCase(); - if (!lower.startsWith(OFFSET_PARAM_PREFIX) - && !lower.contains("&" + OFFSET_PARAM_PREFIX)) { - return url; - } - - //rebuild query - StringBuilder b = null; - for(final StringTokenizer st = new StringTokenizer(query, "&"); - st.hasMoreTokens();) { - final String token = st.nextToken(); - if (!token.toLowerCase().startsWith(OFFSET_PARAM_PREFIX)) { - if (b == null) { - b = new StringBuilder("?").append(token); - } else { - b.append('&').append(token); - } - } - } - query = b == null? "": b.toString(); - - final String urlStr = url.toString(); - return new URL(urlStr.substring(0, urlStr.indexOf('?')) + query); + protected abstract HttpURLConnection openConnection(final long offset) throws IOException; } enum StreamStatus { @@ -120,11 +71,6 @@ enum StreamStatus { StreamStatus status = StreamStatus.SEEK; - /** Create an input stream with the URL. */ - public ByteRangeInputStream(final URL url) { - this(new URLOpener(url), new URLOpener(null)); - } - /** * Create with the specified URLOpeners. Original url is used to open the * stream for the first time. Resolved url is used in subsequent requests. @@ -136,6 +82,12 @@ public ByteRangeInputStream(URLOpener o, URLOpener r) { this.resolvedURL = r; } + protected abstract void checkResponseCode(final HttpURLConnection connection + ) throws IOException; + + protected abstract URL getResolvedUrl(final HttpURLConnection connection + ) throws IOException; + private InputStream getInputStream() throws IOException { if (status != StreamStatus.NORMAL) { @@ -150,32 +102,14 @@ private InputStream getInputStream() throws IOException { (resolvedURL.getURL() == null) ? originalURL : resolvedURL; final HttpURLConnection connection = opener.openConnection(startPos); - try { - connection.connect(); - final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH); - filelength = (cl == null) ? -1 : Long.parseLong(cl); - if (HftpFileSystem.LOG.isDebugEnabled()) { - HftpFileSystem.LOG.debug("filelength = " + filelength); - } - in = connection.getInputStream(); - } catch (FileNotFoundException fnfe) { - throw fnfe; - } catch (IOException ioe) { - HftpFileSystem.throwIOExceptionFromConnection(connection, ioe); - } - - int respCode = connection.getResponseCode(); - if (startPos != 0 && respCode != HttpURLConnection.HTTP_PARTIAL) { - // We asked for a byte range but did not receive a partial content - // response... - throw new IOException("HTTP_PARTIAL expected, received " + respCode); - } else if (startPos == 0 && respCode != HttpURLConnection.HTTP_OK) { - // We asked for all bytes from the beginning but didn't receive a 200 - // response (none of the other 2xx codes are valid here) - throw new IOException("HTTP_OK expected, received " + respCode); - } + connection.connect(); + checkResponseCode(connection); - resolvedURL.setURL(removeOffsetParam(connection.getURL())); + final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH); + filelength = (cl == null) ? -1 : Long.parseLong(cl); + in = connection.getInputStream(); + + resolvedURL.setURL(getResolvedUrl(connection)); status = StreamStatus.NORMAL; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index f6dbd3a61a..c964346c74 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -100,6 +100,7 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenRenewer; +import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.ReflectionUtils; @@ -142,6 +143,7 @@ static class Conf { final int maxBlockAcquireFailures; final int confTime; final int ioBufferSize; + final int checksumType; final int bytesPerChecksum; final int writePacketSize; final int socketTimeout; @@ -156,6 +158,7 @@ static class Conf { final short defaultReplication; final String taskId; final FsPermission uMask; + final boolean useLegacyBlockReader; Conf(Configuration conf) { maxBlockAcquireFailures = conf.getInt( @@ -166,6 +169,7 @@ static class Conf { ioBufferSize = conf.getInt( CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY, CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT); + checksumType = getChecksumType(conf); bytesPerChecksum = conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY, DFS_BYTES_PER_CHECKSUM_DEFAULT); socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, @@ -192,6 +196,29 @@ static class Conf { .getInt(DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT); uMask = FsPermission.getUMask(conf); + useLegacyBlockReader = conf.getBoolean( + DFS_CLIENT_USE_LEGACY_BLOCKREADER, + DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT); + } + + private int getChecksumType(Configuration conf) { + String checksum = conf.get(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, + DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT); + if ("CRC32".equals(checksum)) { + return DataChecksum.CHECKSUM_CRC32; + } else if ("CRC32C".equals(checksum)) { + return DataChecksum.CHECKSUM_CRC32C; + } else if ("NULL".equals(checksum)) { + return DataChecksum.CHECKSUM_NULL; + } else { + LOG.warn("Bad checksum type: " + checksum + ". Using default."); + return DataChecksum.CHECKSUM_CRC32C; + } + } + + private DataChecksum createChecksum() { + return DataChecksum.newDataChecksum( + checksumType, bytesPerChecksum); } } @@ -813,7 +840,7 @@ public OutputStream create(String src, } final DFSOutputStream result = new DFSOutputStream(this, src, masked, flag, createParent, replication, blockSize, progress, buffersize, - dfsClientConf.bytesPerChecksum); + dfsClientConf.createChecksum()); leaserenewer.put(src, result, this); return result; } @@ -857,9 +884,12 @@ public OutputStream primitiveCreate(String src, CreateFlag.validate(flag); DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress); if (result == null) { + DataChecksum checksum = DataChecksum.newDataChecksum( + dfsClientConf.checksumType, + bytesPerChecksum); result = new DFSOutputStream(this, src, absPermission, flag, createParent, replication, blockSize, progress, buffersize, - bytesPerChecksum); + checksum); } leaserenewer.put(src, result, this); return result; @@ -917,7 +947,7 @@ private DFSOutputStream callAppend(HdfsFileStatus stat, String src, UnresolvedPathException.class); } return new DFSOutputStream(this, src, buffersize, progress, - lastBlock, stat, dfsClientConf.bytesPerChecksum); + lastBlock, stat, dfsClientConf.createChecksum()); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index b3a4407193..27a29775bd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -38,6 +38,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final int DFS_STREAM_BUFFER_SIZE_DEFAULT = 4096; public static final String DFS_BYTES_PER_CHECKSUM_KEY = "dfs.bytes-per-checksum"; public static final int DFS_BYTES_PER_CHECKSUM_DEFAULT = 512; + public static final String DFS_CHECKSUM_TYPE_KEY = "dfs.checksum.type"; + public static final String DFS_CHECKSUM_TYPE_DEFAULT = "CRC32C"; public static final String DFS_CLIENT_WRITE_PACKET_SIZE_KEY = "dfs.client-write-packet-size"; public static final int DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024; public static final String DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY = "dfs.client.block.write.replace-datanode-on-failure.enable"; @@ -180,6 +182,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final int DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT = 3; public static final String DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY = "dfs.client.max.block.acquire.failures"; public static final int DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT = 3; + public static final String DFS_CLIENT_USE_LEGACY_BLOCKREADER = "dfs.client.use.legacy.blockreader"; + public static final boolean DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT = false; public static final String DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth"; public static final long DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L; public static final String DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 2964ccb5a7..71ec00e20e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -780,7 +780,8 @@ protected BlockReader getBlockReader(InetSocketAddress dnAddr, try { // The OP_READ_BLOCK request is sent as we make the BlockReader BlockReader reader = - BlockReaderFactory.newBlockReader(sock, file, block, + BlockReaderFactory.newBlockReader(dfsClient.getConf(), + sock, file, block, blockToken, startOffset, len, bufferSize, verifyChecksum, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 174da6841d..8422ecca00 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -74,7 +74,6 @@ import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Progressable; -import org.apache.hadoop.util.PureJavaCrc32; /**************************************************************** @@ -1206,8 +1205,9 @@ synchronized DatanodeInfo[] getPipeline() { } private DFSOutputStream(DFSClient dfsClient, String src, long blockSize, Progressable progress, - int bytesPerChecksum, short replication) throws IOException { - super(new PureJavaCrc32(), bytesPerChecksum, 4); + DataChecksum checksum, short replication) throws IOException { + super(checksum, checksum.getBytesPerChecksum(), checksum.getChecksumSize()); + int bytesPerChecksum = checksum.getBytesPerChecksum(); this.dfsClient = dfsClient; this.src = src; this.blockSize = blockSize; @@ -1225,8 +1225,7 @@ private DFSOutputStream(DFSClient dfsClient, String src, long blockSize, Progres "multiple of io.bytes.per.checksum"); } - checksum = DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_CRC32, - bytesPerChecksum); + this.checksum = checksum; } /** @@ -1235,11 +1234,12 @@ private DFSOutputStream(DFSClient dfsClient, String src, long blockSize, Progres */ DFSOutputStream(DFSClient dfsClient, String src, FsPermission masked, EnumSet flag, boolean createParent, short replication, long blockSize, Progressable progress, - int buffersize, int bytesPerChecksum) + int buffersize, DataChecksum checksum) throws IOException { - this(dfsClient, src, blockSize, progress, bytesPerChecksum, replication); + this(dfsClient, src, blockSize, progress, checksum, replication); - computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum); + computePacketChunkSize(dfsClient.getConf().writePacketSize, + checksum.getBytesPerChecksum()); try { dfsClient.namenode.create( @@ -1264,8 +1264,8 @@ private DFSOutputStream(DFSClient dfsClient, String src, long blockSize, Progres */ DFSOutputStream(DFSClient dfsClient, String src, int buffersize, Progressable progress, LocatedBlock lastBlock, HdfsFileStatus stat, - int bytesPerChecksum) throws IOException { - this(dfsClient, src, stat.getBlockSize(), progress, bytesPerChecksum, stat.getReplication()); + DataChecksum checksum) throws IOException { + this(dfsClient, src, stat.getBlockSize(), progress, checksum, stat.getReplication()); initialFileSize = stat.getLen(); // length of file when opened // @@ -1274,9 +1274,10 @@ private DFSOutputStream(DFSClient dfsClient, String src, long blockSize, Progres if (lastBlock != null) { // indicate that we are appending to an existing block bytesCurBlock = lastBlock.getBlockSize(); - streamer = new DataStreamer(lastBlock, stat, bytesPerChecksum); + streamer = new DataStreamer(lastBlock, stat, checksum.getBytesPerChecksum()); } else { - computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum); + computePacketChunkSize(dfsClient.getConf().writePacketSize, + checksum.getBytesPerChecksum()); streamer = new DataStreamer(); } streamer.start(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java index 1ef00793de..7e86b9e851 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java @@ -372,13 +372,66 @@ protected String addDelegationTokenParam(String query) throws IOException { return query; } + static class RangeHeaderUrlOpener extends ByteRangeInputStream.URLOpener { + RangeHeaderUrlOpener(final URL url) { + super(url); + } + + @Override + protected HttpURLConnection openConnection() throws IOException { + return (HttpURLConnection)url.openConnection(); + } + + /** Use HTTP Range header for specifying offset. */ + @Override + protected HttpURLConnection openConnection(final long offset) throws IOException { + final HttpURLConnection conn = openConnection(); + conn.setRequestMethod("GET"); + if (offset != 0L) { + conn.setRequestProperty("Range", "bytes=" + offset + "-"); + } + return conn; + } + } + + static class RangeHeaderInputStream extends ByteRangeInputStream { + RangeHeaderInputStream(RangeHeaderUrlOpener o, RangeHeaderUrlOpener r) { + super(o, r); + } + + RangeHeaderInputStream(final URL url) { + this(new RangeHeaderUrlOpener(url), new RangeHeaderUrlOpener(null)); + } + + /** Expects HTTP_OK and HTTP_PARTIAL response codes. */ + @Override + protected void checkResponseCode(final HttpURLConnection connection + ) throws IOException { + final int code = connection.getResponseCode(); + if (startPos != 0 && code != HttpURLConnection.HTTP_PARTIAL) { + // We asked for a byte range but did not receive a partial content + // response... + throw new IOException("HTTP_PARTIAL expected, received " + code); + } else if (startPos == 0 && code != HttpURLConnection.HTTP_OK) { + // We asked for all bytes from the beginning but didn't receive a 200 + // response (none of the other 2xx codes are valid here) + throw new IOException("HTTP_OK expected, received " + code); + } + } + + @Override + protected URL getResolvedUrl(final HttpURLConnection connection) { + return connection.getURL(); + } + } + @Override public FSDataInputStream open(Path f, int buffersize) throws IOException { f = f.makeQualified(getUri(), getWorkingDirectory()); String path = "/data" + ServletUtil.encodePath(f.toUri().getPath()); String query = addDelegationTokenParam("ugi=" + getEncodedUgiParameter()); URL u = getNamenodeURL(path, query); - return new FSDataInputStream(new ByteRangeInputStream(u)); + return new FSDataInputStream(new RangeHeaderInputStream(u)); } /** Class to parse and store a listing reply from the server. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 8ffd56b913..44b35b4022 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -50,27 +50,13 @@ import org.apache.hadoop.util.DataChecksum; -/** This is a wrapper around connection to datanode - * and understands checksum, offset etc. - * - * Terminology: - *

- *
block
- *
The hdfs block, typically large (~64MB). - *
- *
chunk
- *
A block is divided into chunks, each comes with a checksum. - * We want transfers to be chunk-aligned, to be able to - * verify checksums. - *
- *
packet
- *
A grouping of chunks used for transport. It contains a - * header, followed by checksum data, followed by real data. - *
- *
- * Please see DataNode for the RPC specification. +/** + * @deprecated this is an old implementation that is being left around + * in case any issues spring up with the new {@link RemoteBlockReader2} implementation. + * It will be removed in the next release. */ @InterfaceAudience.Private +@Deprecated public class RemoteBlockReader extends FSInputChecker implements BlockReader { Socket dnSock; //for now just sending the status code (e.g. checksumOk) after the read. @@ -410,7 +396,7 @@ public static RemoteBlockReader newBlockReader( Socket sock, String file, BlockOpResponseProto status = BlockOpResponseProto.parseFrom( vintPrefixed(in)); - checkSuccess(status, sock, block, file); + RemoteBlockReader2.checkSuccess(status, sock, block, file); ReadOpChecksumInfoProto checksumInfo = status.getReadOpChecksumInfo(); DataChecksum checksum = DataTransferProtoUtil.fromProto( @@ -431,28 +417,6 @@ public static RemoteBlockReader newBlockReader( Socket sock, String file, in, checksum, verifyChecksum, startOffset, firstChunkOffset, len, sock); } - private static void checkSuccess( - BlockOpResponseProto status, Socket sock, - ExtendedBlock block, String file) - throws IOException { - if (status.getStatus() != Status.SUCCESS) { - if (status.getStatus() == Status.ERROR_ACCESS_TOKEN) { - throw new InvalidBlockTokenException( - "Got access token error for OP_READ_BLOCK, self=" - + sock.getLocalSocketAddress() + ", remote=" - + sock.getRemoteSocketAddress() + ", for file " + file - + ", for pool " + block.getBlockPoolId() + " block " - + block.getBlockId() + "_" + block.getGenerationStamp()); - } else { - throw new IOException("Got error for OP_READ_BLOCK, self=" - + sock.getLocalSocketAddress() + ", remote=" - + sock.getRemoteSocketAddress() + ", for file " + file - + ", for pool " + block.getBlockPoolId() + " block " - + block.getBlockId() + "_" + block.getGenerationStamp()); - } - } - } - @Override public synchronized void close() throws IOException { startOffset = -1; @@ -464,6 +428,12 @@ public synchronized void close() throws IOException { // in will be closed when its Socket is closed. } + @Override + public void readFully(byte[] buf, int readOffset, int amtToRead) + throws IOException { + IOUtils.readFully(this, buf, readOffset, amtToRead); + } + @Override public int readAll(byte[] buf, int offset, int len) throws IOException { return readFully(this, buf, offset, len); @@ -492,14 +462,7 @@ public boolean hasSentStatusCode() { void sendReadResult(Socket sock, Status statusCode) { assert !sentStatusCode : "already sent status code to " + sock; try { - OutputStream out = NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT); - - ClientReadStatusProto.newBuilder() - .setStatus(statusCode) - .build() - .writeDelimitedTo(out); - - out.flush(); + RemoteBlockReader2.writeReadResult(sock, statusCode); sentStatusCode = true; } catch (IOException e) { // It's ok not to be able to send this. But something is probably wrong. @@ -519,4 +482,5 @@ public static String getFileName(final InetSocketAddress s, final String poolId, final long blockId) { return s.toString() + ":" + poolId + ":" + blockId; } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java new file mode 100644 index 0000000000..2b2f77ecb4 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -0,0 +1,500 @@ +/** + * 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.hdfs; + +import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed; + +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.ChecksumException; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; +import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.util.DirectBufferPool; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.net.SocketInputStream; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.DataChecksum; + +import com.google.common.base.Preconditions; + +/** + * This is a wrapper around connection to datanode + * and understands checksum, offset etc. + * + * Terminology: + *
+ *
block
+ *
The hdfs block, typically large (~64MB). + *
+ *
chunk
+ *
A block is divided into chunks, each comes with a checksum. + * We want transfers to be chunk-aligned, to be able to + * verify checksums. + *
+ *
packet
+ *
A grouping of chunks used for transport. It contains a + * header, followed by checksum data, followed by real data. + *
+ *
+ * Please see DataNode for the RPC specification. + * + * This is a new implementation introduced in Hadoop 0.23 which + * is more efficient and simpler than the older BlockReader + * implementation. It should be renamed to RemoteBlockReader + * once we are confident in it. + */ +@InterfaceAudience.Private +public class RemoteBlockReader2 implements BlockReader { + + static final Log LOG = LogFactory.getLog(RemoteBlockReader2.class); + + Socket dnSock; //for now just sending the status code (e.g. checksumOk) after the read. + private ReadableByteChannel in; + private DataChecksum checksum; + + private PacketHeader curHeader; + private ByteBuffer curPacketBuf = null; + private ByteBuffer curDataSlice = null; + + + /** offset in block of the last chunk received */ + private long lastSeqNo = -1; + + /** offset in block where reader wants to actually read */ + private long startOffset; + private final String filename; + + private static DirectBufferPool bufferPool = + new DirectBufferPool(); + private ByteBuffer headerBuf = ByteBuffer.allocate( + PacketHeader.PKT_HEADER_LEN); + + private int bytesPerChecksum; + private int checksumSize; + + /** + * The total number of bytes we need to transfer from the DN. + * This is the amount that the user has requested plus some padding + * at the beginning so that the read can begin on a chunk boundary. + */ + private long bytesNeededToFinish; + + private final boolean verifyChecksum; + + private boolean sentStatusCode = false; + + byte[] skipBuf = null; + ByteBuffer checksumBytes = null; + /** Amount of unread data in the current received packet */ + int dataLeft = 0; + + @Override + public synchronized int read(byte[] buf, int off, int len) + throws IOException { + + if (curPacketBuf == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) { + readNextPacket(); + } + if (curDataSlice.remaining() == 0) { + // we're at EOF now + return -1; + } + + int nRead = Math.min(curDataSlice.remaining(), len); + curDataSlice.get(buf, off, nRead); + + return nRead; + } + + private void readNextPacket() throws IOException { + Preconditions.checkState(curHeader == null || !curHeader.isLastPacketInBlock()); + + //Read packet headers. + readPacketHeader(); + + if (LOG.isTraceEnabled()) { + LOG.trace("DFSClient readNextPacket got header " + curHeader); + } + + // Sanity check the lengths + if (!curHeader.sanityCheck(lastSeqNo)) { + throw new IOException("BlockReader: error in packet header " + + curHeader); + } + + if (curHeader.getDataLen() > 0) { + int chunks = 1 + (curHeader.getDataLen() - 1) / bytesPerChecksum; + int checksumsLen = chunks * checksumSize; + int bufsize = checksumsLen + curHeader.getDataLen(); + + resetPacketBuffer(checksumsLen, curHeader.getDataLen()); + + lastSeqNo = curHeader.getSeqno(); + if (bufsize > 0) { + readChannelFully(in, curPacketBuf); + curPacketBuf.flip(); + if (verifyChecksum) { + verifyPacketChecksums(); + } + } + bytesNeededToFinish -= curHeader.getDataLen(); + } + + // First packet will include some data prior to the first byte + // the user requested. Skip it. + if (curHeader.getOffsetInBlock() < startOffset) { + int newPos = (int) (startOffset - curHeader.getOffsetInBlock()); + curDataSlice.position(newPos); + } + + // If we've now satisfied the whole client read, read one last packet + // header, which should be empty + if (bytesNeededToFinish <= 0) { + readTrailingEmptyPacket(); + if (verifyChecksum) { + sendReadResult(dnSock, Status.CHECKSUM_OK); + } else { + sendReadResult(dnSock, Status.SUCCESS); + } + } + } + + private void verifyPacketChecksums() throws ChecksumException { + // N.B.: the checksum error offset reported here is actually + // relative to the start of the block, not the start of the file. + // This is slightly misleading, but preserves the behavior from + // the older BlockReader. + checksum.verifyChunkedSums(curDataSlice, curPacketBuf, + filename, curHeader.getOffsetInBlock()); + } + + private static void readChannelFully(ReadableByteChannel ch, ByteBuffer buf) + throws IOException { + while (buf.remaining() > 0) { + int n = ch.read(buf); + if (n < 0) { + throw new IOException("Premature EOF reading from " + ch); + } + } + } + + private void resetPacketBuffer(int checksumsLen, int dataLen) { + int packetLen = checksumsLen + dataLen; + if (curPacketBuf == null || + curPacketBuf.capacity() < packetLen) { + returnPacketBufToPool(); + curPacketBuf = bufferPool.getBuffer(packetLen); + } + curPacketBuf.position(checksumsLen); + curDataSlice = curPacketBuf.slice(); + curDataSlice.limit(dataLen); + curPacketBuf.clear(); + curPacketBuf.limit(checksumsLen + dataLen); + } + + @Override + public synchronized long skip(long n) throws IOException { + /* How can we make sure we don't throw a ChecksumException, at least + * in majority of the cases?. This one throws. */ + if ( skipBuf == null ) { + skipBuf = new byte[bytesPerChecksum]; + } + + long nSkipped = 0; + while ( nSkipped < n ) { + int toSkip = (int)Math.min(n-nSkipped, skipBuf.length); + int ret = read(skipBuf, 0, toSkip); + if ( ret <= 0 ) { + return nSkipped; + } + nSkipped += ret; + } + return nSkipped; + } + + private void readPacketHeader() throws IOException { + headerBuf.clear(); + readChannelFully(in, headerBuf); + headerBuf.flip(); + if (curHeader == null) curHeader = new PacketHeader(); + curHeader.readFields(headerBuf); + } + + private void readTrailingEmptyPacket() throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("Reading empty packet at end of read"); + } + headerBuf.clear(); + readChannelFully(in, headerBuf); + headerBuf.flip(); + PacketHeader trailer = new PacketHeader(); + trailer.readFields(headerBuf); + if (!trailer.isLastPacketInBlock() || + trailer.getDataLen() != 0) { + throw new IOException("Expected empty end-of-read packet! Header: " + + trailer); + } + } + + private RemoteBlockReader2(String file, String bpid, long blockId, + ReadableByteChannel in, DataChecksum checksum, boolean verifyChecksum, + long startOffset, long firstChunkOffset, long bytesToRead, Socket dnSock) { + // Path is used only for printing block and file information in debug + this.dnSock = dnSock; + this.in = in; + this.checksum = checksum; + this.verifyChecksum = verifyChecksum; + this.startOffset = Math.max( startOffset, 0 ); + this.filename = file; + + // The total number of bytes that we need to transfer from the DN is + // the amount that the user wants (bytesToRead), plus the padding at + // the beginning in order to chunk-align. Note that the DN may elect + // to send more than this amount if the read starts/ends mid-chunk. + this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset); + bytesPerChecksum = this.checksum.getBytesPerChecksum(); + checksumSize = this.checksum.getChecksumSize(); + } + + + @Override + public synchronized void close() throws IOException { + returnPacketBufToPool(); + + startOffset = -1; + checksum = null; + if (dnSock != null) { + dnSock.close(); + } + + // in will be closed when its Socket is closed. + } + + @Override + protected void finalize() throws Throwable { + try { + // just in case it didn't get closed, we + // may as well still try to return the buffer + returnPacketBufToPool(); + } finally { + super.finalize(); + } + } + + private void returnPacketBufToPool() { + if (curPacketBuf != null) { + bufferPool.returnBuffer(curPacketBuf); + curPacketBuf = null; + } + } + + /** + * Take the socket used to talk to the DN. + */ + public Socket takeSocket() { + assert hasSentStatusCode() : + "BlockReader shouldn't give back sockets mid-read"; + Socket res = dnSock; + dnSock = null; + return res; + } + + /** + * Whether the BlockReader has reached the end of its input stream + * and successfully sent a status code back to the datanode. + */ + public boolean hasSentStatusCode() { + return sentStatusCode; + } + + /** + * When the reader reaches end of the read, it sends a status response + * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN + * closing our connection (which we will re-open), but won't affect + * data correctness. + */ + void sendReadResult(Socket sock, Status statusCode) { + assert !sentStatusCode : "already sent status code to " + sock; + try { + writeReadResult(sock, statusCode); + sentStatusCode = true; + } catch (IOException e) { + // It's ok not to be able to send this. But something is probably wrong. + LOG.info("Could not send read status (" + statusCode + ") to datanode " + + sock.getInetAddress() + ": " + e.getMessage()); + } + } + + /** + * Serialize the actual read result on the wire. + */ + static void writeReadResult(Socket sock, Status statusCode) + throws IOException { + OutputStream out = NetUtils.getOutputStream(sock, HdfsServerConstants.WRITE_TIMEOUT); + + ClientReadStatusProto.newBuilder() + .setStatus(statusCode) + .build() + .writeDelimitedTo(out); + + out.flush(); + } + + /** + * File name to print when accessing a block directly (from servlets) + * @param s Address of the block location + * @param poolId Block pool ID of the block + * @param blockId Block ID of the block + * @return string that has a file name for debug purposes + */ + public static String getFileName(final InetSocketAddress s, + final String poolId, final long blockId) { + return s.toString() + ":" + poolId + ":" + blockId; + } + + @Override + public int readAll(byte[] buf, int offset, int len) throws IOException { + int n = 0; + for (;;) { + int nread = read(buf, offset + n, len - n); + if (nread <= 0) + return (n == 0) ? nread : n; + n += nread; + if (n >= len) + return n; + } + } + + @Override + public void readFully(byte[] buf, int off, int len) + throws IOException { + int toRead = len; + while (toRead > 0) { + int ret = read(buf, off, toRead); + if (ret < 0) { + throw new IOException("Premature EOF from inputStream"); + } + toRead -= ret; + off += ret; + } + } + + /** + * Create a new BlockReader specifically to satisfy a read. + * This method also sends the OP_READ_BLOCK request. + * + * @param sock An established Socket to the DN. The BlockReader will not close it normally. + * This socket must have an associated Channel. + * @param file File location + * @param block The block object + * @param blockToken The block token for security + * @param startOffset The read offset, relative to block head + * @param len The number of bytes to read + * @param bufferSize The IO buffer size (not the client buffer size) + * @param verifyChecksum Whether to verify checksum + * @param clientName Client name + * @return New BlockReader instance, or null on error. + */ + public static BlockReader newBlockReader( Socket sock, String file, + ExtendedBlock block, + Token blockToken, + long startOffset, long len, + int bufferSize, boolean verifyChecksum, + String clientName) + throws IOException { + // in and out will be closed when sock is closed (by the caller) + final DataOutputStream out = new DataOutputStream(new BufferedOutputStream( + NetUtils.getOutputStream(sock, + HdfsServerConstants.WRITE_TIMEOUT))); + new Sender(out).readBlock(block, blockToken, clientName, startOffset, len); + + // + // Get bytes in block, set streams + // + Preconditions.checkArgument(sock.getChannel() != null, + "Socket %s does not have an associated Channel.", + sock); + SocketInputStream sin = + (SocketInputStream)NetUtils.getInputStream(sock); + DataInputStream in = new DataInputStream(sin); + + BlockOpResponseProto status = BlockOpResponseProto.parseFrom( + vintPrefixed(in)); + checkSuccess(status, sock, block, file); + ReadOpChecksumInfoProto checksumInfo = + status.getReadOpChecksumInfo(); + DataChecksum checksum = DataTransferProtoUtil.fromProto( + checksumInfo.getChecksum()); + //Warning when we get CHECKSUM_NULL? + + // Read the first chunk offset. + long firstChunkOffset = checksumInfo.getChunkOffset(); + + if ( firstChunkOffset < 0 || firstChunkOffset > startOffset || + firstChunkOffset >= (startOffset + checksum.getBytesPerChecksum())) { + throw new IOException("BlockReader: error in first chunk offset (" + + firstChunkOffset + ") startOffset is " + + startOffset + " for file " + file); + } + + return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(), + sin, checksum, verifyChecksum, startOffset, firstChunkOffset, len, sock); + } + + static void checkSuccess( + BlockOpResponseProto status, Socket sock, + ExtendedBlock block, String file) + throws IOException { + if (status.getStatus() != Status.SUCCESS) { + if (status.getStatus() == Status.ERROR_ACCESS_TOKEN) { + throw new InvalidBlockTokenException( + "Got access token error for OP_READ_BLOCK, self=" + + sock.getLocalSocketAddress() + ", remote=" + + sock.getRemoteSocketAddress() + ", for file " + file + + ", for pool " + block.getBlockPoolId() + " block " + + block.getBlockId() + "_" + block.getGenerationStamp()); + } else { + throw new IOException("Got error for OP_READ_BLOCK, self=" + + sock.getLocalSocketAddress() + ", remote=" + + sock.getRemoteSocketAddress() + ", for file " + file + + ", for pool " + block.getBlockPoolId() + " block " + + block.getBlockId() + "_" + block.getGenerationStamp()); + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java index 73e1b20a71..d8b9f2b620 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java @@ -136,7 +136,7 @@ public void write(DataOutputStream out) throws IOException { */ public boolean sanityCheck(long lastSeqNo) { // We should only have a non-positive data length for the last packet - if (proto.getDataLen() <= 0 && proto.getLastPacketInBlock()) return false; + if (proto.getDataLen() <= 0 && !proto.getLastPacketInBlock()) return false; // The last packet should not contain data if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false; // Seqnos should always increase by 1 with each packet received diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenRenewer.java index 3419be2969..349d71baeb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenRenewer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenRenewer.java @@ -134,7 +134,7 @@ public String toString() { private DelayQueue> queue = new DelayQueue>(); - public DelegationTokenRenewer(final Class clazz) { + public DelegationTokenRenewer(final Class clazz) { super(clazz.getSimpleName() + "-" + DelegationTokenRenewer.class.getSimpleName()); setDaemon(true); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java index 293d5c5969..c66c5cf3de 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java @@ -122,6 +122,38 @@ void setNext(int index, BlockInfo to) { triplets[index*3+2] = to; } + /** + * Return the previous block on the block list for the datanode at + * position index. Set the previous block on the list to "to". + * + * @param index - the datanode index + * @param to - block to be set to previous on the list of blocks + * @return current previous block on the list of blocks + */ + BlockInfo getSetPrevious(int index, BlockInfo to) { + assert this.triplets != null : "BlockInfo is not initialized"; + assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound"; + BlockInfo info = (BlockInfo)triplets[index*3+1]; + triplets[index*3+1] = to; + return info; + } + + /** + * Return the next block on the block list for the datanode at + * position index. Set the next block on the list to "to". + * + * @param index - the datanode index + * @param to - block to be set to next on the list of blocks + * * @return current next block on the list of blocks + */ + BlockInfo getSetNext(int index, BlockInfo to) { + assert this.triplets != null : "BlockInfo is not initialized"; + assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound"; + BlockInfo info = (BlockInfo)triplets[index*3+2]; + triplets[index*3+2] = to; + return info; + } + int getCapacity() { assert this.triplets != null : "BlockInfo is not initialized"; assert triplets.length % 3 == 0 : "Malformed BlockInfo"; @@ -259,6 +291,27 @@ public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) { return head; } + /** + * Remove this block from the list of blocks related to the specified + * DatanodeDescriptor. Insert it into the head of the list of blocks. + * + * @return the new head of the list. + */ + public BlockInfo moveBlockToHead(BlockInfo head, DatanodeDescriptor dn, + int curIndex, int headIndex) { + if (head == this) { + return this; + } + BlockInfo next = this.getSetNext(curIndex, head); + BlockInfo prev = this.getSetPrevious(curIndex, null); + + head.setPrevious(headIndex, this); + prev.setNext(prev.findDatanode(dn), next); + if (next != null) + next.setPrevious(next.findDatanode(dn), prev); + return this; + } + /** * BlockInfo represents a block that is not being constructed. * In order to start modifying the block, the BlockInfo should be converted @@ -317,4 +370,4 @@ public LightWeightGSet.LinkedElement getNext() { public void setNext(LightWeightGSet.LinkedElement next) { this.nextLinkedElement = next; } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 250781320c..64f76e39ba 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -65,6 +65,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; +import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; import org.apache.hadoop.net.Node; import org.apache.hadoop.util.Daemon; @@ -75,6 +76,7 @@ */ @InterfaceAudience.Private public class BlockManager { + static final Log LOG = LogFactory.getLog(BlockManager.class); /** Default load factor of map */ @@ -141,8 +143,8 @@ public long getExcessBlocksCount() { // eventually remove these extras. // Mapping: StorageID -> TreeSet // - public final Map> excessReplicateMap = - new TreeMap>(); + public final Map> excessReplicateMap = + new TreeMap>(); // // Store set of Blocks that need to be replicated 1 or more times. @@ -928,15 +930,7 @@ private int computeReplicationWork(int blocksToProcess) throws IOException { chooseUnderReplicatedBlocks(blocksToProcess); // replicate blocks - int scheduledReplicationCount = 0; - for (int i=0; i> chooseUnderReplicatedBlocks(int blocksToProcess) { return blocksToReplicate; } - /** Replicate a block + /** Replicate a set of blocks * - * @param block block to be replicated - * @param priority a hint of its priority in the neededReplication queue - * @return if the block gets replicated or not + * @param blocksToReplicate blocks to be replicated, for each priority + * @return the number of blocks scheduled for replication */ @VisibleForTesting - boolean computeReplicationWorkForBlock(Block block, int priority) { + int computeReplicationWorkForBlocks(List> blocksToReplicate) { int requiredReplication, numEffectiveReplicas; List containingNodes, liveReplicaNodes; DatanodeDescriptor srcNode; INodeFile fileINode = null; int additionalReplRequired; + int scheduledWork = 0; + List work = new LinkedList(); + namesystem.writeLock(); try { synchronized (neededReplications) { - // block should belong to a file - fileINode = blocksMap.getINode(block); - // abandoned block or block reopened for append - if(fileINode == null || fileINode.isUnderConstruction()) { - neededReplications.remove(block, priority); // remove from neededReplications - replIndex--; - return false; - } + for (int priority = 0; priority < blocksToReplicate.size(); priority++) { + for (Block block : blocksToReplicate.get(priority)) { + // block should belong to a file + fileINode = blocksMap.getINode(block); + // abandoned block or block reopened for append + if(fileINode == null || fileINode.isUnderConstruction()) { + neededReplications.remove(block, priority); // remove from neededReplications + replIndex--; + continue; + } - requiredReplication = fileINode.getReplication(); + requiredReplication = fileINode.getReplication(); - // get a source data-node - containingNodes = new ArrayList(); - liveReplicaNodes = new ArrayList(); - NumberReplicas numReplicas = new NumberReplicas(); - srcNode = chooseSourceDatanode( - block, containingNodes, liveReplicaNodes, numReplicas); - if(srcNode == null) // block can not be replicated from any node - return false; + // get a source data-node + containingNodes = new ArrayList(); + liveReplicaNodes = new ArrayList(); + NumberReplicas numReplicas = new NumberReplicas(); + srcNode = chooseSourceDatanode( + block, containingNodes, liveReplicaNodes, numReplicas); + if(srcNode == null) // block can not be replicated from any node + continue; - assert liveReplicaNodes.size() == numReplicas.liveReplicas(); - // do not schedule more if enough replicas is already pending - numEffectiveReplicas = numReplicas.liveReplicas() + - pendingReplications.getNumReplicas(block); + assert liveReplicaNodes.size() == numReplicas.liveReplicas(); + // do not schedule more if enough replicas is already pending + numEffectiveReplicas = numReplicas.liveReplicas() + + pendingReplications.getNumReplicas(block); - if (numEffectiveReplicas >= requiredReplication) { - if ( (pendingReplications.getNumReplicas(block) > 0) || - (blockHasEnoughRacks(block)) ) { - neededReplications.remove(block, priority); // remove from neededReplications - replIndex--; - NameNode.stateChangeLog.info("BLOCK* " - + "Removing block " + block - + " from neededReplications as it has enough replicas."); - return false; + if (numEffectiveReplicas >= requiredReplication) { + if ( (pendingReplications.getNumReplicas(block) > 0) || + (blockHasEnoughRacks(block)) ) { + neededReplications.remove(block, priority); // remove from neededReplications + replIndex--; + NameNode.stateChangeLog.info("BLOCK* " + + "Removing block " + block + + " from neededReplications as it has enough replicas."); + continue; + } + } + + if (numReplicas.liveReplicas() < requiredReplication) { + additionalReplRequired = requiredReplication + - numEffectiveReplicas; + } else { + additionalReplRequired = 1; // Needed on a new rack + } + work.add(new ReplicationWork(block, fileINode, srcNode, + containingNodes, liveReplicaNodes, additionalReplRequired, + priority)); } } - - if (numReplicas.liveReplicas() < requiredReplication) { - additionalReplRequired = requiredReplication - numEffectiveReplicas; - } else { - additionalReplRequired = 1; //Needed on a new rack - } - } } finally { namesystem.writeUnlock(); } - - // Exclude all of the containing nodes from being targets. - // This list includes decommissioning or corrupt nodes. - HashMap excludedNodes = new HashMap(); - for (DatanodeDescriptor dn : containingNodes) { - excludedNodes.put(dn, dn); - } - // choose replication targets: NOT HOLDING THE GLOBAL LOCK - // It is costly to extract the filename for which chooseTargets is called, - // so for now we pass in the Inode itself. - DatanodeDescriptor targets[] = - blockplacement.chooseTarget(fileINode, additionalReplRequired, - srcNode, liveReplicaNodes, excludedNodes, block.getNumBytes()); - if(targets.length == 0) - return false; + HashMap excludedNodes + = new HashMap(); + for(ReplicationWork rw : work){ + // Exclude all of the containing nodes from being targets. + // This list includes decommissioning or corrupt nodes. + excludedNodes.clear(); + for (DatanodeDescriptor dn : rw.containingNodes) { + excludedNodes.put(dn, dn); + } + + // choose replication targets: NOT HOLDING THE GLOBAL LOCK + // It is costly to extract the filename for which chooseTargets is called, + // so for now we pass in the Inode itself. + rw.targets = blockplacement.chooseTarget(rw.fileINode, + rw.additionalReplRequired, rw.srcNode, rw.liveReplicaNodes, + excludedNodes, rw.block.getNumBytes()); + } namesystem.writeLock(); try { - synchronized (neededReplications) { - // Recheck since global lock was released - // block should belong to a file - fileINode = blocksMap.getINode(block); - // abandoned block or block reopened for append - if(fileINode == null || fileINode.isUnderConstruction()) { - neededReplications.remove(block, priority); // remove from neededReplications - replIndex--; - return false; + for(ReplicationWork rw : work){ + DatanodeDescriptor[] targets = rw.targets; + if(targets == null || targets.length == 0){ + rw.targets = null; + continue; } - requiredReplication = fileINode.getReplication(); - // do not schedule more if enough replicas is already pending - NumberReplicas numReplicas = countNodes(block); - numEffectiveReplicas = numReplicas.liveReplicas() + - pendingReplications.getNumReplicas(block); + synchronized (neededReplications) { + Block block = rw.block; + int priority = rw.priority; + // Recheck since global lock was released + // block should belong to a file + fileINode = blocksMap.getINode(block); + // abandoned block or block reopened for append + if(fileINode == null || fileINode.isUnderConstruction()) { + neededReplications.remove(block, priority); // remove from neededReplications + rw.targets = null; + replIndex--; + continue; + } + requiredReplication = fileINode.getReplication(); - if (numEffectiveReplicas >= requiredReplication) { - if ( (pendingReplications.getNumReplicas(block) > 0) || - (blockHasEnoughRacks(block)) ) { + // do not schedule more if enough replicas is already pending + NumberReplicas numReplicas = countNodes(block); + numEffectiveReplicas = numReplicas.liveReplicas() + + pendingReplications.getNumReplicas(block); + + if (numEffectiveReplicas >= requiredReplication) { + if ( (pendingReplications.getNumReplicas(block) > 0) || + (blockHasEnoughRacks(block)) ) { + neededReplications.remove(block, priority); // remove from neededReplications + replIndex--; + rw.targets = null; + NameNode.stateChangeLog.info("BLOCK* " + + "Removing block " + block + + " from neededReplications as it has enough replicas."); + continue; + } + } + + if ( (numReplicas.liveReplicas() >= requiredReplication) && + (!blockHasEnoughRacks(block)) ) { + if (rw.srcNode.getNetworkLocation().equals(targets[0].getNetworkLocation())) { + //No use continuing, unless a new rack in this case + continue; + } + } + + // Add block to the to be replicated list + rw.srcNode.addBlockToBeReplicated(block, targets); + scheduledWork++; + + for (DatanodeDescriptor dn : targets) { + dn.incBlocksScheduled(); + } + + // Move the block-replication into a "pending" state. + // The reason we use 'pending' is so we can retry + // replications that fail after an appropriate amount of time. + pendingReplications.add(block, targets.length); + if(NameNode.stateChangeLog.isDebugEnabled()) { + NameNode.stateChangeLog.debug( + "BLOCK* block " + block + + " is moved from neededReplications to pendingReplications"); + } + + // remove from neededReplications + if(numEffectiveReplicas + targets.length >= requiredReplication) { neededReplications.remove(block, priority); // remove from neededReplications replIndex--; - NameNode.stateChangeLog.info("BLOCK* " - + "Removing block " + block - + " from neededReplications as it has enough replicas."); - return false; } } + } + } finally { + namesystem.writeUnlock(); + } - if ( (numReplicas.liveReplicas() >= requiredReplication) && - (!blockHasEnoughRacks(block)) ) { - if (srcNode.getNetworkLocation().equals(targets[0].getNetworkLocation())) { - //No use continuing, unless a new rack in this case - return false; - } - } - - // Add block to the to be replicated list - srcNode.addBlockToBeReplicated(block, targets); - - for (DatanodeDescriptor dn : targets) { - dn.incBlocksScheduled(); - } - - // Move the block-replication into a "pending" state. - // The reason we use 'pending' is so we can retry - // replications that fail after an appropriate amount of time. - pendingReplications.add(block, targets.length); - if(NameNode.stateChangeLog.isDebugEnabled()) { - NameNode.stateChangeLog.debug( - "BLOCK* block " + block - + " is moved from neededReplications to pendingReplications"); - } - - // remove from neededReplications - if(numEffectiveReplicas + targets.length >= requiredReplication) { - neededReplications.remove(block, priority); // remove from neededReplications - replIndex--; - } - if (NameNode.stateChangeLog.isInfoEnabled()) { + if (NameNode.stateChangeLog.isInfoEnabled()) { + // log which blocks have been scheduled for replication + for(ReplicationWork rw : work){ + DatanodeDescriptor[] targets = rw.targets; + if (targets != null && targets.length != 0) { StringBuilder targetList = new StringBuilder("datanode(s)"); for (int k = 0; k < targets.length; k++) { targetList.append(' '); targetList.append(targets[k].getName()); } NameNode.stateChangeLog.info( - "BLOCK* ask " - + srcNode.getName() + " to replicate " - + block + " to " + targetList); - if(NameNode.stateChangeLog.isDebugEnabled()) { - NameNode.stateChangeLog.debug( - "BLOCK* neededReplications = " + neededReplications.size() - + " pendingReplications = " + pendingReplications.size()); - } + "BLOCK* ask " + + rw.srcNode.getName() + " to replicate " + + rw.block + " to " + targetList); } } - } finally { - namesystem.writeUnlock(); + } + if(NameNode.stateChangeLog.isDebugEnabled()) { + NameNode.stateChangeLog.debug( + "BLOCK* neededReplications = " + neededReplications.size() + + " pendingReplications = " + pendingReplications.size()); } - return true; + return scheduledWork; } /** @@ -1220,7 +1245,7 @@ private DatanodeDescriptor chooseSourceDatanode( Collection nodesCorrupt = corruptReplicas.getNodes(block); while(it.hasNext()) { DatanodeDescriptor node = it.next(); - Collection excessBlocks = + LightWeightLinkedSet excessBlocks = excessReplicateMap.get(node.getStorageID()); if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) corrupt++; @@ -1443,7 +1468,10 @@ private void reportDiff(DatanodeDescriptor dn, BlockInfo delimiter = new BlockInfo(new Block(), 1); boolean added = dn.addBlock(delimiter); assert added : "Delimiting block cannot be present in the node"; - if(newReport == null) + int headIndex = 0; //currently the delimiter is in the head of the list + int curIndex; + + if (newReport == null) newReport = new BlockListAsLongs(); // scan the report and process newly reported blocks BlockReportIterator itBR = newReport.getBlockReportIterator(); @@ -1453,8 +1481,9 @@ private void reportDiff(DatanodeDescriptor dn, BlockInfo storedBlock = processReportedBlock(dn, iblk, iState, toAdd, toInvalidate, toCorrupt, toUC); // move block to the head of the list - if(storedBlock != null && storedBlock.findDatanode(dn) >= 0) - dn.moveBlockToHead(storedBlock); + if (storedBlock != null && (curIndex = storedBlock.findDatanode(dn)) >= 0) { + headIndex = dn.moveBlockToHead(storedBlock, curIndex, headIndex); + } } // collect blocks that have not been reported // all of them are next to the delimiter @@ -1871,7 +1900,7 @@ private void processOverReplicatedBlock(final Block block, for (Iterator it = blocksMap.nodeIterator(block); it.hasNext();) { DatanodeDescriptor cur = it.next(); - Collection excessBlocks = excessReplicateMap.get(cur + LightWeightLinkedSet excessBlocks = excessReplicateMap.get(cur .getStorageID()); if (excessBlocks == null || !excessBlocks.contains(block)) { if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { @@ -1989,9 +2018,9 @@ private void chooseExcessReplicates(Collection nonExcess, private void addToExcessReplicate(DatanodeInfo dn, Block block) { assert namesystem.hasWriteLock(); - Collection excessBlocks = excessReplicateMap.get(dn.getStorageID()); + LightWeightLinkedSet excessBlocks = excessReplicateMap.get(dn.getStorageID()); if (excessBlocks == null) { - excessBlocks = new TreeSet(); + excessBlocks = new LightWeightLinkedSet(); excessReplicateMap.put(dn.getStorageID(), excessBlocks); } if (excessBlocks.add(block)) { @@ -2039,7 +2068,7 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { // We've removed a block from a node, so it's definitely no longer // in "excess" there. // - Collection excessBlocks = excessReplicateMap.get(node + LightWeightLinkedSet excessBlocks = excessReplicateMap.get(node .getStorageID()); if (excessBlocks != null) { if (excessBlocks.remove(block)) { @@ -2189,8 +2218,8 @@ public NumberReplicas countNodes(Block b) { } else if (node.isDecommissionInProgress() || node.isDecommissioned()) { count++; } else { - Collection blocksExcess = - excessReplicateMap.get(node.getStorageID()); + LightWeightLinkedSet blocksExcess = excessReplicateMap.get(node + .getStorageID()); if (blocksExcess != null && blocksExcess.contains(b)) { excess++; } else { @@ -2591,4 +2620,34 @@ int computeDatanodeWork() throws IOException { return workFound; } + private static class ReplicationWork { + + private Block block; + private INodeFile fileINode; + + private DatanodeDescriptor srcNode; + private List containingNodes; + private List liveReplicaNodes; + private int additionalReplRequired; + + private DatanodeDescriptor targets[]; + private int priority; + + public ReplicationWork(Block block, + INodeFile fileINode, + DatanodeDescriptor srcNode, + List containingNodes, + List liveReplicaNodes, + int additionalReplRequired, + int priority) { + this.block = block; + this.fileINode = fileINode; + this.srcNode = srcNode; + this.containingNodes = containingNodes; + this.liveReplicaNodes = liveReplicaNodes; + this.additionalReplRequired = additionalReplRequired; + this.priority = priority; + this.targets = null; + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java index 0f310d36eb..403fbabaa3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.util.LightWeightHashSet; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableUtils; @@ -120,11 +121,11 @@ boolean contains(E e) { private BlockQueue recoverBlocks = new BlockQueue(); /** A set of blocks to be invalidated by this datanode */ - private Set invalidateBlocks = new TreeSet(); + private LightWeightHashSet invalidateBlocks = new LightWeightHashSet(); /* Variables for maintaining number of blocks scheduled to be written to * this datanode. This count is approximate and might be slightly bigger - * in case of errors (e.g. datanode does not report if an error occurs + * in case of errors (e.g. datanode does not report if an error occurs * while writing the block). */ private int currApproxBlocksScheduled = 0; @@ -244,15 +245,24 @@ public boolean removeBlock(BlockInfo b) { /** * Move block to the head of the list of blocks belonging to the data-node. + * @return the index of the head of the blockList */ - void moveBlockToHead(BlockInfo b) { - blockList = b.listRemove(blockList, this); - blockList = b.listInsert(blockList, this); + int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) { + blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex); + return curIndex; + } + + /** + * Used for testing only + * @return the head of the blockList + */ + protected BlockInfo getHead(){ + return blockList; } /** * Replace specified old block with a new one in the DataNodeDescriptor. - * + * * @param oldBlock - block to be replaced * @param newBlock - a replacement block * @return the new block @@ -391,45 +401,11 @@ public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) { * Remove the specified number of blocks to be invalidated */ public Block[] getInvalidateBlocks(int maxblocks) { - return getBlockArray(invalidateBlocks, maxblocks); - } - - static private Block[] getBlockArray(Collection blocks, int max) { - Block[] blockarray = null; - synchronized(blocks) { - int available = blocks.size(); - int n = available; - if (max > 0 && n > 0) { - if (max < n) { - n = max; - } - // allocate the properly sized block array ... - blockarray = new Block[n]; - - // iterate tree collecting n blocks... - Iterator e = blocks.iterator(); - int blockCount = 0; - - while (blockCount < n && e.hasNext()) { - // insert into array ... - blockarray[blockCount++] = e.next(); - - // remove from tree via iterator, if we are removing - // less than total available blocks - if (n < available){ - e.remove(); - } - } - assert(blockarray.length == n); - - // now if the number of blocks removed equals available blocks, - // them remove all blocks in one fell swoop via clear - if (n == available) { - blocks.clear(); - } - } + synchronized (invalidateBlocks) { + Block[] deleteList = invalidateBlocks.pollToArray(new Block[Math.min( + invalidateBlocks.size(), maxblocks)]); + return deleteList.length == 0 ? null : deleteList; } - return blockarray; } /** Serialization for FSEditLog */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java index 7bad2b78fc..6181588475 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java @@ -30,8 +30,9 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.util.LightWeightHashSet; -/** +/** * Keeps a Collection for every named machine containing blocks * that have recently been invalidated and are thought to live * on the machine in question. @@ -39,8 +40,8 @@ @InterfaceAudience.Private class InvalidateBlocks { /** Mapping: StorageID -> Collection of Blocks */ - private final Map> node2blocks = - new TreeMap>(); + private final Map> node2blocks = + new TreeMap>(); /** The total number of blocks in the map. */ private long numBlocks = 0L; @@ -67,9 +68,9 @@ synchronized boolean contains(final String storageID, final Block block) { */ synchronized void add(final Block block, final DatanodeInfo datanode, final boolean log) { - Collection set = node2blocks.get(datanode.getStorageID()); + LightWeightHashSet set = node2blocks.get(datanode.getStorageID()); if (set == null) { - set = new HashSet(); + set = new LightWeightHashSet(); node2blocks.put(datanode.getStorageID(), set); } if (set.add(block)) { @@ -83,7 +84,7 @@ synchronized void add(final Block block, final DatanodeInfo datanode, /** Remove a storage from the invalidatesSet */ synchronized void remove(final String storageID) { - final Collection blocks = node2blocks.remove(storageID); + final LightWeightHashSet blocks = node2blocks.remove(storageID); if (blocks != null) { numBlocks -= blocks.size(); } @@ -91,7 +92,7 @@ synchronized void remove(final String storageID) { /** Remove the block from the specified storage. */ synchronized void remove(final String storageID, final Block block) { - final Collection v = node2blocks.get(storageID); + final LightWeightHashSet v = node2blocks.get(storageID); if (v != null && v.remove(block)) { numBlocks--; if (v.isEmpty()) { @@ -109,8 +110,8 @@ synchronized void dump(final PrintWriter out) { return; } - for(Map.Entry> entry : node2blocks.entrySet()) { - final Collection blocks = entry.getValue(); + for(Map.Entry> entry : node2blocks.entrySet()) { + final LightWeightHashSet blocks = entry.getValue(); if (blocks.size() > 0) { out.println(datanodeManager.getDatanode(entry.getKey()).getName() + blocks); } @@ -143,21 +144,17 @@ int invalidateWork(final String storageId) { private synchronized List invalidateWork( final String storageId, final DatanodeDescriptor dn) { - final Collection set = node2blocks.get(storageId); + final LightWeightHashSet set = node2blocks.get(storageId); if (set == null) { return null; } // # blocks that can be sent in one message is limited final int limit = datanodeManager.blockInvalidateLimit; - final List toInvalidate = new ArrayList(limit); - final Iterator it = set.iterator(); - for(int count = 0; count < limit && it.hasNext(); count++) { - toInvalidate.add(it.next()); - it.remove(); - } + final List toInvalidate = set.pollN(limit); + // If we send everything in this message, remove this node entry - if (!it.hasNext()) { + if (set.isEmpty()) { remove(storageId); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java index dc8d9e8db3..d35db96ec0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java @@ -24,6 +24,7 @@ import java.util.TreeSet; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; import org.apache.hadoop.hdfs.server.namenode.NameNode; /** @@ -80,13 +81,13 @@ class UnderReplicatedBlocks implements Iterable { /** The queue for corrupt blocks: {@value} */ static final int QUEUE_WITH_CORRUPT_BLOCKS = 4; /** the queues themselves */ - private final List> priorityQueues - = new ArrayList>(LEVEL); + private List> priorityQueues + = new ArrayList>(); /** Create an object. */ UnderReplicatedBlocks() { for (int i = 0; i < LEVEL; i++) { - priorityQueues.add(new TreeSet()); + priorityQueues.add(new LightWeightLinkedSet()); } } @@ -123,10 +124,10 @@ synchronized int getUnderReplicatedBlockCount() { synchronized int getCorruptBlockSize() { return priorityQueues.get(QUEUE_WITH_CORRUPT_BLOCKS).size(); } - + /** Check if a block is in the neededReplication queue */ synchronized boolean contains(Block block) { - for (NavigableSet set : priorityQueues) { + for(LightWeightLinkedSet set : priorityQueues) { if (set.contains(block)) { return true; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java index 8ae1390ed8..b20e694acf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java @@ -56,14 +56,17 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer; import org.apache.hadoop.hdfs.web.resources.DelegationParam; +import org.apache.hadoop.hdfs.web.resources.DoAsParam; import org.apache.hadoop.hdfs.web.resources.UserParam; import org.apache.hadoop.http.HtmlQuoting; import org.apache.hadoop.io.Text; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.AccessControlException; -import org.apache.hadoop.security.authentication.util.KerberosName; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; +import org.apache.hadoop.security.authentication.util.KerberosName; +import org.apache.hadoop.security.authorize.AuthorizationException; +import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.VersionInfo; @@ -117,7 +120,8 @@ public int compare(NodeRecord o1, NodeRecord o2) { return 0; } } - public static DatanodeInfo bestNode(LocatedBlocks blks) throws IOException { + public static DatanodeInfo bestNode(LocatedBlocks blks, Configuration conf) + throws IOException { HashMap map = new HashMap(); for (LocatedBlock block : blks.getLocatedBlocks()) { @@ -133,16 +137,17 @@ public static DatanodeInfo bestNode(LocatedBlocks blks) throws IOException { } NodeRecord[] nodes = map.values().toArray(new NodeRecord[map.size()]); Arrays.sort(nodes, new NodeRecordComparator()); - return bestNode(nodes, false); + return bestNode(nodes, false, conf); } - public static DatanodeInfo bestNode(LocatedBlock blk) throws IOException { + public static DatanodeInfo bestNode(LocatedBlock blk, Configuration conf) + throws IOException { DatanodeInfo[] nodes = blk.getLocations(); - return bestNode(nodes, true); + return bestNode(nodes, true, conf); } - public static DatanodeInfo bestNode(DatanodeInfo[] nodes, boolean doRandom) - throws IOException { + public static DatanodeInfo bestNode(DatanodeInfo[] nodes, boolean doRandom, + Configuration conf) throws IOException { TreeSet deadNodes = new TreeSet(); DatanodeInfo chosenNode = null; int failures = 0; @@ -169,7 +174,7 @@ public static DatanodeInfo bestNode(DatanodeInfo[] nodes, boolean doRandom) chosenNode.getHost() + ":" + chosenNode.getInfoPort()); try { - s = new Socket(); + s = NetUtils.getDefaultSocketFactory(conf).createSocket(); s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT); s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); } catch (IOException e) { @@ -191,27 +196,26 @@ public static void streamBlockInAscii(InetSocketAddress addr, String poolId, long blockSize, long offsetIntoBlock, long chunkSizeToView, JspWriter out, Configuration conf) throws IOException { if (chunkSizeToView == 0) return; - Socket s = new Socket(); + Socket s = NetUtils.getDefaultSocketFactory(conf).createSocket(); s.connect(addr, HdfsServerConstants.READ_TIMEOUT); s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); - long amtToRead = Math.min(chunkSizeToView, blockSize - offsetIntoBlock); + int amtToRead = (int)Math.min(chunkSizeToView, blockSize - offsetIntoBlock); // Use the block name for file name. - int bufferSize = conf.getInt(DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY, - DFSConfigKeys.IO_FILE_BUFFER_SIZE_DEFAULT); String file = BlockReaderFactory.getFileName(addr, poolId, blockId); - BlockReader blockReader = BlockReaderFactory.newBlockReader(s, file, + BlockReader blockReader = BlockReaderFactory.newBlockReader( + conf, s, file, new ExtendedBlock(poolId, blockId, 0, genStamp), blockToken, - offsetIntoBlock, amtToRead, bufferSize); + offsetIntoBlock, amtToRead); byte[] buf = new byte[(int)amtToRead]; int readOffset = 0; int retries = 2; while ( amtToRead > 0 ) { - int numRead; + int numRead = amtToRead; try { - numRead = blockReader.readAll(buf, readOffset, (int)amtToRead); + blockReader.readFully(buf, readOffset, amtToRead); } catch (IOException e) { retries--; @@ -534,9 +538,10 @@ public static UserGroupInformation getUGI(ServletContext context, final boolean tryUgiParameter) throws IOException { final UserGroupInformation ugi; final String usernameFromQuery = getUsernameFromQuery(request, tryUgiParameter); + final String doAsUserFromQuery = request.getParameter(DoAsParam.NAME); if(UserGroupInformation.isSecurityEnabled()) { - final String user = request.getRemoteUser(); + final String remoteUser = request.getRemoteUser(); String tokenString = request.getParameter(DELEGATION_PARAMETER_NAME); if (tokenString != null) { Token token = @@ -544,9 +549,8 @@ public static UserGroupInformation getUGI(ServletContext context, token.decodeFromUrlString(tokenString); String serviceAddress = getNNServiceAddress(context, request); if (serviceAddress != null) { - LOG.info("Setting service in token: " - + new Text(serviceAddress)); token.setService(new Text(serviceAddress)); + token.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND); } ByteArrayInputStream buf = new ByteArrayInputStream(token .getIdentifier()); @@ -561,26 +565,36 @@ public static UserGroupInformation getUGI(ServletContext context, } } ugi = id.getUser(); - checkUsername(ugi.getShortUserName(), usernameFromQuery); - checkUsername(ugi.getShortUserName(), user); + if (ugi.getRealUser() == null) { + //non-proxy case + checkUsername(ugi.getShortUserName(), usernameFromQuery); + checkUsername(null, doAsUserFromQuery); + } else { + //proxy case + checkUsername(ugi.getRealUser().getShortUserName(), usernameFromQuery); + checkUsername(ugi.getShortUserName(), doAsUserFromQuery); + ProxyUsers.authorize(ugi, request.getRemoteAddr(), conf); + } ugi.addToken(token); ugi.setAuthenticationMethod(AuthenticationMethod.TOKEN); } else { - if(user == null) { + if(remoteUser == null) { throw new IOException("Security enabled but user not " + "authenticated by filter"); } - ugi = UserGroupInformation.createRemoteUser(user); - checkUsername(ugi.getShortUserName(), usernameFromQuery); + final UserGroupInformation realUgi = UserGroupInformation.createRemoteUser(remoteUser); + checkUsername(realUgi.getShortUserName(), usernameFromQuery); // This is not necessarily true, could have been auth'ed by user-facing // filter - ugi.setAuthenticationMethod(secureAuthMethod); + realUgi.setAuthenticationMethod(secureAuthMethod); + ugi = initUGI(realUgi, doAsUserFromQuery, request, true, conf); } } else { // Security's not on, pull from url - ugi = usernameFromQuery == null? + final UserGroupInformation realUgi = usernameFromQuery == null? getDefaultWebUser(conf) // not specified in request : UserGroupInformation.createRemoteUser(usernameFromQuery); - ugi.setAuthenticationMethod(AuthenticationMethod.SIMPLE); + realUgi.setAuthenticationMethod(AuthenticationMethod.SIMPLE); + ugi = initUGI(realUgi, doAsUserFromQuery, request, false, conf); } if(LOG.isDebugEnabled()) @@ -588,12 +602,34 @@ public static UserGroupInformation getUGI(ServletContext context, return ugi; } + private static UserGroupInformation initUGI(final UserGroupInformation realUgi, + final String doAsUserFromQuery, final HttpServletRequest request, + final boolean isSecurityEnabled, final Configuration conf + ) throws AuthorizationException { + final UserGroupInformation ugi; + if (doAsUserFromQuery == null) { + //non-proxy case + ugi = realUgi; + } else { + //proxy case + ugi = UserGroupInformation.createProxyUser(doAsUserFromQuery, realUgi); + ugi.setAuthenticationMethod( + isSecurityEnabled? AuthenticationMethod.PROXY: AuthenticationMethod.SIMPLE); + ProxyUsers.authorize(ugi, request.getRemoteAddr(), conf); + } + return ugi; + } + /** * Expected user name should be a short name. */ private static void checkUsername(final String expected, final String name ) throws IOException { - if (name == null) { + if (expected == null && name != null) { + throw new IOException("Usernames not matched: expecting null but name=" + + name); + } + if (name == null) { //name is optional, null is okay return; } KerberosName u = new KerberosName(name); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java index 3254e982cd..ad5c6d878a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java @@ -18,12 +18,15 @@ package org.apache.hadoop.hdfs.server.datanode; import java.io.BufferedInputStream; +import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.IOException; +import java.io.RandomAccessFile; +import org.apache.commons.httpclient.methods.GetMethod; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.DataChecksum; @@ -88,6 +91,18 @@ static BlockMetadataHeader readHeader(File file) throws IOException { } } + /** + * Read the header at the beginning of the given block meta file. + * The current file position will be altered by this method. + * If an error occurs, the file is not closed. + */ + static BlockMetadataHeader readHeader(RandomAccessFile raf) throws IOException { + byte[] buf = new byte[getHeaderSize()]; + raf.seek(0); + raf.readFully(buf, 0, buf.length); + return readHeader(new DataInputStream(new ByteArrayInputStream(buf))); + } + // Version is already read. private static BlockMetadataHeader readHeader(short version, DataInputStream in) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index 94920fd5bc..9277956e1f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -63,7 +63,15 @@ class BlockReceiver implements Closeable { private static final long CACHE_DROP_LAG_BYTES = 8 * 1024 * 1024; private DataInputStream in = null; // from where data are read - private DataChecksum checksum; // from where chunks of a block can be read + private DataChecksum clientChecksum; // checksum used by client + private DataChecksum diskChecksum; // checksum we write to disk + + /** + * In the case that the client is writing with a different + * checksum polynomial than the block is stored with on disk, + * the DataNode needs to recalculate checksums before writing. + */ + private boolean needsChecksumTranslation; private OutputStream out = null; // to block file at local disk private FileDescriptor outFd; private OutputStream cout = null; // output stream for cehcksum file @@ -177,33 +185,35 @@ class BlockReceiver implements Closeable { " while receiving block " + block + " from " + inAddr); } } - // read checksum meta information - this.checksum = requestedChecksum; - this.bytesPerChecksum = checksum.getBytesPerChecksum(); - this.checksumSize = checksum.getChecksumSize(); this.dropCacheBehindWrites = datanode.shouldDropCacheBehindWrites(); this.syncBehindWrites = datanode.shouldSyncBehindWrites(); final boolean isCreate = isDatanode || isTransfer || stage == BlockConstructionStage.PIPELINE_SETUP_CREATE; - streams = replicaInfo.createStreams(isCreate, - this.bytesPerChecksum, this.checksumSize); - if (streams != null) { - this.out = streams.dataOut; - if (out instanceof FileOutputStream) { - this.outFd = ((FileOutputStream)out).getFD(); - } else { - LOG.warn("Could not get file descriptor for outputstream of class " + - out.getClass()); - } - this.cout = streams.checksumOut; - this.checksumOut = new DataOutputStream(new BufferedOutputStream( - streams.checksumOut, HdfsConstants.SMALL_BUFFER_SIZE)); - // write data chunk header if creating a new replica - if (isCreate) { - BlockMetadataHeader.writeHeader(checksumOut, checksum); - } + streams = replicaInfo.createStreams(isCreate, requestedChecksum); + assert streams != null : "null streams!"; + + // read checksum meta information + this.clientChecksum = requestedChecksum; + this.diskChecksum = streams.getChecksum(); + this.needsChecksumTranslation = !clientChecksum.equals(diskChecksum); + this.bytesPerChecksum = diskChecksum.getBytesPerChecksum(); + this.checksumSize = diskChecksum.getChecksumSize(); + + this.out = streams.dataOut; + if (out instanceof FileOutputStream) { + this.outFd = ((FileOutputStream)out).getFD(); + } else { + LOG.warn("Could not get file descriptor for outputstream of class " + + out.getClass()); } + this.cout = streams.checksumOut; + this.checksumOut = new DataOutputStream(new BufferedOutputStream( + streams.checksumOut, HdfsConstants.SMALL_BUFFER_SIZE)); + // write data chunk header if creating a new replica + if (isCreate) { + BlockMetadataHeader.writeHeader(checksumOut, diskChecksum); + } } catch (ReplicaAlreadyExistsException bae) { throw bae; } catch (ReplicaNotFoundException bne) { @@ -315,9 +325,9 @@ private void verifyChunks( byte[] dataBuf, int dataOff, int len, while (len > 0) { int chunkLen = Math.min(len, bytesPerChecksum); - checksum.update(dataBuf, dataOff, chunkLen); + clientChecksum.update(dataBuf, dataOff, chunkLen); - if (!checksum.compare(checksumBuf, checksumOff)) { + if (!clientChecksum.compare(checksumBuf, checksumOff)) { if (srcDataNode != null) { try { LOG.info("report corrupt block " + block + " from datanode " + @@ -334,12 +344,32 @@ private void verifyChunks( byte[] dataBuf, int dataOff, int len, "while writing " + block + " from " + inAddr); } - checksum.reset(); + clientChecksum.reset(); dataOff += chunkLen; checksumOff += checksumSize; len -= chunkLen; } } + + + /** + * Translate CRC chunks from the client's checksum implementation + * to the disk checksum implementation. + * + * This does not verify the original checksums, under the assumption + * that they have already been validated. + */ + private void translateChunks( byte[] dataBuf, int dataOff, int len, + byte[] checksumBuf, int checksumOff ) + throws IOException { + if (len == 0) return; + + int numChunks = (len - 1)/bytesPerChecksum + 1; + + diskChecksum.calculateChunkedSums( + ByteBuffer.wrap(dataBuf, dataOff, len), + ByteBuffer.wrap(checksumBuf, checksumOff, numChunks * checksumSize)); + } /** * Makes sure buf.position() is zero without modifying buf.remaining(). @@ -583,9 +613,16 @@ private int receivePacket(long offsetInBlock, long seqno, * protocol includes acks and only the last datanode needs to verify * checksum. */ - if (mirrorOut == null || isDatanode) { + if (mirrorOut == null || isDatanode || needsChecksumTranslation) { verifyChunks(pktBuf, dataOff, len, pktBuf, checksumOff); + if (needsChecksumTranslation) { + // overwrite the checksums in the packet buffer with the + // appropriate polynomial for the disk storage. + translateChunks(pktBuf, dataOff, len, pktBuf, checksumOff); + } } + + // by this point, the data in the buffer uses the disk checksum byte[] lastChunkChecksum; @@ -807,7 +844,7 @@ private void computePartialChunkCrc(long blkoff, long ckoff, // find offset of the beginning of partial chunk. // int sizePartialChunk = (int) (blkoff % bytesPerChecksum); - int checksumSize = checksum.getChecksumSize(); + int checksumSize = diskChecksum.getChecksumSize(); blkoff = blkoff - sizePartialChunk; LOG.info("computePartialChunkCrc sizePartialChunk " + sizePartialChunk + @@ -832,7 +869,8 @@ private void computePartialChunkCrc(long blkoff, long ckoff, } // compute crc of partial chunk from data read in the block file. - partialCrc = new PureJavaCrc32(); + partialCrc = DataChecksum.newDataChecksum( + diskChecksum.getChecksumType(), diskChecksum.getBytesPerChecksum()); partialCrc.update(buf, 0, sizePartialChunk); LOG.info("Read in partial CRC chunk from disk for block " + block); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java index 6217bb6fc4..d017679e0a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java @@ -124,7 +124,7 @@ static void generateDirectoryStructure(JspWriter out, if (locations == null || locations.length == 0) { out.print("Empty file"); } else { - DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock); + DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock, conf); String fqdn = InetAddress.getByName(chosenNode.getHost()) .getCanonicalHostName(); String datanodeAddr = chosenNode.getName(); @@ -299,7 +299,7 @@ static void generateFileDetails(JspWriter out, // URL for TAIL LocatedBlock lastBlk = blocks.get(blocks.size() - 1); try { - chosenNode = JspHelper.bestNode(lastBlk); + chosenNode = JspHelper.bestNode(lastBlk, conf); } catch (IOException e) { out.print(e.toString()); dfs.close(); @@ -514,7 +514,7 @@ static void generateFileChunks(JspWriter out, HttpServletRequest req, .getGenerationStamp()); nextStartOffset = 0; nextBlockSize = nextBlock.getBlock().getNumBytes(); - DatanodeInfo d = JspHelper.bestNode(nextBlock); + DatanodeInfo d = JspHelper.bestNode(nextBlock, conf); String datanodeAddr = d.getName(); nextDatanodePort = Integer.parseInt(datanodeAddr.substring( datanodeAddr.indexOf(':') + 1, datanodeAddr.length())); @@ -569,7 +569,7 @@ static void generateFileChunks(JspWriter out, HttpServletRequest req, if (prevStartOffset < 0) prevStartOffset = 0; prevBlockSize = prevBlock.getBlock().getNumBytes(); - DatanodeInfo d = JspHelper.bestNode(prevBlock); + DatanodeInfo d = JspHelper.bestNode(prevBlock, conf); String datanodeAddr = d.getName(); prevDatanodePort = Integer.parseInt(datanodeAddr.substring( datanodeAddr.indexOf(':') + 1, datanodeAddr.length())); @@ -686,7 +686,7 @@ static void generateFileChunksForTail(JspWriter out, HttpServletRequest req, long genStamp = lastBlk.getBlock().getGenerationStamp(); DatanodeInfo chosenNode; try { - chosenNode = JspHelper.bestNode(lastBlk); + chosenNode = JspHelper.bestNode(lastBlk, conf); } catch (IOException e) { out.print(e.toString()); dfs.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java index 512d0b64bb..e309dc1f47 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java @@ -1258,7 +1258,7 @@ public File getBlockFile(ExtendedBlock b) throws IOException { /** * Get File name for a given block. */ - public synchronized File getBlockFile(String bpid, Block b) + public File getBlockFile(String bpid, Block b) throws IOException { File f = validateBlockFile(bpid, b); if(f == null) { @@ -1271,16 +1271,44 @@ public synchronized File getBlockFile(String bpid, Block b) } @Override // FSDatasetInterface - public synchronized InputStream getBlockInputStream(ExtendedBlock b) + public InputStream getBlockInputStream(ExtendedBlock b) throws IOException { - return new FileInputStream(getBlockFile(b)); + File f = getBlockFileNoExistsCheck(b); + try { + return new FileInputStream(f); + } catch (FileNotFoundException fnfe) { + throw new IOException("Block " + b + " is not valid. " + + "Expected block file at " + f + " does not exist."); + } + } + + /** + * Return the File associated with a block, without first + * checking that it exists. This should be used when the + * next operation is going to open the file for read anyway, + * and thus the exists check is redundant. + */ + private File getBlockFileNoExistsCheck(ExtendedBlock b) + throws IOException { + File f = getFile(b.getBlockPoolId(), b.getLocalBlock()); + if (f == null) { + throw new IOException("Block " + b + " is not valid"); + } + return f; } @Override // FSDatasetInterface - public synchronized InputStream getBlockInputStream(ExtendedBlock b, + public InputStream getBlockInputStream(ExtendedBlock b, long seekOffset) throws IOException { - File blockFile = getBlockFile(b); - RandomAccessFile blockInFile = new RandomAccessFile(blockFile, "r"); + File blockFile = getBlockFileNoExistsCheck(b); + RandomAccessFile blockInFile; + try { + blockInFile = new RandomAccessFile(blockFile, "r"); + } catch (FileNotFoundException fnfe) { + throw new IOException("Block " + b + " is not valid. " + + "Expected block file at " + blockFile + " does not exist."); + } + if (seekOffset > 0) { blockInFile.seek(seekOffset); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java index 867eebe5b5..38017cfdb8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DiskChecker.DiskErrorException; /** @@ -158,15 +159,23 @@ public BlockInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff, static class BlockWriteStreams { OutputStream dataOut; OutputStream checksumOut; - BlockWriteStreams(OutputStream dOut, OutputStream cOut) { + DataChecksum checksum; + + BlockWriteStreams(OutputStream dOut, OutputStream cOut, + DataChecksum checksum) { dataOut = dOut; checksumOut = cOut; + this.checksum = checksum; } void close() throws IOException { IOUtils.closeStream(dataOut); IOUtils.closeStream(checksumOut); } + + DataChecksum getChecksum() { + return checksum; + } } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java index 447b9337ce..14c1258fe4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.datanode; +import java.io.DataInputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -169,7 +170,7 @@ public int hashCode() { @Override // ReplicaInPipelineInterface public BlockWriteStreams createStreams(boolean isCreate, - int bytesPerChunk, int checksumSize) throws IOException { + DataChecksum requestedChecksum) throws IOException { File blockFile = getBlockFile(); File metaFile = getMetaFile(); if (DataNode.LOG.isDebugEnabled()) { @@ -180,30 +181,64 @@ public BlockWriteStreams createStreams(boolean isCreate, } long blockDiskSize = 0L; long crcDiskSize = 0L; - if (!isCreate) { // check on disk file - blockDiskSize = bytesOnDisk; - crcDiskSize = BlockMetadataHeader.getHeaderSize() + - (blockDiskSize+bytesPerChunk-1)/bytesPerChunk*checksumSize; - if (blockDiskSize>0 && - (blockDiskSize>blockFile.length() || crcDiskSize>metaFile.length())) { - throw new IOException("Corrupted block: " + this); + + // the checksum that should actually be used -- this + // may differ from requestedChecksum for appends. + DataChecksum checksum; + + RandomAccessFile metaRAF = new RandomAccessFile(metaFile, "rw"); + + if (!isCreate) { + // For append or recovery, we must enforce the existing checksum. + // Also, verify that the file has correct lengths, etc. + boolean checkedMeta = false; + try { + BlockMetadataHeader header = BlockMetadataHeader.readHeader(metaRAF); + checksum = header.getChecksum(); + + if (checksum.getBytesPerChecksum() != + requestedChecksum.getBytesPerChecksum()) { + throw new IOException("Client requested checksum " + + requestedChecksum + " when appending to an existing block " + + "with different chunk size: " + checksum); + } + + int bytesPerChunk = checksum.getBytesPerChecksum(); + int checksumSize = checksum.getChecksumSize(); + + blockDiskSize = bytesOnDisk; + crcDiskSize = BlockMetadataHeader.getHeaderSize() + + (blockDiskSize+bytesPerChunk-1)/bytesPerChunk*checksumSize; + if (blockDiskSize>0 && + (blockDiskSize>blockFile.length() || crcDiskSize>metaFile.length())) { + throw new IOException("Corrupted block: " + this); + } + checkedMeta = true; + } finally { + if (!checkedMeta) { + // clean up in case of exceptions. + IOUtils.closeStream(metaRAF); + } } + } else { + // for create, we can use the requested checksum + checksum = requestedChecksum; } + FileOutputStream blockOut = null; FileOutputStream crcOut = null; try { blockOut = new FileOutputStream( new RandomAccessFile( blockFile, "rw" ).getFD() ); - crcOut = new FileOutputStream( - new RandomAccessFile( metaFile, "rw" ).getFD() ); + crcOut = new FileOutputStream(metaRAF.getFD() ); if (!isCreate) { blockOut.getChannel().position(blockDiskSize); crcOut.getChannel().position(crcDiskSize); } - return new BlockWriteStreams(blockOut, crcOut); + return new BlockWriteStreams(blockOut, crcOut, checksum); } catch (IOException e) { IOUtils.closeStream(blockOut); - IOUtils.closeStream(crcOut); + IOUtils.closeStream(metaRAF); throw e; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java index 1acc76f437..17eefa98da 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.BlockWriteStreams; +import org.apache.hadoop.util.DataChecksum; /** * This defines the interface of a replica in Pipeline that's being written to @@ -61,11 +62,10 @@ interface ReplicaInPipelineInterface extends Replica { * one for block file and one for CRC file * * @param isCreate if it is for creation - * @param bytePerChunk number of bytes per CRC chunk - * @param checksumSize number of bytes per checksum + * @param requestedChecksum the checksum the writer would prefer to use * @return output streams for writing * @throws IOException if any error occurs */ public BlockWriteStreams createStreams(boolean isCreate, - int bytesPerChunk, int checksumSize) throws IOException; + DataChecksum requestedChecksum) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java index e8c00ca005..0ecf5fadba 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream; +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.web.JsonUtil; @@ -58,7 +59,9 @@ import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; import org.apache.hadoop.hdfs.web.resources.BufferSizeParam; +import org.apache.hadoop.hdfs.web.resources.DelegationParam; import org.apache.hadoop.hdfs.web.resources.GetOpParam; +import org.apache.hadoop.hdfs.web.resources.HttpOpParam; import org.apache.hadoop.hdfs.web.resources.LengthParam; import org.apache.hadoop.hdfs.web.resources.OffsetParam; import org.apache.hadoop.hdfs.web.resources.OverwriteParam; @@ -69,7 +72,9 @@ import org.apache.hadoop.hdfs.web.resources.ReplicationParam; import org.apache.hadoop.hdfs.web.resources.UriFsPathParam; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; import com.sun.jersey.spi.container.ResourceFilters; @@ -84,6 +89,29 @@ public class DatanodeWebHdfsMethods { private @Context ServletContext context; private @Context HttpServletResponse response; + private void init(final UserGroupInformation ugi, final DelegationParam delegation, + final UriFsPathParam path, final HttpOpParam op, + final Param... parameters) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("HTTP " + op.getValue().getType() + ": " + op + ", " + path + + ", ugi=" + ugi + Param.toSortedString(", ", parameters)); + } + + //clear content type + response.setContentType(null); + + if (UserGroupInformation.isSecurityEnabled()) { + //add a token for RPC. + final DataNode datanode = (DataNode)context.getAttribute("datanode"); + final InetSocketAddress nnRpcAddr = NameNode.getAddress(datanode.getConf()); + final Token token = new Token(); + token.decodeFromUrlString(delegation.getValue()); + SecurityUtil.setTokenService(token, nnRpcAddr); + token.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND); + ugi.addToken(token); + } + } + /** Handle HTTP PUT request for the root. */ @PUT @Path("/") @@ -92,6 +120,8 @@ public class DatanodeWebHdfsMethods { public Response putRoot( final InputStream in, @Context final UserGroupInformation ugi, + @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) + final DelegationParam delegation, @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT) final PutOpParam op, @QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT) @@ -105,7 +135,7 @@ public Response putRoot( @QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT) final BlockSizeParam blockSize ) throws IOException, InterruptedException { - return put(in, ugi, ROOT, op, permission, overwrite, bufferSize, + return put(in, ugi, delegation, ROOT, op, permission, overwrite, bufferSize, replication, blockSize); } @@ -117,6 +147,8 @@ public Response putRoot( public Response put( final InputStream in, @Context final UserGroupInformation ugi, + @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) + final DelegationParam delegation, @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT) final PutOpParam op, @@ -132,14 +164,8 @@ public Response put( final BlockSizeParam blockSize ) throws IOException, InterruptedException { - if (LOG.isTraceEnabled()) { - LOG.trace(op + ": " + path + ", ugi=" + ugi - + Param.toSortedString(", ", permission, overwrite, bufferSize, - replication, blockSize)); - } - - //clear content type - response.setContentType(null); + init(ugi, delegation, path, op, permission, overwrite, bufferSize, + replication, blockSize); return ugi.doAs(new PrivilegedExceptionAction() { @Override @@ -193,12 +219,14 @@ public Response run() throws IOException, URISyntaxException { public Response postRoot( final InputStream in, @Context final UserGroupInformation ugi, + @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) + final DelegationParam delegation, @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT) final PostOpParam op, @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) final BufferSizeParam bufferSize ) throws IOException, InterruptedException { - return post(in, ugi, ROOT, op, bufferSize); + return post(in, ugi, delegation, ROOT, op, bufferSize); } /** Handle HTTP POST request. */ @@ -209,6 +237,8 @@ public Response postRoot( public Response post( final InputStream in, @Context final UserGroupInformation ugi, + @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) + final DelegationParam delegation, @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT) final PostOpParam op, @@ -216,13 +246,7 @@ public Response post( final BufferSizeParam bufferSize ) throws IOException, InterruptedException { - if (LOG.isTraceEnabled()) { - LOG.trace(op + ": " + path + ", ugi=" + ugi - + Param.toSortedString(", ", bufferSize)); - } - - //clear content type - response.setContentType(null); + init(ugi, delegation, path, op, bufferSize); return ugi.doAs(new PrivilegedExceptionAction() { @Override @@ -265,6 +289,8 @@ public Response run() throws IOException { @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON}) public Response getRoot( @Context final UserGroupInformation ugi, + @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) + final DelegationParam delegation, @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT) final GetOpParam op, @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT) @@ -274,7 +300,7 @@ public Response getRoot( @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) final BufferSizeParam bufferSize ) throws IOException, InterruptedException { - return get(ugi, ROOT, op, offset, length, bufferSize); + return get(ugi, delegation, ROOT, op, offset, length, bufferSize); } /** Handle HTTP GET request. */ @@ -283,6 +309,8 @@ public Response getRoot( @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON}) public Response get( @Context final UserGroupInformation ugi, + @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) + final DelegationParam delegation, @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT) final GetOpParam op, @@ -294,13 +322,7 @@ public Response get( final BufferSizeParam bufferSize ) throws IOException, InterruptedException { - if (LOG.isTraceEnabled()) { - LOG.trace(op + ": " + path + ", ugi=" + ugi - + Param.toSortedString(", ", offset, length, bufferSize)); - } - - //clear content type - response.setContentType(null); + init(ugi, delegation, path, op, offset, length, bufferSize); return ugi.doAs(new PrivilegedExceptionAction() { @Override @@ -350,9 +372,7 @@ public void write(final OutputStream out) throws IOException { } }; - final int status = offset.getValue() == 0? - HttpServletResponse.SC_OK: HttpServletResponse.SC_PARTIAL_CONTENT; - return Response.status(status).entity(streaming).type( + return Response.ok(streaming).type( MediaType.APPLICATION_OCTET_STREAM).build(); } case GETFILECHECKSUM: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 95a343daec..70680e3649 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -4050,7 +4050,7 @@ public String toString() { * @throws IOException */ Collection listCorruptFileBlocks(String path, - String startBlockAfter) throws IOException { + String[] cookieTab) throws IOException { readLock(); try { @@ -4059,23 +4059,27 @@ Collection listCorruptFileBlocks(String path, "replication queues have not been initialized."); } checkSuperuserPrivilege(); - long startBlockId = 0; // print a limited # of corrupt files per call int count = 0; ArrayList corruptFiles = new ArrayList(); - - if (startBlockAfter != null) { - startBlockId = Block.filename2id(startBlockAfter); - } final Iterator blkIterator = blockManager.getCorruptReplicaBlockIterator(); + + if (cookieTab == null) { + cookieTab = new String[] { null }; + } + int skip = getIntCookie(cookieTab[0]); + for (int i = 0; i < skip && blkIterator.hasNext(); i++) { + blkIterator.next(); + } + while (blkIterator.hasNext()) { Block blk = blkIterator.next(); INode inode = blockManager.getINode(blk); + skip++; if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) { String src = FSDirectory.getFullPathName(inode); - if (((startBlockAfter == null) || (blk.getBlockId() > startBlockId)) - && (src.startsWith(path))) { + if (src.startsWith(path)){ corruptFiles.add(new CorruptFileBlockInfo(src, blk)); count++; if (count >= DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED) @@ -4083,13 +4087,32 @@ Collection listCorruptFileBlocks(String path, } } } + cookieTab[0] = String.valueOf(skip); LOG.info("list corrupt file blocks returned: " + count); return corruptFiles; } finally { readUnlock(); } } - + + /** + * Convert string cookie to integer. + */ + private static int getIntCookie(String cookie){ + int c; + if(cookie == null){ + c = 0; + } else { + try{ + c = Integer.parseInt(cookie); + }catch (NumberFormatException e) { + c = 0; + } + } + c = Math.max(0, c); + return c; + } + /** * Create delegation token secret manager */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java index 54b0ec64ab..3e630661e9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java @@ -52,7 +52,9 @@ private URL createRedirectURL(String path, String encodedPath, HdfsFileStatus st String scheme = request.getScheme(); final LocatedBlocks blks = nnproxy.getBlockLocations( status.getFullPath(new Path(path)).toUri().getPath(), 0, 1); - final DatanodeID host = pickSrcDatanode(blks, status); + final Configuration conf = NameNodeHttpServer.getConfFromContext( + getServletContext()); + final DatanodeID host = pickSrcDatanode(blks, status, conf); final String hostname; if (host instanceof DatanodeInfo) { hostname = ((DatanodeInfo)host).getHostName(); @@ -83,16 +85,17 @@ private URL createRedirectURL(String path, String encodedPath, HdfsFileStatus st /** Select a datanode to service this request. * Currently, this looks at no more than the first five blocks of a file, * selecting a datanode randomly from the most represented. + * @param conf */ - private DatanodeID pickSrcDatanode(LocatedBlocks blks, HdfsFileStatus i) - throws IOException { + private DatanodeID pickSrcDatanode(LocatedBlocks blks, HdfsFileStatus i, + Configuration conf) throws IOException { if (i.getLen() == 0 || blks.getLocatedBlocks().size() <= 0) { // pick a random datanode NameNode nn = NameNodeHttpServer.getNameNodeFromContext( getServletContext()); return NamenodeJspHelper.getRandomDatanode(nn); } - return JspHelper.bestNode(blks); + return JspHelper.bestNode(blks, conf); } /** 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 97fce223ee..2a2318cb6e 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 @@ -744,17 +744,16 @@ public void metaSave(String filename) throws IOException { public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie) throws IOException { nn.checkOperation(OperationCategory.READ); + String[] cookieTab = new String[] { cookie }; Collection fbs = - namesystem.listCorruptFileBlocks(path, cookie); - + namesystem.listCorruptFileBlocks(path, cookieTab); + String[] files = new String[fbs.size()]; - String lastCookie = ""; int i = 0; for(FSNamesystem.CorruptFileBlockInfo fb: fbs) { files[i++] = fb.path; - lastCookie = fb.block.getBlockName(); } - return new CorruptFileBlocks(files, lastCookie); + return new CorruptFileBlocks(files, cookieTab[0]); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java index 358d778eaf..74f8b2d5b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java @@ -114,11 +114,11 @@ public class NamenodeFsck { // We return back N files that are corrupt; the list of files returned is // ordered by block id; to allow continuation support, pass in the last block // # from previous call - private String startBlockAfter = null; - + private String[] currentCookie = new String[] { null }; + private final Configuration conf; private final PrintWriter out; - + /** * Filesystem checker. * @param conf configuration (namenode config) @@ -156,11 +156,11 @@ else if (key.equals("listcorruptfileblocks")) { this.showCorruptFileBlocks = true; } else if (key.equals("startblockafter")) { - this.startBlockAfter = pmap.get("startblockafter")[0]; + this.currentCookie[0] = pmap.get("startblockafter")[0]; } } } - + /** * Check files on DFS, starting from the indicated path. */ @@ -216,19 +216,20 @@ public void fsck() { out.close(); } } - + private void listCorruptFileBlocks() throws IOException { Collection corruptFiles = namenode. - getNamesystem().listCorruptFileBlocks(path, startBlockAfter); + getNamesystem().listCorruptFileBlocks(path, currentCookie); int numCorruptFiles = corruptFiles.size(); String filler; if (numCorruptFiles > 0) { filler = Integer.toString(numCorruptFiles); - } else if (startBlockAfter == null) { + } else if (currentCookie[0].equals("0")) { filler = "no"; } else { filler = "no more"; } + out.println("Cookie:\t" + currentCookie[0]); for (FSNamesystem.CorruptFileBlockInfo c : corruptFiles) { out.println(c.toString()); } @@ -509,8 +510,9 @@ private void copyBlock(DFSClient dfs, LocatedBlock lblock, String file = BlockReaderFactory.getFileName(targetAddr, block.getBlockPoolId(), block.getBlockId()); - blockReader = BlockReaderFactory.newBlockReader(s, file, block, lblock - .getBlockToken(), 0, -1, conf.getInt("io.file.buffer.size", 4096)); + blockReader = BlockReaderFactory.newBlockReader( + conf, s, file, block, lblock + .getBlockToken(), 0, -1); } catch (IOException ex) { // Put chosen node into dead list, continue diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java index 46ea367cb9..2af0aad1ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java @@ -42,14 +42,13 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.ResponseBuilder; -import javax.ws.rs.core.Response.Status; import javax.ws.rs.core.StreamingOutput; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Options; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; @@ -68,9 +67,9 @@ import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; import org.apache.hadoop.hdfs.web.resources.BufferSizeParam; import org.apache.hadoop.hdfs.web.resources.DelegationParam; -import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam; import org.apache.hadoop.hdfs.web.resources.DeleteOpParam; import org.apache.hadoop.hdfs.web.resources.DestinationParam; +import org.apache.hadoop.hdfs.web.resources.DoAsParam; import org.apache.hadoop.hdfs.web.resources.GetOpParam; import org.apache.hadoop.hdfs.web.resources.GroupParam; import org.apache.hadoop.hdfs.web.resources.HttpOpParam; @@ -87,6 +86,7 @@ import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam; import org.apache.hadoop.hdfs.web.resources.RenewerParam; import org.apache.hadoop.hdfs.web.resources.ReplicationParam; +import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam; import org.apache.hadoop.hdfs.web.resources.UriFsPathParam; import org.apache.hadoop.hdfs.web.resources.UserParam; import org.apache.hadoop.net.NodeBase; @@ -117,9 +117,24 @@ public static String getRemoteAddress() { private @Context HttpServletRequest request; private @Context HttpServletResponse response; + private void init(final UserGroupInformation ugi, + final DelegationParam delegation, + final UserParam username, final DoAsParam doAsUser, + final UriFsPathParam path, final HttpOpParam op, + final Param... parameters) throws IOException { + if (LOG.isTraceEnabled()) { + LOG.trace("HTTP " + op.getValue().getType() + ": " + op + ", " + path + + ", ugi=" + ugi + ", " + username + ", " + doAsUser + + Param.toSortedString(", ", parameters)); + } + + //clear content type + response.setContentType(null); + } + private static DatanodeInfo chooseDatanode(final NameNode namenode, - final String path, final HttpOpParam.Op op, final long openOffset - ) throws IOException { + final String path, final HttpOpParam.Op op, final long openOffset, + Configuration conf) throws IOException { if (op == GetOpParam.Op.OPEN || op == GetOpParam.Op.GETFILECHECKSUM || op == PostOpParam.Op.APPEND) { @@ -139,7 +154,7 @@ private static DatanodeInfo chooseDatanode(final NameNode namenode, final LocatedBlocks locations = np.getBlockLocations(path, offset, 1); final int count = locations.locatedBlockCount(); if (count > 0) { - return JspHelper.bestNode(locations.get(0)); + return JspHelper.bestNode(locations.get(0), conf); } } } @@ -153,24 +168,25 @@ private Token generateDelegationToken( final NameNode namenode, final UserGroupInformation ugi, final String renewer) throws IOException { final Credentials c = DelegationTokenSecretManager.createCredentials( - namenode, ugi, - renewer != null? renewer: request.getUserPrincipal().getName()); + namenode, ugi, renewer != null? renewer: ugi.getShortUserName()); final Token t = c.getAllTokens().iterator().next(); t.setKind(WebHdfsFileSystem.TOKEN_KIND); - SecurityUtil.setTokenService(t, namenode.getNameNodeAddress()); + SecurityUtil.setTokenService(t, namenode.getHttpAddress()); return t; } private URI redirectURI(final NameNode namenode, final UserGroupInformation ugi, final DelegationParam delegation, + final UserParam username, final DoAsParam doAsUser, final String path, final HttpOpParam.Op op, final long openOffset, final Param... parameters) throws URISyntaxException, IOException { - final DatanodeInfo dn = chooseDatanode(namenode, path, op, openOffset); + final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF); + final DatanodeInfo dn = chooseDatanode(namenode, path, op, openOffset, conf); final String delegationQuery; if (!UserGroupInformation.isSecurityEnabled()) { //security disabled - delegationQuery = ""; + delegationQuery = Param.toSortedString("&", doAsUser, username); } else if (delegation.getValue() != null) { //client has provided a token delegationQuery = "&" + delegation; @@ -180,8 +196,7 @@ private URI redirectURI(final NameNode namenode, namenode, ugi, request.getUserPrincipal().getName()); delegationQuery = "&" + new DelegationParam(t.encodeToUrlString()); } - final String query = op.toQueryString() - + '&' + new UserParam(ugi) + delegationQuery + final String query = op.toQueryString() + delegationQuery + Param.toSortedString("&", parameters); final String uripath = WebHdfsFileSystem.PATH_PREFIX + path; @@ -202,6 +217,10 @@ public Response putRoot( @Context final UserGroupInformation ugi, @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) final DelegationParam delegation, + @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT) + final UserParam username, + @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) + final DoAsParam doAsUser, @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT) final PutOpParam op, @QueryParam(DestinationParam.NAME) @DefaultValue(DestinationParam.DEFAULT) @@ -226,12 +245,13 @@ public Response putRoot( final AccessTimeParam accessTime, @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT) final RenameOptionSetParam renameOptions, - @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT) + @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT) final TokenArgumentParam delegationTokenArgument ) throws IOException, InterruptedException { - return put(ugi, delegation, ROOT, op, destination, owner, group, - permission, overwrite, bufferSize, replication, blockSize, - modificationTime, accessTime, renameOptions, delegationTokenArgument); + return put(ugi, delegation, username, doAsUser, ROOT, op, destination, + owner, group, permission, overwrite, bufferSize, replication, + blockSize, modificationTime, accessTime, renameOptions, + delegationTokenArgument); } /** Handle HTTP PUT request. */ @@ -243,6 +263,10 @@ public Response put( @Context final UserGroupInformation ugi, @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) final DelegationParam delegation, + @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT) + final UserParam username, + @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) + final DoAsParam doAsUser, @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT) final PutOpParam op, @@ -268,19 +292,13 @@ public Response put( final AccessTimeParam accessTime, @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT) final RenameOptionSetParam renameOptions, - @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT) + @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT) final TokenArgumentParam delegationTokenArgument ) throws IOException, InterruptedException { - if (LOG.isTraceEnabled()) { - LOG.trace(op + ": " + path + ", ugi=" + ugi - + Param.toSortedString(", ", destination, owner, group, permission, - overwrite, bufferSize, replication, blockSize, - modificationTime, accessTime, renameOptions)); - } - - //clear content type - response.setContentType(null); + init(ugi, delegation, username, doAsUser, path, op, destination, owner, + group, permission, overwrite, bufferSize, replication, blockSize, + modificationTime, accessTime, renameOptions, delegationTokenArgument); return ugi.doAs(new PrivilegedExceptionAction() { @Override @@ -296,8 +314,8 @@ public Response run() throws IOException, URISyntaxException { switch(op.getValue()) { case CREATE: { - final URI uri = redirectURI(namenode, ugi, delegation, fullpath, - op.getValue(), -1L, + final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser, + fullpath, op.getValue(), -1L, permission, overwrite, bufferSize, replication, blockSize); return Response.temporaryRedirect(uri).build(); } @@ -324,8 +342,7 @@ public Response run() throws IOException, URISyntaxException { { final boolean b = np.setReplication(fullpath, replication.getValue(conf)); final String js = JsonUtil.toJsonString("boolean", b); - final ResponseBuilder r = b? Response.ok(): Response.status(Status.FORBIDDEN); - return r.entity(js).type(MediaType.APPLICATION_JSON).build(); + return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); } case SETOWNER: { @@ -381,12 +398,16 @@ public Response postRoot( @Context final UserGroupInformation ugi, @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) final DelegationParam delegation, + @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT) + final UserParam username, + @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) + final DoAsParam doAsUser, @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT) final PostOpParam op, @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) final BufferSizeParam bufferSize ) throws IOException, InterruptedException { - return post(ugi, delegation, ROOT, op, bufferSize); + return post(ugi, delegation, username, doAsUser, ROOT, op, bufferSize); } /** Handle HTTP POST request. */ @@ -398,6 +419,10 @@ public Response post( @Context final UserGroupInformation ugi, @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) final DelegationParam delegation, + @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT) + final UserParam username, + @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) + final DoAsParam doAsUser, @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT) final PostOpParam op, @@ -405,13 +430,7 @@ public Response post( final BufferSizeParam bufferSize ) throws IOException, InterruptedException { - if (LOG.isTraceEnabled()) { - LOG.trace(op + ": " + path + ", ugi=" + ugi - + Param.toSortedString(", ", bufferSize)); - } - - //clear content type - response.setContentType(null); + init(ugi, delegation, username, doAsUser, path, op, bufferSize); return ugi.doAs(new PrivilegedExceptionAction() { @Override @@ -425,8 +444,8 @@ public Response run() throws IOException, URISyntaxException { switch(op.getValue()) { case APPEND: { - final URI uri = redirectURI(namenode, ugi, delegation, fullpath, - op.getValue(), -1L, bufferSize); + final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser, + fullpath, op.getValue(), -1L, bufferSize); return Response.temporaryRedirect(uri).build(); } default: @@ -448,6 +467,10 @@ public Response getRoot( @Context final UserGroupInformation ugi, @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) final DelegationParam delegation, + @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT) + final UserParam username, + @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) + final DoAsParam doAsUser, @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT) final GetOpParam op, @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT) @@ -459,7 +482,8 @@ public Response getRoot( @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT) final BufferSizeParam bufferSize ) throws IOException, URISyntaxException, InterruptedException { - return get(ugi, delegation, ROOT, op, offset, length, renewer, bufferSize); + return get(ugi, delegation, username, doAsUser, ROOT, op, + offset, length, renewer, bufferSize); } /** Handle HTTP GET request. */ @@ -470,6 +494,10 @@ public Response get( @Context final UserGroupInformation ugi, @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) final DelegationParam delegation, + @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT) + final UserParam username, + @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) + final DoAsParam doAsUser, @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT) final GetOpParam op, @@ -483,13 +511,8 @@ public Response get( final BufferSizeParam bufferSize ) throws IOException, InterruptedException { - if (LOG.isTraceEnabled()) { - LOG.trace(op + ": " + path + ", ugi=" + ugi - + Param.toSortedString(", ", offset, length, renewer, bufferSize)); - } - - //clear content type - response.setContentType(null); + init(ugi, delegation, username, doAsUser, path, op, + offset, length, renewer, bufferSize); return ugi.doAs(new PrivilegedExceptionAction() { @Override @@ -504,11 +527,11 @@ public Response run() throws IOException, URISyntaxException { switch(op.getValue()) { case OPEN: { - final URI uri = redirectURI(namenode, ugi, delegation, fullpath, - op.getValue(), offset.getValue(), offset, length, bufferSize); + final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser, + fullpath, op.getValue(), offset.getValue(), offset, length, bufferSize); return Response.temporaryRedirect(uri).build(); } - case GETFILEBLOCKLOCATIONS: + case GET_BLOCK_LOCATIONS: { final long offsetValue = offset.getValue(); final Long lengthValue = length.getValue(); @@ -540,17 +563,28 @@ public Response run() throws IOException, URISyntaxException { } case GETFILECHECKSUM: { - final URI uri = redirectURI(namenode, ugi, delegation, fullpath, - op.getValue(), -1L); + final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser, + fullpath, op.getValue(), -1L); return Response.temporaryRedirect(uri).build(); } case GETDELEGATIONTOKEN: { + if (delegation.getValue() != null) { + throw new IllegalArgumentException(delegation.getName() + + " parameter is not null."); + } final Token token = generateDelegationToken( namenode, ugi, renewer.getValue()); final String js = JsonUtil.toJsonString(token); return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); } + case GETHOMEDIRECTORY: + { + final String js = JsonUtil.toJsonString( + org.apache.hadoop.fs.Path.class.getSimpleName(), + WebHdfsFileSystem.getHomeDirectoryString(ugi)); + return Response.ok(js).type(MediaType.APPLICATION_JSON).build(); + } default: throw new UnsupportedOperationException(op + " is not supported"); } @@ -580,8 +614,8 @@ private static StreamingOutput getListingStream(final NamenodeProtocols np, @Override public void write(final OutputStream outstream) throws IOException { final PrintStream out = new PrintStream(outstream); - out.println("{\"" + HdfsFileStatus.class.getSimpleName() + "es\":{\"" - + HdfsFileStatus.class.getSimpleName() + "\":["); + out.println("{\"" + FileStatus.class.getSimpleName() + "es\":{\"" + + FileStatus.class.getSimpleName() + "\":["); final HdfsFileStatus[] partial = first.getPartialListing(); if (partial.length > 0) { @@ -612,12 +646,18 @@ public void write(final OutputStream outstream) throws IOException { @Produces(MediaType.APPLICATION_JSON) public Response deleteRoot( @Context final UserGroupInformation ugi, + @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) + final DelegationParam delegation, + @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT) + final UserParam username, + @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) + final DoAsParam doAsUser, @QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT) final DeleteOpParam op, @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT) final RecursiveParam recursive ) throws IOException, InterruptedException { - return delete(ugi, ROOT, op, recursive); + return delete(ugi, delegation, username, doAsUser, ROOT, op, recursive); } /** Handle HTTP DELETE request. */ @@ -626,6 +666,12 @@ public Response deleteRoot( @Produces(MediaType.APPLICATION_JSON) public Response delete( @Context final UserGroupInformation ugi, + @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT) + final DelegationParam delegation, + @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT) + final UserParam username, + @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT) + final DoAsParam doAsUser, @PathParam(UriFsPathParam.NAME) final UriFsPathParam path, @QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT) final DeleteOpParam op, @@ -633,13 +679,7 @@ public Response delete( final RecursiveParam recursive ) throws IOException, InterruptedException { - if (LOG.isTraceEnabled()) { - LOG.trace(op + ": " + path + ", ugi=" + ugi - + Param.toSortedString(", ", recursive)); - } - - //clear content type - response.setContentType(null); + init(ugi, delegation, username, doAsUser, path, op, recursive); return ugi.doAs(new PrivilegedExceptionAction() { @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java index 93e0be8bf9..b2a84a956a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java @@ -145,14 +145,15 @@ private Integer listCorruptFileBlocks(String dir, String baseUrl) throws IOException { int errCode = -1; int numCorrupt = 0; - String lastBlock = null; + int cookie = 0; final String noCorruptLine = "has no CORRUPT files"; final String noMoreCorruptLine = "has no more CORRUPT files"; + final String cookiePrefix = "Cookie:"; boolean allDone = false; while (!allDone) { final StringBuffer url = new StringBuffer(baseUrl); - if (lastBlock != null) { - url.append("&startblockafter=").append(lastBlock); + if (cookie > 0) { + url.append("&startblockafter=").append(String.valueOf(cookie)); } URL path = new URL(url.toString()); SecurityUtil.fetchServiceTicket(path); @@ -163,29 +164,31 @@ private Integer listCorruptFileBlocks(String dir, String baseUrl) try { String line = null; while ((line = input.readLine()) != null) { - if ((line.endsWith(noCorruptLine)) || + if (line.startsWith(cookiePrefix)){ + try{ + cookie = Integer.parseInt(line.split("\t")[1]); + } catch (Exception e){ + allDone = true; + break; + } + continue; + } + if ((line.endsWith(noCorruptLine)) || (line.endsWith(noMoreCorruptLine)) || (line.endsWith(NamenodeFsck.NONEXISTENT_STATUS))) { allDone = true; break; } if ((line.isEmpty()) - || (line.startsWith("FSCK started by")) + || (line.startsWith("FSCK started by")) || (line.startsWith("The filesystem under path"))) continue; numCorrupt++; if (numCorrupt == 1) { - out.println("The list of corrupt files under path '" + out.println("The list of corrupt files under path '" + dir + "' are:"); } out.println(line); - try { - // Get the block # that we need to send in next call - lastBlock = line.split("\t")[0]; - } catch (Exception e) { - allDone = true; - break; - } } } finally { input.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java new file mode 100644 index 0000000000..69b238bbbf --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DirectBufferPool.java @@ -0,0 +1,112 @@ +/** + * 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.hdfs.util; + +import java.lang.ref.WeakReference; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; + +import org.apache.hadoop.classification.InterfaceAudience; + +import com.google.common.annotations.VisibleForTesting; + +/** + * A simple class for pooling direct ByteBuffers. This is necessary + * because Direct Byte Buffers do not take up much space on the heap, + * and hence will not trigger GCs on their own. However, they do take + * native memory, and thus can cause high memory usage if not pooled. + * The pooled instances are referred to only via weak references, allowing + * them to be collected when a GC does run. + * + * This class only does effective pooling when many buffers will be + * allocated at the same size. There is no attempt to reuse larger + * buffers to satisfy smaller allocations. + */ +@InterfaceAudience.Private +public class DirectBufferPool { + + // Essentially implement a multimap with weak values. + ConcurrentMap>> buffersBySize = + new ConcurrentHashMap>>(); + + /** + * Allocate a direct buffer of the specified size, in bytes. + * If a pooled buffer is available, returns that. Otherwise + * allocates a new one. + */ + public ByteBuffer getBuffer(int size) { + Queue> list = buffersBySize.get(size); + if (list == null) { + // no available buffers for this size + return ByteBuffer.allocateDirect(size); + } + + WeakReference ref; + while ((ref = list.poll()) != null) { + ByteBuffer b = ref.get(); + if (b != null) { + return b; + } + } + + return ByteBuffer.allocateDirect(size); + } + + /** + * Return a buffer into the pool. After being returned, + * the buffer may be recycled, so the user must not + * continue to use it in any way. + * @param buf the buffer to return + */ + public void returnBuffer(ByteBuffer buf) { + buf.clear(); // reset mark, limit, etc + int size = buf.capacity(); + Queue> list = buffersBySize.get(size); + if (list == null) { + list = new ConcurrentLinkedQueue>(); + Queue> prev = buffersBySize.putIfAbsent(size, list); + // someone else put a queue in the map before we did + if (prev != null) { + list = prev; + } + } + list.add(new WeakReference(buf)); + } + + /** + * Return the number of available buffers of a given size. + * This is used only for tests. + */ + @VisibleForTesting + int countBuffersOfSize(int size) { + Queue> list = buffersBySize.get(size); + if (list == null) { + return 0; + } + + return list.size(); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java new file mode 100644 index 0000000000..866734f17b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java @@ -0,0 +1,618 @@ +/** + * 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.hdfs.util; + +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.NoSuchElementException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +/** + * A low memory linked hash set implementation, which uses an array for storing + * the elements and linked lists for collision resolution. This class does not + * support null element. + * + * This class is not thread safe. + * + */ +public class LightWeightHashSet implements Collection { + /** + * Elements of {@link LightWeightLinkedSet}. + */ + static class LinkedElement { + protected final T element; + + // reference to the next entry within a bucket linked list + protected LinkedElement next; + + //hashCode of the element + protected final int hashCode; + + public LinkedElement(T elem, int hash) { + this.element = elem; + this.next = null; + this.hashCode = hash; + } + + public String toString() { + return element.toString(); + } + } + + protected static final float DEFAULT_MAX_LOAD_FACTOR = 0.75f; + protected static final float DEFAUT_MIN_LOAD_FACTOR = 0.2f; + protected static final int MINIMUM_CAPACITY = 16; + + static final int MAXIMUM_CAPACITY = 1 << 30; + private static final Log LOG = LogFactory.getLog(LightWeightHashSet.class); + + /** + * An internal array of entries, which are the rows of the hash table. The + * size must be a power of two. + */ + protected LinkedElement[] entries; + /** Size of the entry table. */ + private int capacity; + /** The size of the set (not the entry array). */ + protected int size = 0; + /** Hashmask used for determining the bucket index **/ + private int hash_mask; + /** Capacity at initialization time **/ + private final int initialCapacity; + + /** + * Modification version for fail-fast. + * + * @see ConcurrentModificationException + */ + protected volatile int modification = 0; + + private float maxLoadFactor; + private float minLoadFactor; + private int expandMultiplier = 2; + + private int expandThreshold; + private int shrinkThreshold; + + /** + * @param initCapacity + * Recommended size of the internal array. + * @param maxLoadFactor + * used to determine when to expand the internal array + * @param minLoadFactor + * used to determine when to shrink the internal array + */ + @SuppressWarnings("unchecked") + public LightWeightHashSet(int initCapacity, float maxLoadFactor, + float minLoadFactor) { + + if (maxLoadFactor <= 0 || maxLoadFactor > 1.0f) + throw new IllegalArgumentException("Illegal maxload factor: " + + maxLoadFactor); + + if (minLoadFactor <= 0 || minLoadFactor > maxLoadFactor) + throw new IllegalArgumentException("Illegal minload factor: " + + minLoadFactor); + + this.initialCapacity = computeCapacity(initCapacity); + this.capacity = this.initialCapacity; + this.hash_mask = capacity - 1; + + this.maxLoadFactor = maxLoadFactor; + this.expandThreshold = (int) (capacity * maxLoadFactor); + this.minLoadFactor = minLoadFactor; + this.shrinkThreshold = (int) (capacity * minLoadFactor); + + entries = new LinkedElement[capacity]; + LOG.debug("initial capacity=" + initialCapacity + ", max load factor= " + + maxLoadFactor + ", min load factor= " + minLoadFactor); + } + + public LightWeightHashSet() { + this(MINIMUM_CAPACITY, DEFAULT_MAX_LOAD_FACTOR, DEFAUT_MIN_LOAD_FACTOR); + } + + public LightWeightHashSet(int minCapacity) { + this(minCapacity, DEFAULT_MAX_LOAD_FACTOR, DEFAUT_MIN_LOAD_FACTOR); + } + + /** + * Check if the set is empty. + * + * @return true is set empty, false otherwise + */ + public boolean isEmpty() { + return size == 0; + } + + /** + * Return the current capacity (for testing). + */ + public int getCapacity() { + return capacity; + } + + /** + * Return the number of stored elements. + */ + public int size() { + return size; + } + + /** + * Get index in the internal table for a given hash. + */ + protected int getIndex(int hashCode) { + return hashCode & hash_mask; + } + + /** + * Check if the set contains given element + * + * @return true if element present, false otherwise. + */ + @SuppressWarnings("unchecked") + public boolean contains(final Object key) { + // validate key + if (key == null) { + throw new IllegalArgumentException("Null element is not supported."); + } + // find element + final int hashCode = ((T)key).hashCode(); + final int index = getIndex(hashCode); + return containsElem(index, (T) key, hashCode); + } + + /** + * Check if the set contains given element at given index. + * + * @return true if element present, false otherwise. + */ + protected boolean containsElem(int index, final T key, int hashCode) { + for (LinkedElement e = entries[index]; e != null; e = e.next) { + // element found + if (hashCode == e.hashCode && e.element.equals(key)) { + return true; + } + } + // element not found + return false; + } + + /** + * All all elements in the collection. Expand if necessary. + * + * @param toAdd - elements to add. + * @return true if the set has changed, false otherwise + */ + public boolean addAll(Collection toAdd) { + boolean changed = false; + for (T elem : toAdd) { + changed |= addElem(elem); + } + expandIfNecessary(); + return changed; + } + + /** + * Add given element to the hash table. Expand table if necessary. + * + * @return true if the element was not present in the table, false otherwise + */ + public boolean add(final T element) { + boolean added = addElem(element); + expandIfNecessary(); + return added; + } + + /** + * Add given element to the hash table + * + * @return true if the element was not present in the table, false otherwise + */ + protected boolean addElem(final T element) { + // validate element + if (element == null) { + throw new IllegalArgumentException("Null element is not supported."); + } + // find hashCode & index + final int hashCode = element.hashCode(); + final int index = getIndex(hashCode); + // return false if already present + if (containsElem(index, element, hashCode)) { + return false; + } + + modification++; + size++; + + // update bucket linked list + LinkedElement le = new LinkedElement(element, hashCode); + le.next = entries[index]; + entries[index] = le; + return true; + } + + /** + * Remove the element corresponding to the key. + * + * @return If such element exists, return true. Otherwise, return false. + */ + @SuppressWarnings("unchecked") + public boolean remove(final Object key) { + // validate key + if (key == null) { + throw new IllegalArgumentException("Null element is not supported."); + } + LinkedElement removed = removeElem((T) key); + shrinkIfNecessary(); + return removed == null ? false : true; + } + + /** + * Remove the element corresponding to the key, given key.hashCode() == index. + * + * @return If such element exists, return true. Otherwise, return false. + */ + protected LinkedElement removeElem(final T key) { + LinkedElement found = null; + final int hashCode = key.hashCode(); + final int index = getIndex(hashCode); + if (entries[index] == null) { + return null; + } else if (hashCode == entries[index].hashCode && + entries[index].element.equals(key)) { + // remove the head of the bucket linked list + modification++; + size--; + found = entries[index]; + entries[index] = found.next; + } else { + // head != null and key is not equal to head + // search the element + LinkedElement prev = entries[index]; + for (found = prev.next; found != null;) { + if (hashCode == found.hashCode && + found.element.equals(key)) { + // found the element, remove it + modification++; + size--; + prev.next = found.next; + found.next = null; + break; + } else { + prev = found; + found = found.next; + } + } + } + return found; + } + + /** + * Remove and return n elements from the hashtable. + * The order in which entries are removed is unspecified, and + * and may not correspond to the order in which they were inserted. + * + * @return first element + */ + public List pollN(int n) { + if (n >= size) { + return pollAll(); + } + List retList = new ArrayList(n); + if (n == 0) { + return retList; + } + boolean done = false; + int currentBucketIndex = 0; + + while (!done) { + LinkedElement current = entries[currentBucketIndex]; + while (current != null) { + retList.add(current.element); + current = current.next; + entries[currentBucketIndex] = current; + size--; + modification++; + if (--n == 0) { + done = true; + break; + } + } + currentBucketIndex++; + } + shrinkIfNecessary(); + return retList; + } + + /** + * Remove all elements from the set and return them. Clear the entries. + */ + public List pollAll() { + List retList = new ArrayList(size); + for (int i = 0; i < entries.length; i++) { + LinkedElement current = entries[i]; + while (current != null) { + retList.add(current.element); + current = current.next; + } + } + this.clear(); + return retList; + } + + /** + * Get array.length elements from the set, and put them into the array. + */ + @SuppressWarnings("unchecked") + public T[] pollToArray(T[] array) { + int currentIndex = 0; + LinkedElement current = null; + + if (array.length == 0) { + return array; + } + if (array.length > size) { + array = (T[]) java.lang.reflect.Array.newInstance(array.getClass() + .getComponentType(), size); + } + // do fast polling if the entire set needs to be fetched + if (array.length == size) { + for (int i = 0; i < entries.length; i++) { + current = entries[i]; + while (current != null) { + array[currentIndex++] = current.element; + current = current.next; + } + } + this.clear(); + return array; + } + + boolean done = false; + int currentBucketIndex = 0; + + while (!done) { + current = entries[currentBucketIndex]; + while (current != null) { + array[currentIndex++] = current.element; + current = current.next; + entries[currentBucketIndex] = current; + size--; + modification++; + if (currentIndex == array.length) { + done = true; + break; + } + } + currentBucketIndex++; + } + shrinkIfNecessary(); + return array; + } + + /** + * Compute capacity given initial capacity. + * + * @return final capacity, either MIN_CAPACITY, MAX_CAPACITY, or power of 2 + * closest to the requested capacity. + */ + private int computeCapacity(int initial) { + if (initial < MINIMUM_CAPACITY) { + return MINIMUM_CAPACITY; + } + if (initial > MAXIMUM_CAPACITY) { + return MAXIMUM_CAPACITY; + } + int capacity = 1; + while (capacity < initial) { + capacity <<= 1; + } + return capacity; + } + + /** + * Resize the internal table to given capacity. + */ + @SuppressWarnings("unchecked") + private void resize(int cap) { + int newCapacity = computeCapacity(cap); + if (newCapacity == this.capacity) { + return; + } + this.capacity = newCapacity; + this.expandThreshold = (int) (capacity * maxLoadFactor); + this.shrinkThreshold = (int) (capacity * minLoadFactor); + this.hash_mask = capacity - 1; + LinkedElement[] temp = entries; + entries = new LinkedElement[capacity]; + for (int i = 0; i < temp.length; i++) { + LinkedElement curr = temp[i]; + while (curr != null) { + LinkedElement next = curr.next; + int index = getIndex(curr.hashCode); + curr.next = entries[index]; + entries[index] = curr; + curr = next; + } + } + } + + /** + * Checks if we need to shrink, and shrinks if necessary. + */ + protected void shrinkIfNecessary() { + if (size < this.shrinkThreshold && capacity > initialCapacity) { + resize(capacity / expandMultiplier); + } + } + + /** + * Checks if we need to expand, and expands if necessary. + */ + protected void expandIfNecessary() { + if (size > this.expandThreshold && capacity < MAXIMUM_CAPACITY) { + resize(capacity * expandMultiplier); + } + } + + public Iterator iterator() { + return new LinkedSetIterator(); + } + + @Override + public String toString() { + final StringBuilder b = new StringBuilder(getClass().getSimpleName()); + b.append("(size=").append(size).append(", modification=") + .append(modification).append(", entries.length=") + .append(entries.length).append(")"); + return b.toString(); + } + + /** Print detailed information of this object. */ + public void printDetails(final PrintStream out) { + out.print(this + ", entries = ["); + for (int i = 0; i < entries.length; i++) { + if (entries[i] != null) { + LinkedElement e = entries[i]; + out.print("\n " + i + ": " + e); + for (e = e.next; e != null; e = e.next) { + out.print(" -> " + e); + } + } + } + out.println("\n]"); + } + + private class LinkedSetIterator implements Iterator { + /** The starting modification for fail-fast. */ + private final int startModification = modification; + /** The current index of the entry array. */ + private int index = -1; + /** The next element to return. */ + private LinkedElement next = nextNonemptyEntry(); + + private LinkedElement nextNonemptyEntry() { + for (index++; index < entries.length && entries[index] == null; index++); + return index < entries.length ? entries[index] : null; + } + + @Override + public boolean hasNext() { + return next != null; + } + + @Override + public T next() { + if (modification != startModification) { + throw new ConcurrentModificationException("modification=" + + modification + " != startModification = " + startModification); + } + if (next == null) { + throw new NoSuchElementException(); + } + final T e = next.element; + // find the next element + final LinkedElement n = next.next; + next = n != null ? n : nextNonemptyEntry(); + return e; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Remove is not supported."); + } + } + + /** + * Clear the set. Resize it to the original capacity. + */ + @SuppressWarnings("unchecked") + public void clear() { + this.capacity = this.initialCapacity; + this.hash_mask = capacity - 1; + + this.expandThreshold = (int) (capacity * maxLoadFactor); + this.shrinkThreshold = (int) (capacity * minLoadFactor); + + entries = new LinkedElement[capacity]; + size = 0; + modification++; + } + + @Override + public Object[] toArray() { + Object[] result = new Object[size]; + return toArray(result); + } + + @Override + @SuppressWarnings("unchecked") + public U[] toArray(U[] a) { + if (a == null) { + throw new NullPointerException("Input array can not be null"); + } + if (a.length < size) { + a = (U[]) java.lang.reflect.Array.newInstance(a.getClass() + .getComponentType(), size); + } + int currentIndex = 0; + for (int i = 0; i < entries.length; i++) { + LinkedElement current = entries[i]; + while (current != null) { + a[currentIndex++] = (U) current.element; + current = current.next; + } + } + return a; + } + + @Override + public boolean containsAll(Collection c) { + Iterator iter = c.iterator(); + while (iter.hasNext()) { + if (!contains(iter.next())) { + return false; + } + } + return true; + } + + @Override + public boolean removeAll(Collection c) { + boolean changed = false; + Iterator iter = c.iterator(); + while (iter.hasNext()) { + changed |= remove(iter.next()); + } + return changed; + } + + @Override + public boolean retainAll(Collection c) { + throw new UnsupportedOperationException("retainAll is not supported."); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightLinkedSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightLinkedSet.java new file mode 100644 index 0000000000..c90d2c7aa8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightLinkedSet.java @@ -0,0 +1,259 @@ +/** + * 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.hdfs.util; + +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * A low memory linked hash set implementation, which uses an array for storing + * the elements and linked lists for collision resolution. In addition it stores + * elements in a linked list to ensure ordered traversal. This class does not + * support null element. + * + * This class is not thread safe. + * + */ +public class LightWeightLinkedSet extends LightWeightHashSet { + /** + * Elements of {@link LightWeightLinkedSet}. + */ + static class DoubleLinkedElement extends LinkedElement { + // references to elements within all-element linked list + private DoubleLinkedElement before; + private DoubleLinkedElement after; + + public DoubleLinkedElement(T elem, int hashCode) { + super(elem, hashCode); + this.before = null; + this.after = null; + } + + public String toString() { + return super.toString(); + } + } + + private DoubleLinkedElement head; + private DoubleLinkedElement tail; + + /** + * @param initCapacity + * Recommended size of the internal array. + * @param maxLoadFactor + * used to determine when to expand the internal array + * @param minLoadFactor + * used to determine when to shrink the internal array + */ + public LightWeightLinkedSet(int initCapacity, float maxLoadFactor, + float minLoadFactor) { + super(initCapacity, maxLoadFactor, minLoadFactor); + head = null; + tail = null; + } + + public LightWeightLinkedSet() { + this(MINIMUM_CAPACITY, DEFAULT_MAX_LOAD_FACTOR, DEFAUT_MIN_LOAD_FACTOR); + } + + /** + * Add given element to the hash table + * + * @return true if the element was not present in the table, false otherwise + */ + protected boolean addElem(final T element) { + // validate element + if (element == null) { + throw new IllegalArgumentException("Null element is not supported."); + } + // find hashCode & index + final int hashCode = element.hashCode(); + final int index = getIndex(hashCode); + // return false if already present + if (containsElem(index, element, hashCode)) { + return false; + } + + modification++; + size++; + + // update bucket linked list + DoubleLinkedElement le = new DoubleLinkedElement(element, hashCode); + le.next = entries[index]; + entries[index] = le; + + // insert to the end of the all-element linked list + le.after = null; + le.before = tail; + if (tail != null) { + tail.after = le; + } + tail = le; + if (head == null) { + head = le; + } + return true; + } + + /** + * Remove the element corresponding to the key, given key.hashCode() == index. + * + * @return Return the entry with the element if exists. Otherwise return null. + */ + protected DoubleLinkedElement removeElem(final T key) { + DoubleLinkedElement found = (DoubleLinkedElement) (super + .removeElem(key)); + if (found == null) { + return null; + } + + // update linked list + if (found.after != null) { + found.after.before = found.before; + } + if (found.before != null) { + found.before.after = found.after; + } + if (head == found) { + head = head.after; + } + if (tail == found) { + tail = tail.before; + } + return found; + } + + /** + * Remove and return first element on the linked list of all elements. + * + * @return first element + */ + public T pollFirst() { + if (head == null) { + return null; + } + T first = head.element; + this.remove(first); + return first; + } + + /** + * Remove and return n elements from the hashtable. + * The order in which entries are removed is corresponds + * to the order in which they were inserted. + * + * @return first element + */ + public List pollN(int n) { + if (n >= size) { + // if we need to remove all elements then do fast polling + return pollAll(); + } + List retList = new ArrayList(n); + while (n-- > 0 && head != null) { + T curr = head.element; + this.removeElem(curr); + retList.add(curr); + } + shrinkIfNecessary(); + return retList; + } + + /** + * Remove all elements from the set and return them in order. Traverse the + * link list, don't worry about hashtable - faster version of the parent + * method. + */ + public List pollAll() { + List retList = new ArrayList(size); + while (head != null) { + retList.add(head.element); + head = head.after; + } + this.clear(); + return retList; + } + + @Override + @SuppressWarnings("unchecked") + public U[] toArray(U[] a) { + if (a == null) { + throw new NullPointerException("Input array can not be null"); + } + if (a.length < size) { + a = (U[]) java.lang.reflect.Array.newInstance(a.getClass() + .getComponentType(), size); + } + int currentIndex = 0; + DoubleLinkedElement current = head; + while (current != null) { + T curr = current.element; + a[currentIndex++] = (U) curr; + current = current.after; + } + return a; + } + + public Iterator iterator() { + return new LinkedSetIterator(); + } + + private class LinkedSetIterator implements Iterator { + /** The starting modification for fail-fast. */ + private final int startModification = modification; + /** The next element to return. */ + private DoubleLinkedElement next = head; + + @Override + public boolean hasNext() { + return next != null; + } + + @Override + public T next() { + if (modification != startModification) { + throw new ConcurrentModificationException("modification=" + + modification + " != startModification = " + startModification); + } + if (next == null) { + throw new NoSuchElementException(); + } + final T e = next.element; + // find the next element + next = next.after; + return e; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Remove is not supported."); + } + } + + /** + * Clear the set. Resize it to the original capacity. + */ + public void clear() { + super.clear(); + this.head = null; + this.tail = null; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/AuthFilter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/AuthFilter.java index c4b3d26824..d8eb724791 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/AuthFilter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/AuthFilter.java @@ -18,6 +18,12 @@ package org.apache.hadoop.hdfs.web; import java.io.IOException; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.Properties; import javax.servlet.FilterChain; @@ -26,6 +32,7 @@ import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletRequestWrapper; import org.apache.hadoop.hdfs.web.resources.DelegationParam; import org.apache.hadoop.security.UserGroupInformation; @@ -67,15 +74,77 @@ protected Properties getConfiguration(String prefix, FilterConfig config) @Override public void doFilter(ServletRequest request, ServletResponse response, FilterChain filterChain) throws IOException, ServletException { - HttpServletRequest httpRequest = (HttpServletRequest) request; - String tokenString = httpRequest - .getParameter(DelegationParam.NAME); + final HttpServletRequest httpRequest = toLowerCase((HttpServletRequest)request); + final String tokenString = httpRequest.getParameter(DelegationParam.NAME); if (tokenString != null) { //Token is present in the url, therefore token will be used for //authentication, bypass kerberos authentication. filterChain.doFilter(httpRequest, response); return; } - super.doFilter(request, response, filterChain); + super.doFilter(httpRequest, response, filterChain); + } + + private static HttpServletRequest toLowerCase(final HttpServletRequest request) { + @SuppressWarnings("unchecked") + final Map original = (Map)request.getParameterMap(); + if (!ParamFilter.containsUpperCase(original.keySet())) { + return request; + } + + final Map> m = new HashMap>(); + for(Map.Entry entry : original.entrySet()) { + final String key = entry.getKey().toLowerCase(); + List strings = m.get(key); + if (strings == null) { + strings = new ArrayList(); + m.put(key, strings); + } + for(String v : entry.getValue()) { + strings.add(v); + } + } + + return new HttpServletRequestWrapper(request) { + private Map parameters = null; + + @Override + public Map getParameterMap() { + if (parameters == null) { + parameters = new HashMap(); + for(Map.Entry> entry : m.entrySet()) { + final List a = entry.getValue(); + parameters.put(entry.getKey(), a.toArray(new String[a.size()])); + } + } + return parameters; + } + + @Override + public String getParameter(String name) { + final List a = m.get(name); + return a == null? null: a.get(0); + } + + @Override + public String[] getParameterValues(String name) { + return getParameterMap().get(name); + } + + @Override + public Enumeration getParameterNames() { + final Iterator i = m.keySet().iterator(); + return new Enumeration() { + @Override + public boolean hasMoreElements() { + return i.hasNext(); + } + @Override + public String nextElement() { + return i.next(); + } + }; + } + }; } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java index d166d63a98..a1c5d1ebd1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileChecksum; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtil; @@ -134,6 +135,14 @@ private static FsPermission toFsPermission(final String s) { return new FsPermission(Short.parseShort(s, 8)); } + static enum PathType { + FILE, DIRECTORY, SYMLINK; + + static PathType valueOf(HdfsFileStatus status) { + return status.isDir()? DIRECTORY: status.isSymlink()? SYMLINK: FILE; + } + } + /** Convert a HdfsFileStatus object to a Json string. */ public static String toJsonString(final HdfsFileStatus status, boolean includeType) { @@ -141,14 +150,13 @@ public static String toJsonString(final HdfsFileStatus status, return null; } final Map m = new TreeMap(); - m.put("localName", status.getLocalName()); - m.put("isDir", status.isDir()); - m.put("isSymlink", status.isSymlink()); + m.put("pathSuffix", status.getLocalName()); + m.put("type", PathType.valueOf(status)); if (status.isSymlink()) { m.put("symlink", status.getSymlink()); } - m.put("len", status.getLen()); + m.put("length", status.getLen()); m.put("owner", status.getOwner()); m.put("group", status.getGroup()); m.put("permission", toString(status.getPermission())); @@ -156,8 +164,7 @@ public static String toJsonString(final HdfsFileStatus status, m.put("modificationTime", status.getModificationTime()); m.put("blockSize", status.getBlockSize()); m.put("replication", status.getReplication()); - return includeType ? toJsonString(HdfsFileStatus.class, m) : - JSON.toString(m); + return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m); } /** Convert a Json map to a HdfsFileStatus object. */ @@ -167,14 +174,13 @@ public static HdfsFileStatus toFileStatus(final Map json, boolean includes } final Map m = includesType ? - (Map)json.get(HdfsFileStatus.class.getSimpleName()) : json; - final String localName = (String) m.get("localName"); - final boolean isDir = (Boolean) m.get("isDir"); - final boolean isSymlink = (Boolean) m.get("isSymlink"); - final byte[] symlink = isSymlink? - DFSUtil.string2Bytes((String)m.get("symlink")): null; + (Map)json.get(FileStatus.class.getSimpleName()) : json; + final String localName = (String) m.get("pathSuffix"); + final PathType type = PathType.valueOf((String) m.get("type")); + final byte[] symlink = type != PathType.SYMLINK? null + : DFSUtil.string2Bytes((String)m.get("symlink")); - final long len = (Long) m.get("len"); + final long len = (Long) m.get("length"); final String owner = (String) m.get("owner"); final String group = (String) m.get("group"); final FsPermission permission = toFsPermission((String) m.get("permission")); @@ -182,8 +188,8 @@ public static HdfsFileStatus toFileStatus(final Map json, boolean includes final long mTime = (Long) m.get("modificationTime"); final long blockSize = (Long) m.get("blockSize"); final short replication = (short) (long) (Long) m.get("replication"); - return new HdfsFileStatus(len, isDir, replication, blockSize, mTime, aTime, - permission, owner, group, + return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication, + blockSize, mTime, aTime, permission, owner, group, symlink, DFSUtil.string2Bytes(localName)); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java index 687b874767..2ae3445691 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java @@ -59,7 +59,7 @@ public ContainerResponseFilter getResponseFilter() { } /** Do the strings contain upper case letters? */ - private static boolean containsUpperCase(final Iterable strings) { + static boolean containsUpperCase(final Iterable strings) { for(String s : strings) { for(int i = 0; i < s.length(); i++) { if (Character.isUpperCase(s.charAt(i))) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index b37c0ed6e0..187bb8c9d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -25,12 +25,14 @@ import java.io.InputStreamReader; import java.net.HttpURLConnection; import java.net.InetSocketAddress; +import java.net.MalformedURLException; import java.net.URI; import java.net.URISyntaxException; import java.net.URL; import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.StringTokenizer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -62,7 +64,6 @@ import org.apache.hadoop.hdfs.web.resources.AccessTimeParam; import org.apache.hadoop.hdfs.web.resources.BlockSizeParam; import org.apache.hadoop.hdfs.web.resources.BufferSizeParam; -import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam; import org.apache.hadoop.hdfs.web.resources.DeleteOpParam; import org.apache.hadoop.hdfs.web.resources.DestinationParam; import org.apache.hadoop.hdfs.web.resources.GetOpParam; @@ -81,6 +82,7 @@ import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam; import org.apache.hadoop.hdfs.web.resources.RenewerParam; import org.apache.hadoop.hdfs.web.resources.ReplicationParam; +import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam; import org.apache.hadoop.hdfs.web.resources.UserParam; import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RemoteException; @@ -90,6 +92,8 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.client.AuthenticatedURL; import org.apache.hadoop.security.authentication.client.AuthenticationException; +import org.apache.hadoop.security.authorize.AuthorizationException; +import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenRenewer; @@ -112,17 +116,24 @@ public class WebHdfsFileSystem extends FileSystem private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator(); /** Delegation token kind */ public static final Text TOKEN_KIND = new Text("WEBHDFS delegation"); + /** Token selector */ + public static final AbstractDelegationTokenSelector DT_SELECTOR + = new AbstractDelegationTokenSelector(TOKEN_KIND) {}; - private static final DelegationTokenRenewer dtRenewer - = new DelegationTokenRenewer(WebHdfsFileSystem.class); - static { - dtRenewer.start(); + private static DelegationTokenRenewer DT_RENEWER = null; + + private static synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) { + if (DT_RENEWER == null) { + DT_RENEWER = new DelegationTokenRenewer(WebHdfsFileSystem.class); + DT_RENEWER.start(); + } + + DT_RENEWER.addRenewAction(webhdfs); } private final UserGroupInformation ugi; private InetSocketAddress nnAddr; private Token delegationToken; - private Token renewToken; private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token(); private Path workingDir; @@ -151,8 +162,7 @@ public synchronized void initialize(URI uri, Configuration conf protected void initDelegationToken() throws IOException { // look for webhdfs token, then try hdfs final Text serviceName = SecurityUtil.buildTokenService(nnAddr); - Token token = webhdfspTokenSelector.selectToken( - serviceName, ugi.getTokens()); + Token token = DT_SELECTOR.selectToken(serviceName, ugi.getTokens()); if (token == null) { token = DelegationTokenSelector.selectHdfsDelegationToken( nnAddr, ugi, getConf()); @@ -169,7 +179,7 @@ protected void initDelegationToken() throws IOException { if (token != null) { setDelegationToken(token); if (createdToken) { - dtRenewer.addRenewAction(this); + addRenewAction(this); LOG.debug("Created new DT for " + token.getService()); } else { LOG.debug("Found existing DT for " + token.getService()); @@ -193,9 +203,14 @@ public URI getUri() { } } + /** @return the home directory. */ + public static String getHomeDirectoryString(final UserGroupInformation ugi) { + return "/user/" + ugi.getShortUserName(); + } + @Override public Path getHomeDirectory() { - return makeQualified(new Path("/user/" + ugi.getShortUserName())); + return makeQualified(new Path(getHomeDirectoryString(ugi))); } @Override @@ -217,7 +232,7 @@ private Path makeAbsolute(Path f) { return f.isAbsolute()? f: new Path(workingDir, f); } - private static Map jsonParse(final InputStream in) throws IOException { + static Map jsonParse(final InputStream in) throws IOException { if (in == null) { throw new IOException("The input stream is null."); } @@ -243,13 +258,16 @@ private Path makeAbsolute(Path f) { final RemoteException re = JsonUtil.toRemoteException(m); throw re.unwrapRemoteException(AccessControlException.class, - DSQuotaExceededException.class, + InvalidToken.class, + AuthenticationException.class, + AuthorizationException.class, FileAlreadyExistsException.class, FileNotFoundException.class, ParentNotDirectoryException.class, + UnresolvedPathException.class, SafeModeException.class, - NSQuotaExceededException.class, - UnresolvedPathException.class); + DSQuotaExceededException.class, + NSQuotaExceededException.class); } return null; } @@ -326,14 +344,13 @@ private HttpURLConnection httpConnect(final HttpOpParam.Op op, final Path fspath final URL url = toUrl(op, fspath, parameters); //connect and get response - final HttpURLConnection conn = getHttpUrlConnection(url); + HttpURLConnection conn = getHttpUrlConnection(url); try { conn.setRequestMethod(op.getType().toString()); - conn.setDoOutput(op.getDoOutput()); if (op.getDoOutput()) { - conn.setRequestProperty("Expect", "100-Continue"); - conn.setInstanceFollowRedirects(true); + conn = twoStepWrite(conn, op); } + conn.setDoOutput(op.getDoOutput()); conn.connect(); return conn; } catch (IOException e) { @@ -341,6 +358,35 @@ private HttpURLConnection httpConnect(final HttpOpParam.Op op, final Path fspath throw e; } } + + /** + * Two-step Create/Append: + * Step 1) Submit a Http request with neither auto-redirect nor data. + * Step 2) Submit another Http request with the URL from the Location header with data. + * + * The reason of having two-step create/append is for preventing clients to + * send out the data before the redirect. This issue is addressed by the + * "Expect: 100-continue" header in HTTP/1.1; see RFC 2616, Section 8.2.3. + * Unfortunately, there are software library bugs (e.g. Jetty 6 http server + * and Java 6 http client), which do not correctly implement "Expect: + * 100-continue". The two-step create/append is a temporary workaround for + * the software library bugs. + */ + static HttpURLConnection twoStepWrite(HttpURLConnection conn, + final HttpOpParam.Op op) throws IOException { + //Step 1) Submit a Http request with neither auto-redirect nor data. + conn.setInstanceFollowRedirects(false); + conn.setDoOutput(false); + conn.connect(); + validateResponse(HttpOpParam.TemporaryRedirectOp.valueOf(op), conn); + final String redirect = conn.getHeaderField("Location"); + conn.disconnect(); + + //Step 2) Submit another Http request with the URL from the Location header with data. + conn = (HttpURLConnection)new URL(redirect).openConnection(); + conn.setRequestMethod(op.getType().toString()); + return conn; + } /** * Run a http operation. @@ -388,9 +434,9 @@ public FileStatus getFileStatus(Path f) throws IOException { private FileStatus makeQualified(HdfsFileStatus f, Path parent) { return new FileStatus(f.getLen(), f.isDir(), f.getReplication(), - f.getBlockSize(), f.getModificationTime(), - f.getAccessTime(), + f.getBlockSize(), f.getModificationTime(), f.getAccessTime(), f.getPermission(), f.getOwner(), f.getGroup(), + f.isSymlink() ? new Path(f.getSymlink()) : null, f.getFullPath(parent).makeQualified(getUri(), getWorkingDirectory())); } @@ -471,7 +517,7 @@ public short getDefaultReplication() { DFSConfigKeys.DFS_REPLICATION_DEFAULT); } - private FSDataOutputStream write(final HttpOpParam.Op op, + FSDataOutputStream write(final HttpOpParam.Op op, final HttpURLConnection conn, final int bufferSize) throws IOException { return new FSDataOutputStream(new BufferedOutputStream( conn.getOutputStream(), bufferSize), statistics) { @@ -480,7 +526,11 @@ public void close() throws IOException { try { super.close(); } finally { - validateResponse(op, conn); + try { + validateResponse(op, conn); + } finally { + conn.disconnect(); + } } } }; @@ -532,24 +582,84 @@ public FSDataInputStream open(final Path f, final int buffersize statistics.incrementReadOps(1); final HttpOpParam.Op op = GetOpParam.Op.OPEN; final URL url = toUrl(op, f, new BufferSizeParam(buffersize)); - ByteRangeInputStream str = getByteRangeInputStream(url); - return new FSDataInputStream(str); + return new FSDataInputStream(new OffsetUrlInputStream( + new OffsetUrlOpener(url), new OffsetUrlOpener(null))); } - private class URLOpener extends ByteRangeInputStream.URLOpener { - - public URLOpener(URL u) { - super(u); + class OffsetUrlOpener extends ByteRangeInputStream.URLOpener { + /** The url with offset parameter */ + private URL offsetUrl; + + OffsetUrlOpener(final URL url) { + super(url); } + /** Open connection with offset url. */ @Override - public HttpURLConnection openConnection() throws IOException { + protected HttpURLConnection openConnection() throws IOException { return getHttpUrlConnection(offsetUrl); } + + /** Setup offset url before open connection. */ + @Override + protected HttpURLConnection openConnection(final long offset) throws IOException { + offsetUrl = offset == 0L? url: new URL(url + "&" + new OffsetParam(offset)); + final HttpURLConnection conn = openConnection(); + conn.setRequestMethod("GET"); + return conn; + } } - - private ByteRangeInputStream getByteRangeInputStream(URL url) { - return new ByteRangeInputStream(new URLOpener(url), new URLOpener(null)); + + private static final String OFFSET_PARAM_PREFIX = OffsetParam.NAME + "="; + + /** Remove offset parameter, if there is any, from the url */ + static URL removeOffsetParam(final URL url) throws MalformedURLException { + String query = url.getQuery(); + if (query == null) { + return url; + } + final String lower = query.toLowerCase(); + if (!lower.startsWith(OFFSET_PARAM_PREFIX) + && !lower.contains("&" + OFFSET_PARAM_PREFIX)) { + return url; + } + + //rebuild query + StringBuilder b = null; + for(final StringTokenizer st = new StringTokenizer(query, "&"); + st.hasMoreTokens();) { + final String token = st.nextToken(); + if (!token.toLowerCase().startsWith(OFFSET_PARAM_PREFIX)) { + if (b == null) { + b = new StringBuilder("?").append(token); + } else { + b.append('&').append(token); + } + } + } + query = b == null? "": b.toString(); + + final String urlStr = url.toString(); + return new URL(urlStr.substring(0, urlStr.indexOf('?')) + query); + } + + static class OffsetUrlInputStream extends ByteRangeInputStream { + OffsetUrlInputStream(OffsetUrlOpener o, OffsetUrlOpener r) { + super(o, r); + } + + @Override + protected void checkResponseCode(final HttpURLConnection connection + ) throws IOException { + validateResponse(GetOpParam.Op.OPEN, connection); + } + + /** Remove offset parameter before returning the resolved url. */ + @Override + protected URL getResolvedUrl(final HttpURLConnection connection + ) throws MalformedURLException { + return removeOffsetParam(connection.getURL()); + } } @Override @@ -558,8 +668,8 @@ public FileStatus[] listStatus(final Path f) throws IOException { final HttpOpParam.Op op = GetOpParam.Op.LISTSTATUS; final Map json = run(op, f); - final Map rootmap = (Map)json.get(HdfsFileStatus.class.getSimpleName() + "es"); - final Object[] array = (Object[])rootmap.get(HdfsFileStatus.class.getSimpleName()); + final Map rootmap = (Map)json.get(FileStatus.class.getSimpleName() + "es"); + final Object[] array = (Object[])rootmap.get(FileStatus.class.getSimpleName()); //convert FileStatus final FileStatus[] statuses = new FileStatus[array.length]; @@ -577,7 +687,7 @@ public Token getDelegationToken(final String renewer final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN; final Map m = run(op, null, new RenewerParam(renewer)); final Token token = JsonUtil.toDelegationToken(m); - token.setService(new Text(getCanonicalServiceName())); + SecurityUtil.setTokenService(token, nnAddr); return token; } @@ -590,23 +700,14 @@ public List> getDelegationTokens(final String renewer @Override public Token getRenewToken() { - return renewToken; + return delegationToken; } @Override public void setDelegationToken( final Token token) { synchronized(this) { - renewToken = token; - // emulate the 203 usage of the tokens - // by setting the kind and service as if they were hdfs tokens - delegationToken = new Token(token); - // NOTE: the remote nn must be configured to use hdfs - delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND); - // no need to change service because we aren't exactly sure what it - // should be. we can guess, but it might be wrong if the local conf - // value is incorrect. the service is a client side field, so the remote - // end does not care about the value + delegationToken = token; } } @@ -641,7 +742,7 @@ public BlockLocation[] getFileBlockLocations(final Path p, final long offset, final long length) throws IOException { statistics.incrementReadOps(1); - final HttpOpParam.Op op = GetOpParam.Op.GETFILEBLOCKLOCATIONS; + final HttpOpParam.Op op = GetOpParam.Op.GET_BLOCK_LOCATIONS; final Map m = run(op, p, new OffsetParam(offset), new LengthParam(length)); return DFSUtil.locatedBlocks2Locations(JsonUtil.toLocatedBlocks(m)); @@ -666,15 +767,6 @@ public MD5MD5CRC32FileChecksum getFileChecksum(final Path p return JsonUtil.toMD5MD5CRC32FileChecksum(m); } - private static final DtSelector webhdfspTokenSelector = new DtSelector(); - - private static class DtSelector - extends AbstractDelegationTokenSelector { - private DtSelector() { - super(TOKEN_KIND); - } - } - /** Delegation token renewer. */ public static class DtRenewer extends TokenRenewer { @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java new file mode 100644 index 0000000000..13d188cd13 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java @@ -0,0 +1,41 @@ +/** + * 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.hdfs.web.resources; + +/** DoAs parameter for proxy user. */ +public class DoAsParam extends StringParam { + /** Parameter name. */ + public static final String NAME = "doas"; + /** Default parameter value. */ + public static final String DEFAULT = ""; + + private static final Domain DOMAIN = new Domain(NAME, null); + + /** + * Constructor. + * @param str a string representation of the parameter value. + */ + public DoAsParam(final String str) { + super(DOMAIN, str == null || str.equals(DEFAULT)? null: str); + } + + @Override + public String getName() { + return NAME; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java index bd0003ef0e..adb944a0dc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java @@ -30,14 +30,25 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hdfs.web.JsonUtil; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.security.authorize.AuthorizationException; import com.sun.jersey.api.ParamException; +import com.sun.jersey.api.container.ContainerException; /** Handle exceptions. */ @Provider public class ExceptionHandler implements ExceptionMapper { public static final Log LOG = LogFactory.getLog(ExceptionHandler.class); + private static Exception toCause(Exception e) { + final Throwable t = e.getCause(); + if (t != null && t instanceof Exception) { + e = (Exception)e.getCause(); + } + return e; + } + private @Context HttpServletResponse response; @Override @@ -55,12 +66,20 @@ public Response toResponse(Exception e) { e = new IllegalArgumentException("Invalid value for webhdfs parameter \"" + paramexception.getParameterName() + "\": " + e.getCause().getMessage(), e); - } + } + if (e instanceof ContainerException) { + e = toCause(e); + } + if (e instanceof RemoteException) { + e = ((RemoteException)e).unwrapRemoteException(); + } //Map response status final Response.Status s; if (e instanceof SecurityException) { s = Response.Status.UNAUTHORIZED; + } else if (e instanceof AuthorizationException) { + s = Response.Status.UNAUTHORIZED; } else if (e instanceof FileNotFoundException) { s = Response.Status.NOT_FOUND; } else if (e instanceof IOException) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java index cab71c99d2..2a418ce852 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java @@ -24,15 +24,18 @@ public class GetOpParam extends HttpOpParam { /** Get operations. */ public static enum Op implements HttpOpParam.Op { OPEN(HttpURLConnection.HTTP_OK), - GETFILEBLOCKLOCATIONS(HttpURLConnection.HTTP_OK), GETFILESTATUS(HttpURLConnection.HTTP_OK), LISTSTATUS(HttpURLConnection.HTTP_OK), GETCONTENTSUMMARY(HttpURLConnection.HTTP_OK), GETFILECHECKSUM(HttpURLConnection.HTTP_OK), + GETHOMEDIRECTORY(HttpURLConnection.HTTP_OK), GETDELEGATIONTOKEN(HttpURLConnection.HTTP_OK), + /** GET_BLOCK_LOCATIONS is a private unstable op. */ + GET_BLOCK_LOCATIONS(HttpURLConnection.HTTP_OK), + NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED); final int expectedHttpResponseCode; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java index 422ec0f2f2..ab32ab59aa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java @@ -17,6 +17,9 @@ */ package org.apache.hadoop.hdfs.web.resources; +import javax.ws.rs.core.Response; + + /** Http operation parameter. */ public abstract class HttpOpParam & HttpOpParam.Op> extends EnumParam { @@ -46,6 +49,49 @@ public static interface Op { public String toQueryString(); } + /** Expects HTTP response 307 "Temporary Redirect". */ + public static class TemporaryRedirectOp implements Op { + static final TemporaryRedirectOp CREATE = new TemporaryRedirectOp(PutOpParam.Op.CREATE); + static final TemporaryRedirectOp APPEND = new TemporaryRedirectOp(PostOpParam.Op.APPEND); + + /** Get an object for the given op. */ + public static TemporaryRedirectOp valueOf(final Op op) { + if (op == CREATE.op) { + return CREATE; + } else if (op == APPEND.op) { + return APPEND; + } + throw new IllegalArgumentException(op + " not found."); + } + + private final Op op; + + private TemporaryRedirectOp(final Op op) { + this.op = op; + } + + @Override + public Type getType() { + return op.getType(); + } + + @Override + public boolean getDoOutput() { + return op.getDoOutput(); + } + + /** Override the original expected response with "Temporary Redirect". */ + @Override + public int getExpectedHttpResponseCode() { + return Response.Status.TEMPORARY_REDIRECT.getStatusCode(); + } + + @Override + public String toQueryString() { + return op.toQueryString(); + } + } + HttpOpParam(final Domain domain, final E value) { super(domain, value); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java index 74070243c0..44e8938538 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java @@ -53,7 +53,8 @@ public UserGroupInformation getValue(final HttpContext context) { return JspHelper.getUGI(servletcontext, request, conf, AuthenticationMethod.KERBEROS, false); } catch (IOException e) { - throw new RuntimeException(e); + throw new SecurityException( + "Failed to obtain user group information: " + e, e); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java index c61e65b6c0..3c338e56f5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java @@ -139,15 +139,17 @@ public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToR ExtendedBlock block = testBlock.getBlock(); DatanodeInfo[] nodes = testBlock.getLocations(); targetAddr = NetUtils.createSocketAddr(nodes[0].getName()); - sock = new Socket(); + sock = NetUtils.getDefaultSocketFactory(conf).createSocket(); sock.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT); sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); return BlockReaderFactory.newBlockReader( + new DFSClient.Conf(conf), sock, targetAddr.toString()+ ":" + block.getBlockId(), block, testBlock.getBlockToken(), offset, lenToRead, - conf.getInt("io.file.buffer.size", 4096)); + conf.getInt("io.file.buffer.size", 4096), + true, ""); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java new file mode 100644 index 0000000000..f296419bde --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java @@ -0,0 +1,165 @@ +/** + * 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.hdfs; + +import java.io.IOException; +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.IOUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +/** + * Test cases for trying to append to a file with a different + * checksum than the file was originally written with. + */ +public class TestAppendDifferentChecksum { + private static final int SEGMENT_LENGTH = 1500; + + // run the randomized test for 5 seconds + private static final long RANDOM_TEST_RUNTIME = 5000; + private static MiniDFSCluster cluster; + private static FileSystem fs; + + + @BeforeClass + public static void setupCluster() throws IOException { + Configuration conf = new HdfsConfiguration(); + conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096); + + // disable block scanner, since otherwise this test can trigger + // HDFS-2525, which is a different bug than we're trying to unit test + // here! When HDFS-2525 is fixed, this can be removed. + conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); + + conf.set("fs.hdfs.impl.disable.cache", "true"); + cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(1) + .build(); + fs = cluster.getFileSystem(); + } + + @AfterClass + public static void teardown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * This test does not run, since switching chunksize with append + * is not implemented. Please see HDFS-2130 for a discussion of the + * difficulties in doing so. + */ + @Test + @Ignore("this is not implemented! See HDFS-2130") + public void testSwitchChunkSize() throws IOException { + FileSystem fsWithSmallChunk = createFsWithChecksum("CRC32", 512); + FileSystem fsWithBigChunk = createFsWithChecksum("CRC32", 1024); + Path p = new Path("/testSwitchChunkSize"); + appendWithTwoFs(p, fsWithSmallChunk, fsWithBigChunk); + AppendTestUtil.check(fsWithSmallChunk, p, SEGMENT_LENGTH * 2); + AppendTestUtil.check(fsWithBigChunk, p, SEGMENT_LENGTH * 2); + } + + /** + * Simple unit test which writes some data with one algorithm, + * then appends with another. + */ + @Test + public void testSwitchAlgorithms() throws IOException { + FileSystem fsWithCrc32 = createFsWithChecksum("CRC32", 512); + FileSystem fsWithCrc32C = createFsWithChecksum("CRC32C", 512); + + Path p = new Path("/testSwitchAlgorithms"); + appendWithTwoFs(p, fsWithCrc32, fsWithCrc32C); + // Regardless of which FS is used to read, it should pick up + // the on-disk checksum! + AppendTestUtil.check(fsWithCrc32C, p, SEGMENT_LENGTH * 2); + AppendTestUtil.check(fsWithCrc32, p, SEGMENT_LENGTH * 2); + } + + /** + * Test which randomly alternates between appending with + * CRC32 and with CRC32C, crossing several block boundaries. + * Then, checks that all of the data can be read back correct. + */ + @Test(timeout=RANDOM_TEST_RUNTIME*2) + public void testAlgoSwitchRandomized() throws IOException { + FileSystem fsWithCrc32 = createFsWithChecksum("CRC32", 512); + FileSystem fsWithCrc32C = createFsWithChecksum("CRC32C", 512); + + Path p = new Path("/testAlgoSwitchRandomized"); + long seed = System.currentTimeMillis(); + System.out.println("seed: " + seed); + Random r = new Random(seed); + + // Create empty to start + IOUtils.closeStream(fsWithCrc32.create(p)); + + long st = System.currentTimeMillis(); + int len = 0; + while (System.currentTimeMillis() - st < RANDOM_TEST_RUNTIME) { + int thisLen = r.nextInt(500); + FileSystem fs = (r.nextBoolean() ? fsWithCrc32 : fsWithCrc32C); + FSDataOutputStream stm = fs.append(p); + try { + AppendTestUtil.write(stm, len, thisLen); + } finally { + stm.close(); + } + len += thisLen; + } + + AppendTestUtil.check(fsWithCrc32, p, len); + AppendTestUtil.check(fsWithCrc32C, p, len); + } + + private FileSystem createFsWithChecksum(String type, int bytes) + throws IOException { + Configuration conf = new Configuration(fs.getConf()); + conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, type); + conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, bytes); + return FileSystem.get(conf); + } + + + private void appendWithTwoFs(Path p, FileSystem fs1, FileSystem fs2) + throws IOException { + FSDataOutputStream stm = fs1.create(p); + try { + AppendTestUtil.write(stm, 0, SEGMENT_LENGTH); + } finally { + stm.close(); + } + + stm = fs2.append(p); + try { + AppendTestUtil.write(stm, SEGMENT_LENGTH, SEGMENT_LENGTH); + } finally { + stm.close(); + } + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java index 3be88d3e5c..9e1b73bbd8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java @@ -31,10 +31,10 @@ import java.net.HttpURLConnection; import java.net.URL; -import org.apache.hadoop.hdfs.ByteRangeInputStream.URLOpener; import org.junit.Test; -class MockHttpURLConnection extends HttpURLConnection { +public class TestByteRangeInputStream { +public static class MockHttpURLConnection extends HttpURLConnection { public MockHttpURLConnection(URL u) { super(u); } @@ -85,54 +85,18 @@ public void setResponseCode(int resCode) { responseCode = resCode; } } - -public class TestByteRangeInputStream { - @Test - public void testRemoveOffset() throws IOException { - { //no offset - String s = "http://test/Abc?Length=99"; - assertEquals(s, ByteRangeInputStream.removeOffsetParam(new URL(s)).toString()); - } - - { //no parameters - String s = "http://test/Abc"; - assertEquals(s, ByteRangeInputStream.removeOffsetParam(new URL(s)).toString()); - } - - { //offset as first parameter - String s = "http://test/Abc?offset=10&Length=99"; - assertEquals("http://test/Abc?Length=99", - ByteRangeInputStream.removeOffsetParam(new URL(s)).toString()); - } - - { //offset as second parameter - String s = "http://test/Abc?op=read&OFFset=10&Length=99"; - assertEquals("http://test/Abc?op=read&Length=99", - ByteRangeInputStream.removeOffsetParam(new URL(s)).toString()); - } - - { //offset as last parameter - String s = "http://test/Abc?Length=99&offset=10"; - assertEquals("http://test/Abc?Length=99", - ByteRangeInputStream.removeOffsetParam(new URL(s)).toString()); - } - - { //offset as the only parameter - String s = "http://test/Abc?offset=10"; - assertEquals("http://test/Abc", - ByteRangeInputStream.removeOffsetParam(new URL(s)).toString()); - } - } @Test public void testByteRange() throws IOException { - URLOpener ospy = spy(new URLOpener(new URL("http://test/"))); + HftpFileSystem.RangeHeaderUrlOpener ospy = spy( + new HftpFileSystem.RangeHeaderUrlOpener(new URL("http://test/"))); doReturn(new MockHttpURLConnection(ospy.getURL())).when(ospy) .openConnection(); - URLOpener rspy = spy(new URLOpener((URL) null)); + HftpFileSystem.RangeHeaderUrlOpener rspy = spy( + new HftpFileSystem.RangeHeaderUrlOpener((URL) null)); doReturn(new MockHttpURLConnection(rspy.getURL())).when(rspy) .openConnection(); - ByteRangeInputStream is = new ByteRangeInputStream(ospy, rspy); + ByteRangeInputStream is = new HftpFileSystem.RangeHeaderInputStream(ospy, rspy); assertEquals("getPos wrong", 0, is.getPos()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java index 8315e1f5d0..ec2d41c06d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java @@ -20,11 +20,12 @@ import java.util.List; -import org.apache.hadoop.hdfs.RemoteBlockReader; import org.apache.hadoop.hdfs.DFSClient; +import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.fs.Path; +import org.apache.log4j.Level; import org.junit.Test; import org.junit.AfterClass; @@ -40,6 +41,9 @@ public class TestClientBlockVerification { static final int FILE_SIZE_K = 256; static LocatedBlock testBlock = null; + static { + ((Log4JLogger)RemoteBlockReader2.LOG).getLogger().setLevel(Level.ALL); + } @BeforeClass public static void setupCluster() throws Exception { final int REPLICATION_FACTOR = 1; @@ -54,7 +58,7 @@ public static void setupCluster() throws Exception { */ @Test public void testBlockVerification() throws Exception { - RemoteBlockReader reader = (RemoteBlockReader)spy( + RemoteBlockReader2 reader = (RemoteBlockReader2)spy( util.getBlockReader(testBlock, 0, FILE_SIZE_K * 1024)); util.readAndCheckEOS(reader, FILE_SIZE_K * 1024, true); verify(reader).sendReadResult(reader.dnSock, Status.CHECKSUM_OK); @@ -66,7 +70,7 @@ public void testBlockVerification() throws Exception { */ @Test public void testIncompleteRead() throws Exception { - RemoteBlockReader reader = (RemoteBlockReader)spy( + RemoteBlockReader2 reader = (RemoteBlockReader2)spy( util.getBlockReader(testBlock, 0, FILE_SIZE_K * 1024)); util.readAndCheckEOS(reader, FILE_SIZE_K / 2 * 1024, false); @@ -84,7 +88,7 @@ public void testIncompleteRead() throws Exception { @Test public void testCompletePartialRead() throws Exception { // Ask for half the file - RemoteBlockReader reader = (RemoteBlockReader)spy( + RemoteBlockReader2 reader = (RemoteBlockReader2)spy( util.getBlockReader(testBlock, 0, FILE_SIZE_K * 1024 / 2)); // And read half the file util.readAndCheckEOS(reader, FILE_SIZE_K * 1024 / 2, true); @@ -104,7 +108,7 @@ public void testUnalignedReads() throws Exception { for (int length : lengths) { DFSClient.LOG.info("Testing startOffset = " + startOffset + " and " + " len=" + length); - RemoteBlockReader reader = (RemoteBlockReader)spy( + RemoteBlockReader2 reader = (RemoteBlockReader2)spy( util.getBlockReader(testBlock, startOffset, length)); util.readAndCheckEOS(reader, length, true); verify(reader).sendReadResult(reader.dnSock, Status.CHECKSUM_OK); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java index a7d11c165a..136a72205c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java @@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.RemoteBlockReader; import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSInputStream; import org.apache.hadoop.hdfs.SocketCache; @@ -76,20 +75,20 @@ public class TestConnCache { * It verifies that all invocation to DFSInputStream.getBlockReader() * use the same socket. */ - private class MockGetBlockReader implements Answer { - public RemoteBlockReader reader = null; + private class MockGetBlockReader implements Answer { + public RemoteBlockReader2 reader = null; private Socket sock = null; - public RemoteBlockReader answer(InvocationOnMock invocation) throws Throwable { - RemoteBlockReader prevReader = reader; - reader = (RemoteBlockReader) invocation.callRealMethod(); + public RemoteBlockReader2 answer(InvocationOnMock invocation) throws Throwable { + RemoteBlockReader2 prevReader = reader; + reader = (RemoteBlockReader2) invocation.callRealMethod(); if (sock == null) { sock = reader.dnSock; - } else if (prevReader != null && prevReader.hasSentStatusCode()) { - // Can't reuse socket if the previous BlockReader didn't read till EOS. + } else if (prevReader != null) { assertSame("DFSInputStream should use the same socket", sock, reader.dnSock); - } return reader; + } + return reader; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java index d02ae1da35..4333f1c0f9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java @@ -74,7 +74,7 @@ public class TestDataTransferProtocol extends TestCase { "org.apache.hadoop.hdfs.TestDataTransferProtocol"); private static final DataChecksum DEFAULT_CHECKSUM = - DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_CRC32, 512); + DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_CRC32C, 512); DatanodeID datanode; InetSocketAddress dnAddr; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java index 92401743f1..350d6ac52a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java @@ -19,11 +19,14 @@ import java.io.IOException; +import org.junit.Test; + /** Test for simple signs of life using Avro RPC. Not an exhaustive test * yet, just enough to catch fundamental problems using Avro reflection to * infer namenode RPC protocols. */ public class TestDfsOverAvroRpc extends TestLocalDFS { + @Test(timeout=20000) public void testWorkingDirectory() throws IOException { System.setProperty("hdfs.rpc.engine", "org.apache.hadoop.ipc.AvroRpcEngine"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLocalDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLocalDFS.java index eef6bbed1f..32af360920 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLocalDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLocalDFS.java @@ -17,7 +17,8 @@ */ package org.apache.hadoop.hdfs; -import junit.framework.TestCase; +import org.junit.Test; +import static org.junit.Assert.*; import java.io.*; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -27,7 +28,7 @@ * This class tests the DFS class via the FileSystem interface in a single node * mini-cluster. */ -public class TestLocalDFS extends TestCase { +public class TestLocalDFS { private void writeFile(FileSystem fileSys, Path name) throws IOException { DataOutputStream stm = fileSys.create(name); @@ -59,6 +60,7 @@ static String getUserName(FileSystem fs) { /** * Tests get/set working directory in DFS. */ + @Test(timeout=20000) public void testWorkingDirectory() throws IOException { Configuration conf = new HdfsConfiguration(); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java index d2073a5681..a34e00a03c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; /** * This class tests the presence of seek bug as described @@ -67,12 +68,12 @@ private void seekReadFile(FileSystem fileSys, Path name) throws IOException { stm.read(actual, 0, actual.length); // Now read a byte array that is bigger than the internal buffer actual = new byte[100000]; - stm.read(actual, 0, actual.length); + IOUtils.readFully(stm, actual, 0, actual.length); checkAndEraseData(actual, 128, expected, "First Read Test"); // now do a small seek, within the range that is already read stm.seek(96036); // 4 byte seek actual = new byte[128]; - stm.read(actual, 0, actual.length); + IOUtils.readFully(stm, actual, 0, actual.length); checkAndEraseData(actual, 96036, expected, "Seek Bug"); // all done stm.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java index 4c5e1f9c0c..b1f9dcec92 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java @@ -23,29 +23,46 @@ import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; +import java.net.HttpURLConnection; import java.net.InetAddress; import java.net.NetworkInterface; +import java.net.URL; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Enumeration; +import java.util.Map; -import junit.framework.Assert; +import javax.servlet.http.HttpServletResponse; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods; +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; +import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; +import org.apache.hadoop.hdfs.web.resources.DoAsParam; +import org.apache.hadoop.hdfs.web.resources.ExceptionHandler; +import org.apache.hadoop.hdfs.web.resources.GetOpParam; +import org.apache.hadoop.hdfs.web.resources.PutOpParam; import org.apache.hadoop.security.TestDoAsEffectiveUser; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.security.token.Token; +import org.apache.log4j.Level; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -89,6 +106,7 @@ private void configureSuperUserIPAddresses(Configuration conf, @Before public void setUp() throws Exception { config = new HdfsConfiguration(); + config.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true); config.setLong( DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000); config.setLong( @@ -137,4 +155,63 @@ public Token run() throws IOException { } } + @Test + public void testWebHdfsDoAs() throws Exception { + WebHdfsTestUtil.LOG.info("START: testWebHdfsDoAs()"); + ((Log4JLogger)NamenodeWebHdfsMethods.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)ExceptionHandler.LOG).getLogger().setLevel(Level.ALL); + final UserGroupInformation ugi = UserGroupInformation.createRemoteUser(REAL_USER); + WebHdfsTestUtil.LOG.info("ugi.getShortUserName()=" + ugi.getShortUserName()); + final WebHdfsFileSystem webhdfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(ugi, config); + + final Path root = new Path("/"); + cluster.getFileSystem().setPermission(root, new FsPermission((short)0777)); + + { + //test GETHOMEDIRECTORY with doAs + final URL url = WebHdfsTestUtil.toUrl(webhdfs, + GetOpParam.Op.GETHOMEDIRECTORY, root, new DoAsParam(PROXY_USER)); + final HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + final Map m = WebHdfsTestUtil.connectAndGetJson(conn, HttpServletResponse.SC_OK); + conn.disconnect(); + + final Object responsePath = m.get(Path.class.getSimpleName()); + WebHdfsTestUtil.LOG.info("responsePath=" + responsePath); + Assert.assertEquals("/user/" + PROXY_USER, responsePath); + } + + { + //test GETHOMEDIRECTORY with DOas + final URL url = WebHdfsTestUtil.toUrl(webhdfs, + GetOpParam.Op.GETHOMEDIRECTORY, root, new DoAsParam(PROXY_USER) { + @Override + public String getName() { + return "DOas"; + } + }); + final HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + final Map m = WebHdfsTestUtil.connectAndGetJson(conn, HttpServletResponse.SC_OK); + conn.disconnect(); + + final Object responsePath = m.get(Path.class.getSimpleName()); + WebHdfsTestUtil.LOG.info("responsePath=" + responsePath); + Assert.assertEquals("/user/" + PROXY_USER, responsePath); + } + + { + //test create file with doAs + final Path f = new Path("/testWebHdfsDoAs/a.txt"); + final PutOpParam.Op op = PutOpParam.Op.CREATE; + final URL url = WebHdfsTestUtil.toUrl(webhdfs, op, f, new DoAsParam(PROXY_USER)); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn = WebHdfsTestUtil.twoStepWrite(conn, op); + final FSDataOutputStream out = WebHdfsTestUtil.write(webhdfs, op, conn, 4096); + out.write("Hello, webhdfs user!".getBytes()); + out.close(); + + final FileStatus status = webhdfs.getFileStatus(f); + WebHdfsTestUtil.LOG.info("status.getOwner()=" + status.getOwner()); + Assert.assertEquals(PROXY_USER, status.getOwner()); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java new file mode 100644 index 0000000000..2750b29356 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java @@ -0,0 +1,123 @@ +/** + * 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.hdfs.server.blockmanagement; + +import org.junit.Test; +import static org.junit.Assert.*; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Random; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.common.GenerationStamp; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; + +/** + * This class provides tests for BlockInfo class, which is used in BlocksMap. + * The test covers BlockList.listMoveToHead, used for faster block report + * processing in DatanodeDescriptor.reportDiff. + */ + +public class TestBlockInfo { + + private static final Log LOG = LogFactory + .getLog("org.apache.hadoop.hdfs.TestBlockInfo"); + + @Test + public void testBlockListMoveToHead() throws Exception { + LOG.info("BlockInfo moveToHead tests..."); + + final int MAX_BLOCKS = 10; + + DatanodeDescriptor dd = new DatanodeDescriptor(); + ArrayList blockList = new ArrayList(MAX_BLOCKS); + ArrayList blockInfoList = new ArrayList(); + int headIndex; + int curIndex; + + LOG.info("Building block list..."); + for (int i = 0; i < MAX_BLOCKS; i++) { + blockList.add(new Block(i, 0, GenerationStamp.FIRST_VALID_STAMP)); + blockInfoList.add(new BlockInfo(blockList.get(i), 3)); + dd.addBlock(blockInfoList.get(i)); + + // index of the datanode should be 0 + assertEquals("Find datanode should be 0", 0, blockInfoList.get(i) + .findDatanode(dd)); + } + + // list length should be equal to the number of blocks we inserted + LOG.info("Checking list length..."); + assertEquals("Length should be MAX_BLOCK", MAX_BLOCKS, dd.numBlocks()); + Iterator it = dd.getBlockIterator(); + int len = 0; + while (it.hasNext()) { + it.next(); + len++; + } + assertEquals("There should be MAX_BLOCK blockInfo's", MAX_BLOCKS, len); + + headIndex = dd.getHead().findDatanode(dd); + + LOG.info("Moving each block to the head of the list..."); + for (int i = 0; i < MAX_BLOCKS; i++) { + curIndex = blockInfoList.get(i).findDatanode(dd); + headIndex = dd.moveBlockToHead(blockInfoList.get(i), curIndex, headIndex); + // the moved element must be at the head of the list + assertEquals("Block should be at the head of the list now.", + blockInfoList.get(i), dd.getHead()); + } + + // move head of the list to the head - this should not change the list + LOG.info("Moving head to the head..."); + + BlockInfo temp = dd.getHead(); + curIndex = 0; + headIndex = 0; + dd.moveBlockToHead(temp, curIndex, headIndex); + assertEquals( + "Moving head to the head of the list shopuld not change the list", + temp, dd.getHead()); + + // check all elements of the list against the original blockInfoList + LOG.info("Checking elements of the list..."); + temp = dd.getHead(); + assertNotNull("Head should not be null", temp); + int c = MAX_BLOCKS - 1; + while (temp != null) { + assertEquals("Expected element is not on the list", + blockInfoList.get(c--), temp); + temp = temp.getNext(0); + } + + LOG.info("Moving random blocks to the head of the list..."); + headIndex = dd.getHead().findDatanode(dd); + Random rand = new Random(); + for (int i = 0; i < MAX_BLOCKS; i++) { + int j = rand.nextInt(MAX_BLOCKS); + curIndex = blockInfoList.get(j).findDatanode(dd); + headIndex = dd.moveBlockToHead(blockInfoList.get(j), curIndex, headIndex); + // the moved element must be at the head of the list + assertEquals("Block should be at the head of the list now.", + blockInfoList.get(j), dd.getHead()); + } + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java index 44d733df5a..dbecfe7f78 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java @@ -20,6 +20,7 @@ import static org.junit.Assert.*; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map.Entry; @@ -355,25 +356,35 @@ private BlockInfo addBlockOnNodes(long blockId, List nodes) bm.blocksMap.addINode(blockInfo, iNode); return blockInfo; } - + private DatanodeDescriptor[] scheduleSingleReplication(Block block) { - assertEquals("Block not initially pending replication", - 0, bm.pendingReplications.getNumReplicas(block)); - assertTrue("computeReplicationWork should indicate replication is needed", - bm.computeReplicationWorkForBlock(block, 1)); + // list for priority 1 + List list_p1 = new ArrayList(); + list_p1.add(block); + + // list of lists for each priority + List> list_all = new ArrayList>(); + list_all.add(new ArrayList()); // for priority 0 + list_all.add(list_p1); // for priority 1 + + assertEquals("Block not initially pending replication", 0, + bm.pendingReplications.getNumReplicas(block)); + assertEquals( + "computeReplicationWork should indicate replication is needed", 1, + bm.computeReplicationWorkForBlocks(list_all)); assertTrue("replication is pending after work is computed", bm.pendingReplications.getNumReplicas(block) > 0); - - LinkedListMultimap repls = - getAllPendingReplications(); + + LinkedListMultimap repls = getAllPendingReplications(); assertEquals(1, repls.size()); - Entry repl = repls.entries().iterator().next(); + Entry repl = repls.entries() + .iterator().next(); DatanodeDescriptor[] targets = repl.getValue().targets; - + DatanodeDescriptor[] pipeline = new DatanodeDescriptor[1 + targets.length]; pipeline[0] = repl.getKey(); System.arraycopy(targets, 0, pipeline, 1, targets.length); - + return pipeline; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java index d9309edc1d..ddef17ba0f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java @@ -137,15 +137,15 @@ private static void tryRead(Configuration conf, LocatedBlock lblock, try { DatanodeInfo[] nodes = lblock.getLocations(); targetAddr = NetUtils.createSocketAddr(nodes[0].getName()); - s = new Socket(); + s = NetUtils.getDefaultSocketFactory(conf).createSocket(); s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT); s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); String file = BlockReaderFactory.getFileName(targetAddr, "test-blockpoolid", block.getBlockId()); - blockReader = BlockReaderFactory.newBlockReader(s, file, block, - lblock.getBlockToken(), 0, -1, - conf.getInt("io.file.buffer.size", 4096)); + blockReader = BlockReaderFactory.newBlockReader( + conf, s, file, block, + lblock.getBlockToken(), 0, -1); } catch (IOException ex) { if (ex instanceof InvalidBlockTokenException) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java index df8f089863..ac82322283 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java @@ -204,13 +204,13 @@ synchronized boolean isFinalized() { @Override synchronized public BlockWriteStreams createStreams(boolean isCreate, - int bytesPerChunk, int checksumSize) throws IOException { + DataChecksum requestedChecksum) throws IOException { if (finalized) { throw new IOException("Trying to write to a finalized replica " + theBlock); } else { SimulatedOutputStream crcStream = new SimulatedOutputStream(); - return new BlockWriteStreams(oStream, crcStream); + return new BlockWriteStreams(oStream, crcStream, requestedChecksum); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java index 89e48fb586..c6fb4304da 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java @@ -56,6 +56,7 @@ public class TestDataNodeVolumeFailure { final private int block_size = 512; MiniDFSCluster cluster = null; + private Configuration conf; int dn_num = 2; int blocks_num = 30; short repl=2; @@ -74,7 +75,7 @@ private class BlockLocs { @Before public void setUp() throws Exception { // bring up a cluster of 2 - Configuration conf = new HdfsConfiguration(); + conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, block_size); // Allow a single volume failure (there are two volumes) conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1); @@ -264,7 +265,7 @@ private void accessBlock(DatanodeInfo datanode, LocatedBlock lblock) targetAddr = NetUtils.createSocketAddr(datanode.getName()); - s = new Socket(); + s = NetUtils.getDefaultSocketFactory(conf).createSocket(); s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT); s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); @@ -272,8 +273,8 @@ private void accessBlock(DatanodeInfo datanode, LocatedBlock lblock) "test-blockpoolid", block.getBlockId()); BlockReader blockReader = - BlockReaderFactory.newBlockReader(s, file, block, lblock - .getBlockToken(), 0, -1, 4096); + BlockReaderFactory.newBlockReader(conf, s, file, block, lblock + .getBlockToken(), 0, -1); // nothing - if it fails - it will throw and exception } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java index eb78aced85..d8a7f38ba3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java @@ -67,7 +67,8 @@ private static void testViewingFile(MiniDFSCluster cluster, String filePath, String viewFilePage = DFSTestUtil.urlGet(url); - assertTrue("page should show preview of file contents", viewFilePage.contains(FILE_DATA)); + assertTrue("page should show preview of file contents, got: " + viewFilePage, + viewFilePage.contains(FILE_DATA)); if (!doTail) { assertTrue("page should show link to download file", viewFilePage diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java index a1ba8f71a8..45356d9d19 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java @@ -64,7 +64,8 @@ int addSomeBlocks(FSDatasetInterface fsdataset, int startingBlockId) // we pass expected len as zero, - fsdataset should use the sizeof actual // data written ReplicaInPipelineInterface bInfo = fsdataset.createRbw(b); - BlockWriteStreams out = bInfo.createStreams(true, 512, 4); + BlockWriteStreams out = bInfo.createStreams(true, + DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_CRC32, 512)); try { OutputStream dataOut = out.dataOut; assertEquals(0, fsdataset.getLength(b)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java index 0bb43a1475..79b0bf4152 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java @@ -323,9 +323,10 @@ public void testlistCorruptFileBlocks() throws Exception { FSNamesystem.CorruptFileBlockInfo[] cfb = corruptFileBlocks .toArray(new FSNamesystem.CorruptFileBlockInfo[0]); // now get the 2nd and 3rd file that is corrupt + String[] cookie = new String[]{"1"}; Collection nextCorruptFileBlocks = namenode.getNamesystem() - .listCorruptFileBlocks("/corruptData", cfb[0].block.getBlockName()); + .listCorruptFileBlocks("/corruptData", cookie); FSNamesystem.CorruptFileBlockInfo[] ncfb = nextCorruptFileBlocks .toArray(new FSNamesystem.CorruptFileBlockInfo[0]); numCorrupt = nextCorruptFileBlocks.size(); @@ -333,9 +334,9 @@ public void testlistCorruptFileBlocks() throws Exception { assertTrue(ncfb[0].block.getBlockName() .equalsIgnoreCase(cfb[1].block.getBlockName())); - corruptFileBlocks = - namenode.getNamesystem().listCorruptFileBlocks("/corruptData", - ncfb[1].block.getBlockName()); + corruptFileBlocks = + namenode.getNamesystem() + .listCorruptFileBlocks("/corruptData", cookie); numCorrupt = corruptFileBlocks.size(); assertTrue(numCorrupt == 0); // Do a listing on a dir which doesn't have any corrupt blocks and diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDirectBufferPool.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDirectBufferPool.java new file mode 100644 index 0000000000..8d2edf3e2e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestDirectBufferPool.java @@ -0,0 +1,95 @@ +/** + * 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.hdfs.util; + +import static org.junit.Assert.*; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.junit.Test; + +import com.google.common.collect.Lists; + +public class TestDirectBufferPool { + DirectBufferPool pool = new DirectBufferPool(); + + @Test + public void testBasics() { + ByteBuffer a = pool.getBuffer(100); + assertEquals(100, a.capacity()); + assertEquals(100, a.remaining()); + pool.returnBuffer(a); + + // Getting a new buffer should return the same one + ByteBuffer b = pool.getBuffer(100); + assertSame(a, b); + + // Getting a new buffer before returning "B" should + // not return the same one + ByteBuffer c = pool.getBuffer(100); + assertNotSame(b, c); + pool.returnBuffer(b); + pool.returnBuffer(c); + } + + @Test + public void testBuffersAreReset() { + ByteBuffer a = pool.getBuffer(100); + a.putInt(0xdeadbeef); + assertEquals(96, a.remaining()); + pool.returnBuffer(a); + + // Even though we return the same buffer, + // its position should be reset to 0 + ByteBuffer b = pool.getBuffer(100); + assertSame(a, b); + assertEquals(100, a.remaining()); + pool.returnBuffer(b); + } + + @Test + public void testWeakRefClearing() { + // Allocate and return 10 buffers. + List bufs = Lists.newLinkedList(); + for (int i = 0; i < 10; i++) { + ByteBuffer buf = pool.getBuffer(100); + bufs.add(buf); + } + + for (ByteBuffer buf : bufs) { + pool.returnBuffer(buf); + } + + assertEquals(10, pool.countBuffersOfSize(100)); + + // Clear out any references to the buffers, and force + // GC. Weak refs should get cleared. + bufs.clear(); + bufs = null; + for (int i = 0; i < 3; i++) { + System.gc(); + } + + ByteBuffer buf = pool.getBuffer(100); + // the act of getting a buffer should clear all the nulled + // references from the pool. + assertEquals(0, pool.countBuffersOfSize(100)); + pool.returnBuffer(buf); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightHashSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightHashSet.java new file mode 100644 index 0000000000..e890cae854 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightHashSet.java @@ -0,0 +1,425 @@ +/** + * 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.hdfs.util; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; + +import org.junit.Test; +import org.junit.Before; +import static org.junit.Assert.*; + + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +public class TestLightWeightHashSet{ + + private static final Log LOG = LogFactory + .getLog("org.apache.hadoop.hdfs.TestLightWeightHashSet"); + private ArrayList list = new ArrayList(); + private final int NUM = 100; + private LightWeightHashSet set; + private Random rand; + + @Before + public void setUp() { + float maxF = LightWeightHashSet.DEFAULT_MAX_LOAD_FACTOR; + float minF = LightWeightHashSet.DEFAUT_MIN_LOAD_FACTOR; + int initCapacity = LightWeightHashSet.MINIMUM_CAPACITY; + rand = new Random(System.currentTimeMillis()); + list.clear(); + for (int i = 0; i < NUM; i++) { + list.add(rand.nextInt()); + } + set = new LightWeightHashSet(initCapacity, maxF, minF); + } + + @Test + public void testEmptyBasic() { + LOG.info("Test empty basic"); + Iterator iter = set.iterator(); + // iterator should not have next + assertFalse(iter.hasNext()); + assertEquals(0, set.size()); + assertTrue(set.isEmpty()); + LOG.info("Test empty - DONE"); + } + + @Test + public void testOneElementBasic() { + LOG.info("Test one element basic"); + set.add(list.get(0)); + // set should be non-empty + assertEquals(1, set.size()); + assertFalse(set.isEmpty()); + + // iterator should have next + Iterator iter = set.iterator(); + assertTrue(iter.hasNext()); + + // iterator should not have next + assertEquals(list.get(0), iter.next()); + assertFalse(iter.hasNext()); + LOG.info("Test one element basic - DONE"); + } + + @Test + public void testMultiBasic() { + LOG.info("Test multi element basic"); + // add once + for (Integer i : list) { + assertTrue(set.add(i)); + } + assertEquals(list.size(), set.size()); + + // check if the elements are in the set + for (Integer i : list) { + assertTrue(set.contains(i)); + } + + // add again - should return false each time + for (Integer i : list) { + assertFalse(set.add(i)); + } + + // check again if the elements are there + for (Integer i : list) { + assertTrue(set.contains(i)); + } + + Iterator iter = set.iterator(); + int num = 0; + while (iter.hasNext()) { + Integer next = iter.next(); + assertNotNull(next); + assertTrue(list.contains(next)); + num++; + } + // check the number of element from the iterator + assertEquals(list.size(), num); + LOG.info("Test multi element basic - DONE"); + } + + @Test + public void testRemoveOne() { + LOG.info("Test remove one"); + assertTrue(set.add(list.get(0))); + assertEquals(1, set.size()); + + // remove from the head/tail + assertTrue(set.remove(list.get(0))); + assertEquals(0, set.size()); + + // check the iterator + Iterator iter = set.iterator(); + assertFalse(iter.hasNext()); + + // add the element back to the set + assertTrue(set.add(list.get(0))); + assertEquals(1, set.size()); + + iter = set.iterator(); + assertTrue(iter.hasNext()); + LOG.info("Test remove one - DONE"); + } + + @Test + public void testRemoveMulti() { + LOG.info("Test remove multi"); + for (Integer i : list) { + assertTrue(set.add(i)); + } + for (int i = 0; i < NUM / 2; i++) { + assertTrue(set.remove(list.get(i))); + } + + // the deleted elements should not be there + for (int i = 0; i < NUM / 2; i++) { + assertFalse(set.contains(list.get(i))); + } + + // the rest should be there + for (int i = NUM / 2; i < NUM; i++) { + assertTrue(set.contains(list.get(i))); + } + LOG.info("Test remove multi - DONE"); + } + + @Test + public void testRemoveAll() { + LOG.info("Test remove all"); + for (Integer i : list) { + assertTrue(set.add(i)); + } + for (int i = 0; i < NUM; i++) { + assertTrue(set.remove(list.get(i))); + } + // the deleted elements should not be there + for (int i = 0; i < NUM; i++) { + assertFalse(set.contains(list.get(i))); + } + + // iterator should not have next + Iterator iter = set.iterator(); + assertFalse(iter.hasNext()); + assertTrue(set.isEmpty()); + LOG.info("Test remove all - DONE"); + } + + @Test + public void testPollAll() { + LOG.info("Test poll all"); + for (Integer i : list) { + assertTrue(set.add(i)); + } + // remove all elements by polling + List poll = set.pollAll(); + assertEquals(0, set.size()); + assertTrue(set.isEmpty()); + + // the deleted elements should not be there + for (int i = 0; i < NUM; i++) { + assertFalse(set.contains(list.get(i))); + } + + // we should get all original items + for (Integer i : poll) { + assertTrue(list.contains(i)); + } + + Iterator iter = set.iterator(); + assertFalse(iter.hasNext()); + LOG.info("Test poll all - DONE"); + } + + @Test + public void testPollNMulti() { + LOG.info("Test pollN multi"); + + // use addAll + set.addAll(list); + + // poll zero + List poll = set.pollN(0); + assertEquals(0, poll.size()); + for (Integer i : list) { + assertTrue(set.contains(i)); + } + + // poll existing elements (less than size) + poll = set.pollN(10); + assertEquals(10, poll.size()); + + for (Integer i : poll) { + // should be in original items + assertTrue(list.contains(i)); + // should not be in the set anymore + assertFalse(set.contains(i)); + } + + // poll more elements than present + poll = set.pollN(1000); + assertEquals(NUM - 10, poll.size()); + + for (Integer i : poll) { + // should be in original items + assertTrue(list.contains(i)); + } + + // set is empty + assertTrue(set.isEmpty()); + assertEquals(0, set.size()); + + LOG.info("Test pollN multi - DONE"); + } + + @Test + public void testPollNMultiArray() { + LOG.info("Test pollN multi array"); + + // use addAll + set.addAll(list); + + // poll existing elements (less than size) + Integer[] poll = new Integer[10]; + poll = set.pollToArray(poll); + assertEquals(10, poll.length); + + for (Integer i : poll) { + // should be in original items + assertTrue(list.contains(i)); + // should not be in the set anymore + assertFalse(set.contains(i)); + } + + // poll other elements (more than size) + poll = new Integer[NUM]; + poll = set.pollToArray(poll); + assertEquals(NUM - 10, poll.length); + + for (int i = 0; i < NUM - 10; i++) { + assertTrue(list.contains(poll[i])); + } + + // set is empty + assertTrue(set.isEmpty()); + assertEquals(0, set.size()); + + // ////// + set.addAll(list); + // poll existing elements (exactly the size) + poll = new Integer[NUM]; + poll = set.pollToArray(poll); + assertTrue(set.isEmpty()); + assertEquals(0, set.size()); + assertEquals(NUM, poll.length); + for (int i = 0; i < NUM; i++) { + assertTrue(list.contains(poll[i])); + } + // ////// + + // ////// + set.addAll(list); + // poll existing elements (exactly the size) + poll = new Integer[0]; + poll = set.pollToArray(poll); + for (int i = 0; i < NUM; i++) { + assertTrue(set.contains(list.get(i))); + } + assertEquals(0, poll.length); + // ////// + + LOG.info("Test pollN multi array- DONE"); + } + + @Test + public void testClear() { + LOG.info("Test clear"); + // use addAll + set.addAll(list); + assertEquals(NUM, set.size()); + assertFalse(set.isEmpty()); + + // clear the set + set.clear(); + assertEquals(0, set.size()); + assertTrue(set.isEmpty()); + + // iterator should be empty + Iterator iter = set.iterator(); + assertFalse(iter.hasNext()); + + LOG.info("Test clear - DONE"); + } + + @Test + public void testCapacity() { + LOG.info("Test capacity"); + float maxF = LightWeightHashSet.DEFAULT_MAX_LOAD_FACTOR; + float minF = LightWeightHashSet.DEFAUT_MIN_LOAD_FACTOR; + + // capacity lower than min_capacity + set = new LightWeightHashSet(1, maxF, minF); + assertEquals(LightWeightHashSet.MINIMUM_CAPACITY, set.getCapacity()); + + // capacity not a power of two + set = new LightWeightHashSet(30, maxF, minF); + assertEquals(Math.max(LightWeightHashSet.MINIMUM_CAPACITY, 32), + set.getCapacity()); + + // capacity valid + set = new LightWeightHashSet(64, maxF, minF); + assertEquals(Math.max(LightWeightHashSet.MINIMUM_CAPACITY, 64), + set.getCapacity()); + + // add NUM elements + set.addAll(list); + int expCap = LightWeightHashSet.MINIMUM_CAPACITY; + while (expCap < NUM && maxF * expCap < NUM) + expCap <<= 1; + assertEquals(expCap, set.getCapacity()); + + // see if the set shrinks if we remove elements by removing + set.clear(); + set.addAll(list); + int toRemove = set.size() - (int) (set.getCapacity() * minF) + 1; + for (int i = 0; i < toRemove; i++) { + set.remove(list.get(i)); + } + assertEquals(Math.max(LightWeightHashSet.MINIMUM_CAPACITY, expCap / 2), + set.getCapacity()); + + LOG.info("Test capacity - DONE"); + } + + @Test + public void testOther() { + LOG.info("Test other"); + + // remove all + assertTrue(set.addAll(list)); + assertTrue(set.removeAll(list)); + assertTrue(set.isEmpty()); + + // remove sublist + List sub = new LinkedList(); + for (int i = 0; i < 10; i++) { + sub.add(list.get(i)); + } + assertTrue(set.addAll(list)); + assertTrue(set.removeAll(sub)); + assertFalse(set.isEmpty()); + assertEquals(NUM - 10, set.size()); + + for (Integer i : sub) { + assertFalse(set.contains(i)); + } + + assertFalse(set.containsAll(sub)); + + // the rest of the elements should be there + List sub2 = new LinkedList(); + for (int i = 10; i < NUM; i++) { + sub2.add(list.get(i)); + } + assertTrue(set.containsAll(sub2)); + + // to array + Integer[] array = set.toArray(new Integer[0]); + assertEquals(NUM - 10, array.length); + for (int i = 0; i < array.length; i++) { + assertTrue(sub2.contains(array[i])); + } + assertEquals(NUM - 10, set.size()); + + // to array + Object[] array2 = set.toArray(); + assertEquals(NUM - 10, array2.length); + + for (int i = 0; i < array2.length; i++) { + assertTrue(sub2.contains((Integer) array2[i])); + } + + LOG.info("Test other - DONE"); + } + +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightLinkedSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightLinkedSet.java new file mode 100644 index 0000000000..6a9e21fb56 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightLinkedSet.java @@ -0,0 +1,363 @@ +/** + * 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.hdfs.util; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Random; + +import org.junit.Test; +import org.junit.Before; +import static org.junit.Assert.*; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; + +public class TestLightWeightLinkedSet { + + private static final Log LOG = LogFactory + .getLog("org.apache.hadoop.hdfs.TestLightWeightLinkedSet"); + private ArrayList list = new ArrayList(); + private final int NUM = 100; + private LightWeightLinkedSet set; + private Random rand; + + @Before + public void setUp() { + float maxF = LightWeightLinkedSet.DEFAULT_MAX_LOAD_FACTOR; + float minF = LightWeightLinkedSet.DEFAUT_MIN_LOAD_FACTOR; + int initCapacity = LightWeightLinkedSet.MINIMUM_CAPACITY; + rand = new Random(System.currentTimeMillis()); + list.clear(); + for (int i = 0; i < NUM; i++) { + list.add(rand.nextInt()); + } + set = new LightWeightLinkedSet(initCapacity, maxF, minF); + } + + @Test + public void testEmptyBasic() { + LOG.info("Test empty basic"); + Iterator iter = set.iterator(); + // iterator should not have next + assertFalse(iter.hasNext()); + assertEquals(0, set.size()); + assertTrue(set.isEmpty()); + + // poll should return nothing + assertNull(set.pollFirst()); + assertEquals(0, set.pollAll().size()); + assertEquals(0, set.pollN(10).size()); + + LOG.info("Test empty - DONE"); + } + + @Test + public void testOneElementBasic() { + LOG.info("Test one element basic"); + set.add(list.get(0)); + // set should be non-empty + assertEquals(1, set.size()); + assertFalse(set.isEmpty()); + + // iterator should have next + Iterator iter = set.iterator(); + assertTrue(iter.hasNext()); + + // iterator should not have next + assertEquals(list.get(0), iter.next()); + assertFalse(iter.hasNext()); + LOG.info("Test one element basic - DONE"); + } + + @Test + public void testMultiBasic() { + LOG.info("Test multi element basic"); + // add once + for (Integer i : list) { + assertTrue(set.add(i)); + } + assertEquals(list.size(), set.size()); + + // check if the elements are in the set + for (Integer i : list) { + assertTrue(set.contains(i)); + } + + // add again - should return false each time + for (Integer i : list) { + assertFalse(set.add(i)); + } + + // check again if the elements are there + for (Integer i : list) { + assertTrue(set.contains(i)); + } + + Iterator iter = set.iterator(); + int num = 0; + while (iter.hasNext()) { + assertEquals(list.get(num++), iter.next()); + } + // check the number of element from the iterator + assertEquals(list.size(), num); + LOG.info("Test multi element basic - DONE"); + } + + @Test + public void testRemoveOne() { + LOG.info("Test remove one"); + assertTrue(set.add(list.get(0))); + assertEquals(1, set.size()); + + // remove from the head/tail + assertTrue(set.remove(list.get(0))); + assertEquals(0, set.size()); + + // check the iterator + Iterator iter = set.iterator(); + assertFalse(iter.hasNext()); + + // poll should return nothing + assertNull(set.pollFirst()); + assertEquals(0, set.pollAll().size()); + assertEquals(0, set.pollN(10).size()); + + // add the element back to the set + assertTrue(set.add(list.get(0))); + assertEquals(1, set.size()); + + iter = set.iterator(); + assertTrue(iter.hasNext()); + LOG.info("Test remove one - DONE"); + } + + @Test + public void testRemoveMulti() { + LOG.info("Test remove multi"); + for (Integer i : list) { + assertTrue(set.add(i)); + } + for (int i = 0; i < NUM / 2; i++) { + assertTrue(set.remove(list.get(i))); + } + + // the deleted elements should not be there + for (int i = 0; i < NUM / 2; i++) { + assertFalse(set.contains(list.get(i))); + } + + // the rest should be there + for (int i = NUM / 2; i < NUM; i++) { + assertTrue(set.contains(list.get(i))); + } + + Iterator iter = set.iterator(); + // the remaining elements should be in order + int num = NUM / 2; + while (iter.hasNext()) { + assertEquals(list.get(num++), iter.next()); + } + assertEquals(num, NUM); + LOG.info("Test remove multi - DONE"); + } + + @Test + public void testRemoveAll() { + LOG.info("Test remove all"); + for (Integer i : list) { + assertTrue(set.add(i)); + } + for (int i = 0; i < NUM; i++) { + assertTrue(set.remove(list.get(i))); + } + // the deleted elements should not be there + for (int i = 0; i < NUM; i++) { + assertFalse(set.contains(list.get(i))); + } + + // iterator should not have next + Iterator iter = set.iterator(); + assertFalse(iter.hasNext()); + assertTrue(set.isEmpty()); + LOG.info("Test remove all - DONE"); + } + + @Test + public void testPollOneElement() { + LOG.info("Test poll one element"); + set.add(list.get(0)); + assertEquals(list.get(0), set.pollFirst()); + assertNull(set.pollFirst()); + LOG.info("Test poll one element - DONE"); + } + + @Test + public void testPollMulti() { + LOG.info("Test poll multi"); + for (Integer i : list) { + assertTrue(set.add(i)); + } + // remove half of the elements by polling + for (int i = 0; i < NUM / 2; i++) { + assertEquals(list.get(i), set.pollFirst()); + } + assertEquals(NUM / 2, set.size()); + // the deleted elements should not be there + for (int i = 0; i < NUM / 2; i++) { + assertFalse(set.contains(list.get(i))); + } + // the rest should be there + for (int i = NUM / 2; i < NUM; i++) { + assertTrue(set.contains(list.get(i))); + } + Iterator iter = set.iterator(); + // the remaining elements should be in order + int num = NUM / 2; + while (iter.hasNext()) { + assertEquals(list.get(num++), iter.next()); + } + assertEquals(num, NUM); + + // add elements back + for (int i = 0; i < NUM / 2; i++) { + assertTrue(set.add(list.get(i))); + } + // order should be switched + assertEquals(NUM, set.size()); + for (int i = NUM / 2; i < NUM; i++) { + assertEquals(list.get(i), set.pollFirst()); + } + for (int i = 0; i < NUM / 2; i++) { + assertEquals(list.get(i), set.pollFirst()); + } + assertEquals(0, set.size()); + assertTrue(set.isEmpty()); + LOG.info("Test poll multi - DONE"); + } + + @Test + public void testPollAll() { + LOG.info("Test poll all"); + for (Integer i : list) { + assertTrue(set.add(i)); + } + // remove all elements by polling + while (set.pollFirst() != null); + assertEquals(0, set.size()); + assertTrue(set.isEmpty()); + + // the deleted elements should not be there + for (int i = 0; i < NUM; i++) { + assertFalse(set.contains(list.get(i))); + } + + Iterator iter = set.iterator(); + assertFalse(iter.hasNext()); + LOG.info("Test poll all - DONE"); + } + + @Test + public void testPollNOne() { + LOG.info("Test pollN one"); + set.add(list.get(0)); + List l = set.pollN(10); + assertEquals(1, l.size()); + assertEquals(list.get(0), l.get(0)); + LOG.info("Test pollN one - DONE"); + } + + @Test + public void testPollNMulti() { + LOG.info("Test pollN multi"); + + // use addAll + set.addAll(list); + + // poll existing elements + List l = set.pollN(10); + assertEquals(10, l.size()); + + for (int i = 0; i < 10; i++) { + assertEquals(list.get(i), l.get(i)); + } + + // poll more elements than present + l = set.pollN(1000); + assertEquals(NUM - 10, l.size()); + + // check the order + for (int i = 10; i < NUM; i++) { + assertEquals(list.get(i), l.get(i - 10)); + } + // set is empty + assertTrue(set.isEmpty()); + assertEquals(0, set.size()); + + LOG.info("Test pollN multi - DONE"); + } + + @Test + public void testClear() { + LOG.info("Test clear"); + // use addAll + set.addAll(list); + assertEquals(NUM, set.size()); + assertFalse(set.isEmpty()); + + // clear the set + set.clear(); + assertEquals(0, set.size()); + assertTrue(set.isEmpty()); + + // poll should return an empty list + assertEquals(0, set.pollAll().size()); + assertEquals(0, set.pollN(10).size()); + assertNull(set.pollFirst()); + + // iterator should be empty + Iterator iter = set.iterator(); + assertFalse(iter.hasNext()); + + LOG.info("Test clear - DONE"); + } + + @Test + public void testOther() { + LOG.info("Test other"); + assertTrue(set.addAll(list)); + // to array + Integer[] array = set.toArray(new Integer[0]); + assertEquals(NUM, array.length); + for (int i = 0; i < array.length; i++) { + assertTrue(list.contains(array[i])); + } + assertEquals(NUM, set.size()); + + // to array + Object[] array2 = set.toArray(); + assertEquals(NUM, array2.length); + for (int i = 0; i < array2.length; i++) { + assertTrue(list.contains((Integer) array2[i])); + } + LOG.info("Test capacity - DONE"); + } + +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestOffsetUrlInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestOffsetUrlInputStream.java new file mode 100644 index 0000000000..4ef0dd680e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestOffsetUrlInputStream.java @@ -0,0 +1,137 @@ +/** + * 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.hdfs.web; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.IOException; +import java.net.URI; +import java.net.URL; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hdfs.TestByteRangeInputStream.MockHttpURLConnection; +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem.OffsetUrlInputStream; +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem.OffsetUrlOpener; +import org.junit.Test; + +public class TestOffsetUrlInputStream { + @Test + public void testRemoveOffset() throws IOException { + { //no offset + String s = "http://test/Abc?Length=99"; + assertEquals(s, WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString()); + } + + { //no parameters + String s = "http://test/Abc"; + assertEquals(s, WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString()); + } + + { //offset as first parameter + String s = "http://test/Abc?offset=10&Length=99"; + assertEquals("http://test/Abc?Length=99", + WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString()); + } + + { //offset as second parameter + String s = "http://test/Abc?op=read&OFFset=10&Length=99"; + assertEquals("http://test/Abc?op=read&Length=99", + WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString()); + } + + { //offset as last parameter + String s = "http://test/Abc?Length=99&offset=10"; + assertEquals("http://test/Abc?Length=99", + WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString()); + } + + { //offset as the only parameter + String s = "http://test/Abc?offset=10"; + assertEquals("http://test/Abc", + WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString()); + } + } + + @Test + public void testByteRange() throws Exception { + final Configuration conf = new Configuration(); + final String uri = WebHdfsFileSystem.SCHEME + "://localhost:50070/"; + final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)FileSystem.get(new URI(uri), conf); + + OffsetUrlOpener ospy = spy(webhdfs.new OffsetUrlOpener(new URL("http://test/"))); + doReturn(new MockHttpURLConnection(ospy.getURL())).when(ospy) + .openConnection(); + OffsetUrlOpener rspy = spy(webhdfs.new OffsetUrlOpener((URL) null)); + doReturn(new MockHttpURLConnection(rspy.getURL())).when(rspy) + .openConnection(); + final OffsetUrlInputStream is = new OffsetUrlInputStream(ospy, rspy); + + assertEquals("getPos wrong", 0, is.getPos()); + + is.read(); + + assertNull("Initial call made incorrectly (Range Check)", ospy + .openConnection().getRequestProperty("Range")); + + assertEquals("getPos should be 1 after reading one byte", 1, is.getPos()); + + is.read(); + + assertEquals("getPos should be 2 after reading two bytes", 2, is.getPos()); + + // No additional connections should have been made (no seek) + + rspy.setURL(new URL("http://resolvedurl/")); + + is.seek(100); + is.read(); + + assertEquals("getPos should be 101 after reading one byte", 101, + is.getPos()); + + verify(rspy, times(1)).openConnection(); + + is.seek(101); + is.read(); + + verify(rspy, times(1)).openConnection(); + + // Seek to 101 should not result in another request" + + is.seek(2500); + is.read(); + + ((MockHttpURLConnection) rspy.openConnection()).setResponseCode(206); + is.seek(0); + + try { + is.read(); + fail("Exception should be thrown when 206 response is given " + + "but 200 is expected"); + } catch (IOException e) { + WebHdfsFileSystem.LOG.info(e.toString()); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java index 7d990bded5..0c2372c4ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java @@ -23,9 +23,8 @@ import java.io.IOException; import java.io.InputStreamReader; import java.net.HttpURLConnection; -import java.net.URI; import java.net.URL; -import java.security.PrivilegedExceptionAction; +import java.util.Map; import javax.servlet.http.HttpServletResponse; @@ -34,12 +33,12 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystemContractBaseTest; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.web.resources.DoAsParam; import org.apache.hadoop.hdfs.web.resources.GetOpParam; import org.apache.hadoop.hdfs.web.resources.HttpOpParam; import org.apache.hadoop.hdfs.web.resources.PutOpParam; @@ -51,6 +50,8 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest { private static final Configuration conf = new Configuration(); private static final MiniDFSCluster cluster; private String defaultWorkingDirectory; + + private UserGroupInformation ugi; static { conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true); @@ -68,20 +69,11 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest { @Override protected void setUp() throws Exception { - final String uri = WebHdfsFileSystem.SCHEME + "://" - + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY); - //get file system as a non-superuser final UserGroupInformation current = UserGroupInformation.getCurrentUser(); - final UserGroupInformation ugi = UserGroupInformation.createUserForTesting( + ugi = UserGroupInformation.createUserForTesting( current.getShortUserName() + "x", new String[]{"user"}); - fs = ugi.doAs(new PrivilegedExceptionAction() { - @Override - public FileSystem run() throws Exception { - return FileSystem.get(new URI(uri), conf); - } - }); - + fs = WebHdfsTestUtil.getWebHdfsFileSystemAs(ugi, conf); defaultWorkingDirectory = fs.getWorkingDirectory().toUri().getPath(); } @@ -263,9 +255,29 @@ public void testRootDir() throws IOException { public void testResponseCode() throws IOException { final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)fs; + final Path root = new Path("/"); final Path dir = new Path("/test/testUrl"); assertTrue(webhdfs.mkdirs(dir)); + {//test GETHOMEDIRECTORY + final URL url = webhdfs.toUrl(GetOpParam.Op.GETHOMEDIRECTORY, root); + final HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + final Map m = WebHdfsTestUtil.connectAndGetJson( + conn, HttpServletResponse.SC_OK); + assertEquals(WebHdfsFileSystem.getHomeDirectoryString(ugi), + m.get(Path.class.getSimpleName())); + conn.disconnect(); + } + + {//test GETHOMEDIRECTORY with unauthorized doAs + final URL url = webhdfs.toUrl(GetOpParam.Op.GETHOMEDIRECTORY, root, + new DoAsParam(ugi.getShortUserName() + "proxy")); + final HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.connect(); + assertEquals(HttpServletResponse.SC_UNAUTHORIZED, conn.getResponseCode()); + conn.disconnect(); + } + {//test set owner with empty parameters final URL url = webhdfs.toUrl(PutOpParam.Op.SETOWNER, dir); final HttpURLConnection conn = (HttpURLConnection) url.openConnection(); @@ -280,7 +292,7 @@ public void testResponseCode() throws IOException { final HttpURLConnection conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod(op.getType().toString()); conn.connect(); - assertEquals(HttpServletResponse.SC_FORBIDDEN, conn.getResponseCode()); + assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode()); assertFalse(webhdfs.setReplication(dir, (short)1)); conn.disconnect(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java index 7cae2d6454..ec90146d60 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java @@ -75,7 +75,7 @@ public void testDelegationTokenInUrl() throws IOException { + "&token=" + tokenString, renewTokenUrl.getQuery()); Token delegationToken = new Token( token); - delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND); + delegationToken.setKind(WebHdfsFileSystem.TOKEN_KIND); Assert.assertEquals( generateUrlQueryPrefix(PutOpParam.Op.CANCELDELEGATIONTOKEN, ugi.getUserName()) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java new file mode 100644 index 0000000000..9ea0a4684a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java @@ -0,0 +1,90 @@ +/** + * 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.hdfs.web; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.security.PrivilegedExceptionAction; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.web.resources.HttpOpParam; +import org.apache.hadoop.hdfs.web.resources.Param; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.Assert; + +public class WebHdfsTestUtil { + public static final Log LOG = LogFactory.getLog(WebHdfsTestUtil.class); + + public static WebHdfsFileSystem getWebHdfsFileSystem(final Configuration conf + ) throws IOException, URISyntaxException { + final String uri = WebHdfsFileSystem.SCHEME + "://" + + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY); + return (WebHdfsFileSystem)FileSystem.get(new URI(uri), conf); + } + + public static WebHdfsFileSystem getWebHdfsFileSystemAs( + final UserGroupInformation ugi, final Configuration conf + ) throws IOException, URISyntaxException, InterruptedException { + return ugi.doAs(new PrivilegedExceptionAction() { + @Override + public WebHdfsFileSystem run() throws Exception { + return getWebHdfsFileSystem(conf); + } + }); + } + + public static URL toUrl(final WebHdfsFileSystem webhdfs, + final HttpOpParam.Op op, final Path fspath, + final Param... parameters) throws IOException { + final URL url = webhdfs.toUrl(op, fspath, parameters); + WebHdfsTestUtil.LOG.info("url=" + url); + return url; + } + + public static Map connectAndGetJson(final HttpURLConnection conn, + final int expectedResponseCode) throws IOException { + conn.connect(); + Assert.assertEquals(expectedResponseCode, conn.getResponseCode()); + return WebHdfsFileSystem.jsonParse(conn.getInputStream()); + } + + public static HttpURLConnection twoStepWrite(HttpURLConnection conn, + final HttpOpParam.Op op) throws IOException { + conn.setRequestMethod(op.getType().toString()); + conn = WebHdfsFileSystem.twoStepWrite(conn, op); + conn.setDoOutput(true); + conn.connect(); + return conn; + } + + public static FSDataOutputStream write(final WebHdfsFileSystem webhdfs, + final HttpOpParam.Op op, final HttpURLConnection conn, + final int bufferSize) throws IOException { + return webhdfs.write(op, conn, bufferSize); + } +} diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 5e7b03524a..d2aba338d9 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -10,6 +10,12 @@ Trunk (unreleased changes) (Plamen Jeliazkov via shv) IMPROVEMENTS + MAPREDUCE-3375. [Gridmix] Memory Emulation system tests. + (Vinay Thota via amarrk) + + MAPREDUCE-2733. [Gridmix] Gridmix3 cpu emulation system tests. + (Vinay Thota via amarrk) + MAPREDUCE-3008. Improvements to cumulative CPU emulation for short running tasks in Gridmix. (amarrk) @@ -29,6 +35,8 @@ Trunk (unreleased changes) uri with no authority. (John George via jitendra) BUG FIXES + MAPREDUCE-3346. [Rumen] LoggedTaskAttempt#getHostName() returns null. + (amarrk) MAPREDUCE-2950. [Gridmix] TestUserResolve fails in trunk. (Ravi Gummadi via amarrk) @@ -50,6 +58,80 @@ Trunk (unreleased changes) MAPREDUCE-3014. Rename and invert logic of '-cbuild' profile to 'native' and off by default. (tucu) +Release 0.23.1 - Unreleased + + INCOMPATIBLE CHANGES + + NEW FEATURES + + IMPROVEMENTS + + MAPREDUCE-3297. Moved log related components into yarn-common so that + HistoryServer and clients can use them without depending on the + yarn-server-nodemanager module. (Siddharth Seth via vinodkv) + + MAPREDUCE-3336. Replaced guice internal.Preconditions api usage with the + public Preconditions API. (Thomas Graves via vinodkv) + + MAPREDUCE-3280. Removed the unnecessary job user-name configuration in + mapred-site.xml. (vinodkv) + + MAPREDUCE-3370. Fixed MiniMRYarnCluster and related tests to not use + a hard-coded path for the mr-app jar. (Ahmed Radwan via vinodkv) + + MAPREDUCE-3325. Improvements to CapacityScheduler doc. (Thomas Graves + via mahadev) + + MAPREDUCE-3341. Enhance logging of initalized queue limit values. + (Anupam Seth via mahadev) + + MAPREDUCE-3243. Invalid tracking URL for streaming jobs (Jonathan Eagles + via mahadev) + + MAPREDUCE-3331. Improvement to single node cluster setup documentation for + 0.23 (Anupam Seth via mahadev) + + OPTIMIZATIONS + + BUG FIXES + + MAPREDUCE-3221. Reenabled the previously ignored test in TestSubmitJob + and fixed bugs in it. (Devaraj K via vinodkv) + + MAPREDUCE-3215. Reenabled and fixed bugs in the failing test + TestNoJobSetupCleanup. (Hitesh Shah via vinodkv) + + MAPREDUCE-3219. Reenabled and fixed bugs in the failing test + TestDelegationToken. (Hitesh Shah via vinodkv) + + MAPREDUCE-3217. Reenabled and fixed bugs in the failing ant test + TestAuditLogger. (Devaraj K via vinodkv) + + MAPREDUCE-3291. App fail to launch due to delegation token not + found in cache (Robert Evans via mahadev) + + MAPREDUCE-3344. o.a.h.mapreduce.Reducer since 0.21 blindly casts to + ReduceContext.ValueIterator. (Brock Noland via tomwhite) + + MAPREDUCE-3342. Fixed JobHistoryServer to also show the job's queue + name. (Jonathan Eagles via vinodkv) + + MAPREDUCE-3345. Fixed a race condition in ResourceManager that was causing + TestContainerManagerSecurity to fail sometimes. (Hitesh Shah via vinodkv) + + MAPREDUCE-3333. Fixed bugs in ContainerLauncher of MR AppMaster due to + which per-container connections to NodeManager were lingering long enough + to hit the ulimits on number of processes. (vinodkv) + + MAPREDUCE-3392. Fixed Cluster's getDelegationToken's API to return null + when there isn't a supported token. (John George via vinodkv) + + MAPREDUCE-3379. Fixed LocalResourceTracker in NodeManager to remove deleted + cache entries correctly. (Siddharth Seth via vinodkv) + + MAPREDUCE-3324. Not All HttpServer tools links (stacks,logs,config,metrics) are + accessible through all UI servers (Jonathan Eagles via mahadev) + Release 0.23.0 - 2011-11-01 INCOMPATIBLE CHANGES @@ -1912,6 +1994,8 @@ Release 0.23.0 - 2011-11-01 MAPREDUCE-3332. contrib/raid compile breaks due to changes in hdfs/protocol/datatransfer/ Sender#writeBlock related to checksum handling (Hitesh Shah via mahadev) + MAPREDUCE-3337. Added missing license headers. (acmurthy) + Release 0.22.0 - Unreleased INCOMPATIBLE CHANGES @@ -2506,6 +2590,8 @@ Release 0.22.0 - Unreleased MAPREDUCE-2531. Fixed jobcontrol to downgrade JobID. (Robert Evans via acmurthy) + MAPREDUCE-3139. SlivePartitioner generates negative partitions. (jghoman) + Release 0.21.1 - Unreleased NEW FEATURES diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java index 4dee678b37..4eb12204f9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java @@ -130,8 +130,8 @@ public TaskUmbilicalProtocol run() throws Exception { // Initiate Java VM metrics JvmMetrics.initSingleton(jvmId.toString(), job.getSessionId()); - LOG.debug("Remote user: " + job.get("user.name")); - childUGI = UserGroupInformation.createRemoteUser(job.get("user.name")); + childUGI = UserGroupInformation.createRemoteUser(System + .getenv(ApplicationConstants.Environment.USER.toString())); // Add tokens to new user so that it may execute its task correctly. for(Token token : UserGroupInformation.getCurrentUser().getTokens()) { childUGI.addToken(token); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java index 07f8ecc51f..ae886cf6d1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java @@ -400,6 +400,7 @@ protected void handleEvent(JobHistoryEvent event) { JobSubmittedEvent jobSubmittedEvent = (JobSubmittedEvent) event.getHistoryEvent(); mi.getJobIndexInfo().setSubmitTime(jobSubmittedEvent.getSubmitTime()); + mi.getJobIndexInfo().setQueueName(jobSubmittedEvent.getJobQueueName()); } // If this is JobFinishedEvent, close the writer and setup the job-index diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java index fdba78d9b3..4abcd34184 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java @@ -55,6 +55,7 @@ public interface Job { int getCompletedReduces(); boolean isUber(); String getUserName(); + String getQueueName(); /** * @return a path to where the config file for this job is located. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java index ea8bbdf5d6..dd19ed07b9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java @@ -142,6 +142,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, private final EventHandler eventHandler; private final MRAppMetrics metrics; private final String userName; + private final String queueName; private final long appSubmitTime; private boolean lazyTasksCopyNeeded = false; @@ -375,6 +376,7 @@ public JobImpl(JobId jobId, ApplicationAttemptId applicationAttemptId, this.completedTasksFromPreviousRun = completedTasksFromPreviousRun; this.amInfos = amInfos; this.userName = userName; + this.queueName = conf.get(MRJobConfig.QUEUE_NAME, "default"); this.appSubmitTime = appSubmitTime; this.oldJobId = TypeConverter.fromYarn(jobId); this.newApiCommitter = newApiCommitter; @@ -766,6 +768,11 @@ public String getUserName() { return userName; } + @Override + public String getQueueName() { + return queueName; + } + /* * (non-Javadoc) * @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile() @@ -829,7 +836,7 @@ public JobState transition(JobImpl job, JobEvent event) { job.conf.get(MRJobConfig.USER_NAME, "mapred"), job.appSubmitTime, job.remoteJobConfFile.toString(), - job.jobACLs, job.conf.get(MRJobConfig.QUEUE_NAME, "default")); + job.jobACLs, job.queueName); job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse)); //TODO JH Verify jobACLs, UserName via UGI? diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java index 12ac363875..d9f18b3adb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java @@ -36,5 +36,5 @@ enum EventType { /** * Maximum of 1 minute timeout for a Node to react to the command */ - static final int DEFAULT_NM__COMMAND_TIMEOUT = 60000; + static final int DEFAULT_NM_COMMAND_TIMEOUT = 60000; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java index 083ed58d9d..62ceae99f9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java @@ -21,9 +21,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.security.PrivilegedAction; -import java.util.HashMap; import java.util.HashSet; -import java.util.Map; import java.util.Set; import java.util.Timer; import java.util.TimerTask; @@ -36,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.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.ShuffleHandler; import org.apache.hadoop.mapreduce.MRJobConfig; @@ -59,11 +58,10 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerToken; -import org.apache.hadoop.yarn.factories.RecordFactory; -import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.service.AbstractService; +import org.apache.hadoop.yarn.util.Records; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -84,29 +82,28 @@ public class ContainerLauncherImpl extends AbstractService implements private Thread eventHandlingThread; private BlockingQueue eventQueue = new LinkedBlockingQueue(); - private RecordFactory recordFactory; + YarnRPC rpc; // To track numNodes. Set allNodes = new HashSet(); - // have a cache/map of proxies so as to avoid creating multiple RPC - // client connection objects for the same container. - private Map clientCache - = new HashMap(); - public ContainerLauncherImpl(AppContext context) { super(ContainerLauncherImpl.class.getName()); this.context = context; } @Override - public synchronized void init(Configuration conf) { - this.recordFactory = RecordFactoryProvider.getRecordFactory(conf); + public synchronized void init(Configuration config) { + Configuration conf = new Configuration(config); + conf.setInt( + CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, + 0); this.limitOnPoolSize = conf.getInt( MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT, MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT); this.nmTimeOut = conf.getInt(ContainerLauncher.MR_AM_NM_COMMAND_TIMEOUT, - ContainerLauncher.DEFAULT_NM__COMMAND_TIMEOUT); + ContainerLauncher.DEFAULT_NM_COMMAND_TIMEOUT); + this.rpc = YarnRPC.create(conf); super.init(conf); } @@ -180,37 +177,28 @@ protected ContainerManager getCMProxy(ContainerId containerID, UserGroupInformation user = UserGroupInformation.getCurrentUser(); - synchronized (this.clientCache) { + this.allNodes.add(containerManagerBindAddr); - if (this.clientCache.containsKey(containerID)) { - return this.clientCache.get(containerID); - } - - this.allNodes.add(containerManagerBindAddr); - - if (UserGroupInformation.isSecurityEnabled()) { - Token token = new Token( - containerToken.getIdentifier().array(), containerToken - .getPassword().array(), new Text(containerToken.getKind()), - new Text(containerToken.getService())); - // the user in createRemoteUser in this context has to be ContainerID - user = UserGroupInformation.createRemoteUser(containerID.toString()); - user.addToken(token); - } - - ContainerManager proxy = user - .doAs(new PrivilegedAction() { - @Override - public ContainerManager run() { - YarnRPC rpc = YarnRPC.create(getConfig()); - return (ContainerManager) rpc.getProxy(ContainerManager.class, - NetUtils.createSocketAddr(containerManagerBindAddr), - getConfig()); - } - }); - this.clientCache.put(containerID, proxy); - return proxy; + if (UserGroupInformation.isSecurityEnabled()) { + Token token = new Token( + containerToken.getIdentifier().array(), containerToken + .getPassword().array(), new Text(containerToken.getKind()), + new Text(containerToken.getService())); + // the user in createRemoteUser in this context has to be ContainerID + user = UserGroupInformation.createRemoteUser(containerID.toString()); + user.addToken(token); } + + ContainerManager proxy = user + .doAs(new PrivilegedAction() { + @Override + public ContainerManager run() { + return (ContainerManager) rpc.getProxy(ContainerManager.class, + NetUtils.createSocketAddr(containerManagerBindAddr), + getConfig()); + } + }); + return proxy; } private static class CommandTimer extends TimerTask { @@ -257,6 +245,8 @@ public void run() { Timer timer = new Timer(true); + ContainerManager proxy = null; + switch(event.getType()) { case CONTAINER_REMOTE_LAUNCH: @@ -267,8 +257,8 @@ public void run() { timer.schedule(new CommandTimer(Thread.currentThread(), event), nmTimeOut); - ContainerManager proxy = getCMProxy(containerID, - containerManagerBindAddr, containerToken); + proxy = getCMProxy(containerID, containerManagerBindAddr, + containerToken); // Interruped during getProxy, but that didn't throw exception if (Thread.currentThread().isInterrupted()) { @@ -284,8 +274,8 @@ public void run() { launchEvent.getContainer(); // Now launch the actual container - StartContainerRequest startRequest = recordFactory - .newRecordInstance(StartContainerRequest.class); + StartContainerRequest startRequest = Records + .newRecord(StartContainerRequest.class); startRequest.setContainerLaunchContext(containerLaunchContext); StartContainerResponse response = proxy.startContainer(startRequest); @@ -324,6 +314,7 @@ public void run() { sendContainerLaunchFailedMsg(taskAttemptID, message); } finally { timer.cancel(); + ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig()); } break; @@ -343,8 +334,8 @@ public void run() { timer.schedule(new CommandTimer(Thread.currentThread(), event), nmTimeOut); - ContainerManager proxy = getCMProxy(containerID, - containerManagerBindAddr, containerToken); + proxy = getCMProxy(containerID, containerManagerBindAddr, + containerToken); if (Thread.currentThread().isInterrupted()) { // The timer cancelled the command in the mean while. No need to @@ -356,8 +347,8 @@ public void run() { // TODO:check whether container is launched // kill the remote container if already launched - StopContainerRequest stopRequest = recordFactory - .newRecordInstance(StopContainerRequest.class); + StopContainerRequest stopRequest = Records + .newRecord(StopContainerRequest.class); stopRequest.setContainerId(event.getContainerID()); proxy.stopContainer(stopRequest); } @@ -373,6 +364,7 @@ public void run() { LOG.warn(message); } finally { timer.cancel(); + ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig()); } // after killing, send killed event to taskattempt diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java index 7a6e1f061b..5a67576c44 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java @@ -492,6 +492,11 @@ public String getUserName() { return "mock"; } + @Override + public String getQueueName() { + return "mockqueue"; + } + @Override public Path getConfFile() { throw new UnsupportedOperationException("Not supported yet."); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java index 5669070deb..f82c1d5833 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java @@ -439,6 +439,11 @@ public String getName() { throw new UnsupportedOperationException("Not supported yet."); } + @Override + public String getQueueName() { + throw new UnsupportedOperationException("Not supported yet."); + } + @Override public int getTotalMaps() { return mapTasks.size(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java index c30ee0a253..5a0f89ffd9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.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. + */ package org.apache.hadoop.mapreduce.v2.app.job.impl; import static org.junit.Assert.*; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java index f22d51c7c6..0d0271cee3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java @@ -23,6 +23,9 @@ import java.net.URLDecoder; import java.net.URLEncoder; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.v2.api.records.JobId; @@ -35,6 +38,10 @@ public class FileNameIndexUtils { static final String DELIMITER = "-"; static final String DELIMITER_ESCAPE = "%2D"; + private static final Log LOG = LogFactory.getLog(FileNameIndexUtils.class); + + // Job history file names need to be backwards compatible + // Only append new elements to the end of this list private static final int JOB_ID_INDEX = 0; private static final int SUBMIT_TIME_INDEX = 1; private static final int USER_INDEX = 2; @@ -43,7 +50,7 @@ public class FileNameIndexUtils { private static final int NUM_MAPS_INDEX = 5; private static final int NUM_REDUCES_INDEX = 6; private static final int JOB_STATUS_INDEX = 7; - private static final int MAX_INDEX = JOB_STATUS_INDEX; + private static final int QUEUE_NAME_INDEX = 8; /** * Constructs the job history file name from the JobIndexInfo. @@ -83,7 +90,11 @@ public static String getDoneFileName(JobIndexInfo indexInfo) throws IOException //JobStatus sb.append(indexInfo.getJobStatus()); + sb.append(DELIMITER); + //QueueName + sb.append(indexInfo.getQueueName()); + sb.append(JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION); return encodeJobHistoryFileName(sb.toString()); } @@ -100,27 +111,60 @@ public static JobIndexInfo getIndexInfo(String jhFileName) throws IOException { JobIndexInfo indexInfo = new JobIndexInfo(); String[] jobDetails = fileName.split(DELIMITER); - if (jobDetails.length != MAX_INDEX +1) { - throw new IOException("Failed to parse file: [" + jhFileName + "]. Expected " + (MAX_INDEX + 1) + "parts."); - } JobID oldJobId = JobID.forName(decodeJobHistoryFileName(jobDetails[JOB_ID_INDEX])); JobId jobId = TypeConverter.toYarn(oldJobId); indexInfo.setJobId(jobId); - //TODO Catch NumberFormatException - Do not fail if there's only a few fields missing. - indexInfo.setSubmitTime(Long.parseLong(decodeJobHistoryFileName(jobDetails[SUBMIT_TIME_INDEX]))); - - indexInfo.setUser(decodeJobHistoryFileName(jobDetails[USER_INDEX])); - - indexInfo.setJobName(decodeJobHistoryFileName(jobDetails[JOB_NAME_INDEX])); - - indexInfo.setFinishTime(Long.parseLong(decodeJobHistoryFileName(jobDetails[FINISH_TIME_INDEX]))); - - indexInfo.setNumMaps(Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_MAPS_INDEX]))); - - indexInfo.setNumReduces(Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_REDUCES_INDEX]))); - - indexInfo.setJobStatus(decodeJobHistoryFileName(jobDetails[JOB_STATUS_INDEX])); + + // Do not fail if there are some minor parse errors + try { + try { + indexInfo.setSubmitTime( + Long.parseLong(decodeJobHistoryFileName(jobDetails[SUBMIT_TIME_INDEX]))); + } catch (NumberFormatException e) { + LOG.warn("Unable to parse submit time from job history file " + + jhFileName + " : " + e); + } + + indexInfo.setUser( + decodeJobHistoryFileName(jobDetails[USER_INDEX])); + + indexInfo.setJobName( + decodeJobHistoryFileName(jobDetails[JOB_NAME_INDEX])); + + try { + indexInfo.setFinishTime( + Long.parseLong(decodeJobHistoryFileName(jobDetails[FINISH_TIME_INDEX]))); + } catch (NumberFormatException e) { + LOG.warn("Unable to parse finish time from job history file " + + jhFileName + " : " + e); + } + + try { + indexInfo.setNumMaps( + Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_MAPS_INDEX]))); + } catch (NumberFormatException e) { + LOG.warn("Unable to parse num maps from job history file " + + jhFileName + " : " + e); + } + + try { + indexInfo.setNumReduces( + Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_REDUCES_INDEX]))); + } catch (NumberFormatException e) { + LOG.warn("Unable to parse num reduces from job history file " + + jhFileName + " : " + e); + } + + indexInfo.setJobStatus( + decodeJobHistoryFileName(jobDetails[JOB_STATUS_INDEX])); + + indexInfo.setQueueName( + decodeJobHistoryFileName(jobDetails[QUEUE_NAME_INDEX])); + } catch (IndexOutOfBoundsException e) { + LOG.warn("Parsing job history file with partial data encoded into name: " + + jhFileName); + } return indexInfo; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobIndexInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobIndexInfo.java index 9f83b70be6..3c0f3bb3f1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobIndexInfo.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobIndexInfo.java @@ -28,6 +28,7 @@ public class JobIndexInfo { private long submitTime; private long finishTime; private String user; + private String queueName; private String jobName; private JobId jobId; private int numMaps; @@ -67,6 +68,12 @@ public String getUser() { public void setUser(String user) { this.user = user; } + public String getQueueName() { + return queueName; + } + public void setQueueName(String queueName) { + this.queueName = queueName; + } public String getJobName() { return jobName; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java index 9de3dcdfaa..3a35045ccd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestFileNameIndexUtils.java @@ -29,6 +29,16 @@ public class TestFileNameIndexUtils { + private static final String OLD_JOB_HISTORY_FILE_FORMATTER = "%s" + + FileNameIndexUtils.DELIMITER + "%s" + + FileNameIndexUtils.DELIMITER + "%s" + + FileNameIndexUtils.DELIMITER + "%s" + + FileNameIndexUtils.DELIMITER + "%s" + + FileNameIndexUtils.DELIMITER + "%s" + + FileNameIndexUtils.DELIMITER + "%s" + + FileNameIndexUtils.DELIMITER + "%s" + + JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION; + private static final String JOB_HISTORY_FILE_FORMATTER = "%s" + FileNameIndexUtils.DELIMITER + "%s" + FileNameIndexUtils.DELIMITER + "%s" @@ -37,6 +47,7 @@ public class TestFileNameIndexUtils { + FileNameIndexUtils.DELIMITER + "%s" + FileNameIndexUtils.DELIMITER + "%s" + FileNameIndexUtils.DELIMITER + "%s" + + FileNameIndexUtils.DELIMITER + "%s" + JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION; private static final String JOB_ID = "job_1317928501754_0001"; @@ -55,9 +66,48 @@ public class TestFileNameIndexUtils { private static final String NUM_MAPS = "1"; private static final String NUM_REDUCES = "1"; private static final String JOB_STATUS = "SUCCEEDED"; + private static final String QUEUE_NAME = "default"; @Test - public void testUserNamePercentEncoding() throws IOException{ + public void testEncodingDecodingEquivalence() throws IOException { + JobIndexInfo info = new JobIndexInfo(); + JobID oldJobId = JobID.forName(JOB_ID); + JobId jobId = TypeConverter.toYarn(oldJobId); + info.setJobId(jobId); + info.setSubmitTime(Long.parseLong(SUBMIT_TIME)); + info.setUser(USER_NAME); + info.setJobName(JOB_NAME); + info.setFinishTime(Long.parseLong(FINISH_TIME)); + info.setNumMaps(Integer.parseInt(NUM_MAPS)); + info.setNumReduces(Integer.parseInt(NUM_REDUCES)); + info.setJobStatus(JOB_STATUS); + info.setQueueName(QUEUE_NAME); + + String jobHistoryFile = FileNameIndexUtils.getDoneFileName(info); + JobIndexInfo parsedInfo = FileNameIndexUtils.getIndexInfo(jobHistoryFile); + + Assert.assertEquals("Job id different after encoding and decoding", + info.getJobId(), parsedInfo.getJobId()); + Assert.assertEquals("Submit time different after encoding and decoding", + info.getSubmitTime(), parsedInfo.getSubmitTime()); + Assert.assertEquals("User different after encoding and decoding", + info.getUser(), parsedInfo.getUser()); + Assert.assertEquals("Job name different after encoding and decoding", + info.getJobName(), parsedInfo.getJobName()); + Assert.assertEquals("Finish time different after encoding and decoding", + info.getFinishTime(), parsedInfo.getFinishTime()); + Assert.assertEquals("Num maps different after encoding and decoding", + info.getNumMaps(), parsedInfo.getNumMaps()); + Assert.assertEquals("Num reduces different after encoding and decoding", + info.getNumReduces(), parsedInfo.getNumReduces()); + Assert.assertEquals("Job status different after encoding and decoding", + info.getJobStatus(), parsedInfo.getJobStatus()); + Assert.assertEquals("Queue name different after encoding and decoding", + info.getQueueName(), parsedInfo.getQueueName()); + } + + @Test + public void testUserNamePercentEncoding() throws IOException { JobIndexInfo info = new JobIndexInfo(); JobID oldJobId = JobID.forName(JOB_ID); JobId jobId = TypeConverter.toYarn(oldJobId); @@ -69,6 +119,7 @@ public void testUserNamePercentEncoding() throws IOException{ info.setNumMaps(Integer.parseInt(NUM_MAPS)); info.setNumReduces(Integer.parseInt(NUM_REDUCES)); info.setJobStatus(JOB_STATUS); + info.setQueueName(QUEUE_NAME); String jobHistoryFile = FileNameIndexUtils.getDoneFileName(info); Assert.assertTrue("User name not encoded correctly into job history file", @@ -85,7 +136,8 @@ public void testUserNamePercentDecoding() throws IOException { FINISH_TIME, NUM_MAPS, NUM_REDUCES, - JOB_STATUS); + JOB_STATUS, + QUEUE_NAME); JobIndexInfo info = FileNameIndexUtils.getIndexInfo(jobHistoryFile); Assert.assertEquals("User name doesn't match", @@ -105,6 +157,7 @@ public void testJobNamePercentEncoding() throws IOException { info.setNumMaps(Integer.parseInt(NUM_MAPS)); info.setNumReduces(Integer.parseInt(NUM_REDUCES)); info.setJobStatus(JOB_STATUS); + info.setQueueName(QUEUE_NAME); String jobHistoryFile = FileNameIndexUtils.getDoneFileName(info); Assert.assertTrue("Job name not encoded correctly into job history file", @@ -121,10 +174,52 @@ public void testJobNamePercentDecoding() throws IOException { FINISH_TIME, NUM_MAPS, NUM_REDUCES, - JOB_STATUS); + JOB_STATUS, + QUEUE_NAME); JobIndexInfo info = FileNameIndexUtils.getIndexInfo(jobHistoryFile); Assert.assertEquals("Job name doesn't match", JOB_NAME_WITH_DELIMITER, info.getJobName()); } + + @Test + public void testJobHistoryFileNameBackwardsCompatible() throws IOException { + JobID oldJobId = JobID.forName(JOB_ID); + JobId jobId = TypeConverter.toYarn(oldJobId); + + long submitTime = Long.parseLong(SUBMIT_TIME); + long finishTime = Long.parseLong(FINISH_TIME); + int numMaps = Integer.parseInt(NUM_MAPS); + int numReduces = Integer.parseInt(NUM_REDUCES); + + String jobHistoryFile = String.format(OLD_JOB_HISTORY_FILE_FORMATTER, + JOB_ID, + SUBMIT_TIME, + USER_NAME, + JOB_NAME, + FINISH_TIME, + NUM_MAPS, + NUM_REDUCES, + JOB_STATUS); + + JobIndexInfo info = FileNameIndexUtils.getIndexInfo(jobHistoryFile); + Assert.assertEquals("Job id incorrect after decoding old history file", + jobId, info.getJobId()); + Assert.assertEquals("Submit time incorrect after decoding old history file", + submitTime, info.getSubmitTime()); + Assert.assertEquals("User incorrect after decoding old history file", + USER_NAME, info.getUser()); + Assert.assertEquals("Job name incorrect after decoding old history file", + JOB_NAME, info.getJobName()); + Assert.assertEquals("Finish time incorrect after decoding old history file", + finishTime, info.getFinishTime()); + Assert.assertEquals("Num maps incorrect after decoding old history file", + numMaps, info.getNumMaps()); + Assert.assertEquals("Num reduces incorrect after decoding old history file", + numReduces, info.getNumReduces()); + Assert.assertEquals("Job status incorrect after decoding old history file", + JOB_STATUS, info.getJobStatus()); + Assert.assertNull("Queue name incorrect after decoding old history file", + info.getQueueName()); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml index 138e4332aa..2f083f5275 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml @@ -34,10 +34,6 @@ org.apache.hadoop hadoop-yarn-common - - org.apache.hadoop - hadoop-yarn-server-nodemanager - org.apache.hadoop hadoop-hdfs diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java index 9475681d6d..0ff0a43284 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java @@ -1833,7 +1833,7 @@ int computeNumSlotsPerReduce(long slotSizePerReduce) { * @return a jar file that contains the class, or null. * @throws IOException */ - static String findContainingJar(Class my_class) { + public static String findContainingJar(Class my_class) { ClassLoader loader = my_class.getClassLoader(); String class_file = my_class.getName().replaceAll("\\.", "/") + ".class"; try { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java index 460202167d..4828ebacaa 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java @@ -390,6 +390,11 @@ public long getTaskTrackerExpiryInterval() throws IOException, getDelegationToken(Text renewer) throws IOException, InterruptedException{ Token result = client.getDelegationToken(renewer); + + if (result == null) { + return result; + } + InetSocketAddress addr = Master.getMasterAddress(conf); StringBuilder service = new StringBuilder(); service.append(NetUtils.normalizeHostName(addr.getAddress(). diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java index bc7ad976f3..efb0950715 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java @@ -26,6 +26,8 @@ import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.mapred.RawKeyValueIterator; +import java.util.Iterator; + /** * Reduces a set of intermediate values which share a key to a smaller set of * values. @@ -162,14 +164,15 @@ protected void cleanup(Context context * {@link #run(org.apache.hadoop.mapreduce.Reducer.Context)} method to * control how the reduce task works. */ - @SuppressWarnings("unchecked") public void run(Context context) throws IOException, InterruptedException { setup(context); while (context.nextKey()) { reduce(context.getCurrentKey(), context.getValues(), context); // If a back up store is used, reset it - ((ReduceContext.ValueIterator) - (context.getValues().iterator())).resetBackupStore(); + Iterator iter = context.getValues().iterator(); + if(iter instanceof ReduceContext.ValueIterator) { + ((ReduceContext.ValueIterator)iter).resetBackupStore(); + } } cleanup(context); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java index 7b255289b0..27ccb8a27e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java @@ -46,7 +46,7 @@ import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogDumper; +import org.apache.hadoop.yarn.logaggregation.LogDumper; /** * Interprets the map reduce cli options diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml index 9bc03caa56..5d02250c49 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml @@ -1224,12 +1224,4 @@ mapreduce.job.end-notification.max.retry.interval - - mapreduce.job.user.name - ${user.name} - The user name for the job submitter, configurable only in - non-secure mode. In secure mode Kerberos authentication is necessary. - - - diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestContextFactory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestContextFactory.java index c3d54a548b..d16bcb3351 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestContextFactory.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestContextFactory.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. + */ package org.apache.hadoop.mapreduce; import org.apache.hadoop.conf.Configuration; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java index 71680d5d33..ca4ab183ad 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java @@ -283,6 +283,11 @@ public String getName() { return jobInfo.getJobname(); } + @Override + public String getQueueName() { + return jobInfo.getJobQueueName(); + } + @Override public int getTotalMaps() { return (int) jobInfo.getTotalMaps(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java index fc808e5a70..d412a63864 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java @@ -64,6 +64,11 @@ public String getName() { return jobIndexInfo.getJobName(); } + @Override + public String getQueueName() { + return jobIndexInfo.getQueueName(); + } + @Override public JobState getState() { JobState js = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java index d78f026d2f..b7bdfa4e8f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java @@ -23,8 +23,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.v2.app.webapp.App; import org.apache.hadoop.mapreduce.v2.app.webapp.AppController; -import org.apache.hadoop.yarn.server.nodemanager.webapp.AggregatedLogsPage; import org.apache.hadoop.yarn.webapp.View; +import org.apache.hadoop.yarn.webapp.log.AggregatedLogsPage; import com.google.inject.Inject; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java index c6e3b64f3e..bf6c8450bb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java @@ -104,6 +104,7 @@ public class HsJobBlock extends HtmlBlock { ResponseInfo infoBlock = info("Job Overview"). _("Job Name:", job.getName()). _("User Name:", job.getUserName()). + _("Queue:", job.getQueueName()). _("State:", job.getState()). _("Uberized:", job.isUber()). _("Started:", new Date(startTime)). diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java index 6ef7bf0f37..0a6b9692a4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java @@ -60,6 +60,7 @@ public class HsJobsBlock extends HtmlBlock { th(".id", "Job ID"). th(".name", "Name"). th("User"). + th("Queue"). th(".state", "State"). th("Maps Total"). th("Maps Completed"). @@ -83,6 +84,7 @@ public class HsJobsBlock extends HtmlBlock { td().a(url("job", jobID), jobID)._(). td(job.getName().toString()). td(job.getUserName()). + td(job.getQueueName()). td(job.getState().toString()). td(mapsTotal). td(mapsCompleted). @@ -97,6 +99,7 @@ public class HsJobsBlock extends HtmlBlock { th().input("search_init").$type(InputType.text).$name("start_time").$value("Job ID")._()._(). th().input("search_init").$type(InputType.text).$name("start_time").$value("Name")._()._(). th().input("search_init").$type(InputType.text).$name("start_time").$value("User")._()._(). + th().input("search_init").$type(InputType.text).$name("start_time").$value("Queue")._()._(). th().input("search_init").$type(InputType.text).$name("start_time").$value("State")._()._(). th().input("search_init").$type(InputType.text).$name("start_time").$value("Maps Total")._()._(). th().input("search_init").$type(InputType.text).$name("start_time").$value("Maps Completed")._()._(). diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java index c608b83bda..909a316dbf 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java @@ -1,10 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.mapreduce.v2.hs.webapp; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.CONTAINER_ID; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.ENTITY_STRING; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING; -import org.apache.hadoop.yarn.server.nodemanager.webapp.AggregatedLogsBlock; import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.log.AggregatedLogsBlock; public class HsLogsPage extends HsView { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java index 8d3ccff63d..d4492fb92c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java @@ -68,6 +68,7 @@ public class HsNavBlock extends HtmlBlock { h3("Tools"). ul(). li().a("/conf", "Configuration")._(). + li().a("/logs", "Local logs")._(). li().a("/stacks", "Server stacks")._(). li().a("/metrics", "Server metrics")._()._()._(). div("#themeswitcher")._(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java index 01eb3f06ae..a9f08c5dc0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java @@ -18,11 +18,11 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.CONTAINER_ID; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.NM_NODENAME; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.ENTITY_STRING; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.APP_OWNER; import static org.apache.hadoop.yarn.util.StringHelper.pajoin; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_OWNER; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.NM_NODENAME; import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.webapp.AMParams; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java index 063cb6bdda..2e6bb9cf36 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java @@ -22,10 +22,10 @@ import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.ATTEMPT_STATE; import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.JOB_ID; import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.CONTAINER_ID; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.NM_NODENAME; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.ENTITY_STRING; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.APP_OWNER; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_OWNER; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.NM_NODENAME; import static org.junit.Assert.assertEquals; import java.io.IOException; @@ -44,8 +44,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.server.nodemanager.webapp.AggregatedLogsPage; import org.apache.hadoop.yarn.util.BuilderUtils; +import org.apache.hadoop.yarn.webapp.log.AggregatedLogsPage; import org.apache.hadoop.yarn.webapp.test.WebAppTests; import org.junit.Test; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml index 3a64caa826..ee4e7b4cbe 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml @@ -101,6 +101,9 @@ org.apache.maven.plugins maven-surefire-plugin + + ${project.build.directory}/${project.artifactId}-${project.version}.jar + ${project.build.directory}/${project.artifactId}-${project.version}-tests.jar diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java index 03fc8836d8..f77b6e8674 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java @@ -389,6 +389,8 @@ public ApplicationSubmissionContext createApplicationSubmissionContext( appContext.setApplicationName( // Job name jobConf.get(JobContext.JOB_NAME, YarnConfiguration.DEFAULT_APPLICATION_NAME)); + appContext.setCancelTokensWhenComplete( + conf.getBoolean(MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN, true)); appContext.setAMContainerSpec(amContainer); // AM Container return appContext; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java index 2bc9030bf8..490b7a546e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.YARNRunner; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; +import org.apache.hadoop.io.Text; import org.junit.Test; public class TestYarnClientProtocolProvider extends TestCase { @@ -56,4 +57,23 @@ public void testClusterWithYarnClientProtocolProvider() throws Exception { } } } + + + @Test + public void testClusterGetDelegationToken() throws Exception { + + Configuration conf = new Configuration(false); + Cluster cluster = null; + try { + conf = new Configuration(); + conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME); + cluster = new Cluster(conf); + cluster.getDelegationToken(new Text(" ")); + } finally { + if (cluster != null) { + cluster.close(); + } + } + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java index fca83eeb84..d4d9bb8c3a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java @@ -24,6 +24,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.LocalContainerLauncher; import org.apache.hadoop.mapred.ShuffleHandler; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.MRJobConfig; @@ -43,15 +45,8 @@ */ public class MiniMRYarnCluster extends MiniYARNCluster { - public static final String HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME = - "hadoop-mapreduce-client-app-0.24.0-SNAPSHOT.jar"; - - public static final String YARN_MAPREDUCE_APP_JAR_PATH = - "$YARN_HOME/modules/" + HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME; - - public static final String APPJAR = - "../hadoop-mapreduce-client-app/target/" - + HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME; + public static final String APPJAR = System.getProperty("yarn.mr.jar", JobConf + .findContainingJar(LocalContainerLauncher.class)); private static final Log LOG = LogFactory.getLog(MiniMRYarnCluster.class); private JobHistoryServer historyServer; @@ -71,7 +66,6 @@ public MiniMRYarnCluster(String testName, int noOfNMs) { @Override public void init(Configuration conf) { conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME); - conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name")); conf.set(MRJobConfig.MR_AM_STAGING_DIR, new File(getTestWorkDir(), "apps_staging_dir/${user.name}/").getAbsolutePath()); conf.set(MRConfig.MASTER_ADDRESS, "test"); // The default is local because of diff --git a/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml index b545bf72d0..4b7d45f66a 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -208,7 +208,7 @@ - + diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java index 2b454796c7..5ef71fea94 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java @@ -18,8 +18,11 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ClientRMProtocol; /** @@ -148,4 +151,21 @@ public interface ApplicationSubmissionContext { @Public @Stable public void setAMContainerSpec(ContainerLaunchContext amContainer); + + /** + * @return true if tokens should be canceled when the app completes. + */ + @LimitedPrivate("mapreduce") + @Unstable + public boolean getCancelTokensWhenComplete(); + + /** + * Set to false if tokens should not be canceled when the app finished else + * false. WARNING: this is not recommended unless you want your single job + * tokens to be reused by others jobs. + * @param cancel true if tokens should be canceled when the app finishes. + */ + @LimitedPrivate("mapreduce") + @Unstable + public void setCancelTokensWhenComplete(boolean cancel); } \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java index 1f8b5c24b1..9d66a167b4 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java @@ -206,6 +206,19 @@ public void setAMContainerSpec(ContainerLaunchContext amContainer) { } this.amContainer = amContainer; } + + @Override + public boolean getCancelTokensWhenComplete() { + ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder; + //There is a default so cancelTokens should never be null + return p.getCancelTokensWhenComplete(); + } + + @Override + public void setCancelTokensWhenComplete(boolean cancel) { + maybeInitBuilder(); + builder.setCancelTokensWhenComplete(cancel); + } private PriorityPBImpl convertFromProtoFormat(PriorityProto p) { return new PriorityPBImpl(p); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto index a6dfecd71c..ae6fd1b046 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto @@ -216,6 +216,7 @@ message ApplicationSubmissionContextProto { optional string queue = 4 [default = "default"]; optional PriorityProto priority = 5; optional ContainerLaunchContextProto am_container_spec = 6; + optional bool cancel_tokens_when_complete = 7 [default = true]; } enum ApplicationAccessTypeProto { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml index b3a49bff1d..60baa84d7f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml @@ -58,6 +58,12 @@ test ${project.version} + + org.apache.hadoop + hadoop-mapreduce-client-core + test + ${project.version} + org.apache.hadoop hadoop-yarn-server-tests @@ -99,6 +105,15 @@ + + org.apache.maven.plugins + maven-surefire-plugin + + + ${project.build.directory}/${project.artifactId}-${project.version}.jar + + + diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java index d0407daa2b..78d9869ef5 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -36,7 +37,8 @@ public class TestDistributedShell { protected static MiniYARNCluster yarnCluster = null; protected static Configuration conf = new Configuration(); - protected static String APPMASTER_JAR = "../hadoop-yarn-applications-distributedshell/target/hadoop-yarn-applications-distributedshell-0.24.0-SNAPSHOT.jar"; + protected static String APPMASTER_JAR = System.getProperty("yarn.ds.jar", + JobConf.findContainingJar(ApplicationMaster.class)); @BeforeClass public static void setup() throws InterruptedException, IOException { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java index 07f8c23d7a..34ad56073e 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java @@ -56,6 +56,12 @@ public ContainerManagerPBClientImpl(long clientVersion, InetSocketAddress addr, ContainerManagerService.BlockingInterface.class, clientVersion, addr, conf); } + public void close() { + if (this.proxy != null) { + RPC.stopProxy(this.proxy); + } + } + @Override public GetContainerStatusResponse getContainerStatus( GetContainerStatusRequest request) throws YarnRemoteException { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/RpcClientFactory.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/RpcClientFactory.java index 49aac3f3c7..191091f968 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/RpcClientFactory.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/RpcClientFactory.java @@ -25,6 +25,9 @@ public interface RpcClientFactory { - public Object getClient(Class protocol, long clientVersion, InetSocketAddress addr, Configuration conf) throws YarnException; + public Object getClient(Class protocol, long clientVersion, + InetSocketAddress addr, Configuration conf) throws YarnException; + + public void stopClient(Object proxy); } \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java index e6567ce318..1cb5fa0acd 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/RpcClientFactoryPBImpl.java @@ -20,16 +20,22 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.net.InetSocketAddress; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.factories.RpcClientFactory; public class RpcClientFactoryPBImpl implements RpcClientFactory { + private static final Log LOG = LogFactory + .getLog(RpcClientFactoryPBImpl.class); + private static final String PB_IMPL_PACKAGE_SUFFIX = "impl.pb.client"; private static final String PB_IMPL_CLASS_SUFFIX = "PBClientImpl"; @@ -74,9 +80,21 @@ public Object getClient(Class protocol, long clientVersion, InetSocketAddress throw new YarnException(e); } } - - - + + @Override + public void stopClient(Object proxy) { + try { + Method closeMethod = proxy.getClass().getMethod("close"); + closeMethod.invoke(proxy); + } catch (InvocationTargetException e) { + throw new YarnException(e); + } catch (Exception e) { + LOG.error("Cannot call close method due to Exception. " + + "Ignoring.", e); + throw new YarnException(e); + } + } + private String getPBImplClassName(Class clazz) { String srcPackagePart = getPackageName(clazz); String srcClassName = getClassName(clazz); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java index b40ecad235..ba1dc2ff6b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnProtoRPC.java @@ -20,15 +20,12 @@ import java.net.InetSocketAddress; -import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factory.providers.RpcFactoryProvider; /** @@ -50,6 +47,11 @@ public Object getProxy(Class protocol, InetSocketAddress addr, addr, conf); } + @Override + public void stopProxy(Object proxy, Configuration conf) { + RpcFactoryProvider.getClientFactory(conf).stopClient(proxy); + } + @Override public Server getServer(Class protocol, Object instance, InetSocketAddress addr, Configuration conf, diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java index 3a5146ae1f..838693a8f4 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java @@ -21,12 +21,12 @@ import java.io.IOException; import java.net.InetSocketAddress; -import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.AvroSpecificRpcEngine; import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.YarnException; @@ -54,6 +54,11 @@ public Object getProxy(Class protocol, InetSocketAddress addr, } } + @Override + public void stopProxy(Object proxy, Configuration conf) { + RPC.stopProxy(proxy); + } + @Override public Server getServer(Class protocol, Object instance, InetSocketAddress addr, Configuration conf, 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 9d8b846a3b..ad8938f378 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 @@ -155,6 +155,7 @@ public Object invoke(Object proxy, Method method, Object[] args) return actualReturnMessage; } + @Override public void close() throws IOException { if (!isClosed) { isClosed = true; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java index c6de9ae190..e4e61ddfaa 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/YarnRPC.java @@ -20,10 +20,10 @@ import java.net.InetSocketAddress; -import org.apache.hadoop.ipc.Server; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.Server; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.YarnException; @@ -38,6 +38,8 @@ public abstract class YarnRPC { public abstract Object getProxy(Class protocol, InetSocketAddress addr, Configuration conf); + public abstract void stopProxy(Object proxy, Configuration conf); + public abstract Server getServer(Class protocol, Object instance, InetSocketAddress addr, Configuration conf, SecretManager secretManager, 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/logaggregation/AggregatedLogFormat.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java similarity index 99% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AggregatedLogFormat.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java index 1693b8337e..b49ecc784c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AggregatedLogFormat.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation; +package org.apache.hadoop.yarn.logaggregation; import java.io.DataInput; import java.io.DataInputStream; 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/logaggregation/ContainerLogsRetentionPolicy.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRetentionPolicy.java similarity index 91% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/ContainerLogsRetentionPolicy.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRetentionPolicy.java index 82701c3e4c..b2ccd2c073 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/ContainerLogsRetentionPolicy.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRetentionPolicy.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation; +package org.apache.hadoop.yarn.logaggregation; public enum ContainerLogsRetentionPolicy { APPLICATION_MASTER_ONLY, AM_AND_FAILED_CONTAINERS_ONLY, ALL_CONTAINERS diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java new file mode 100644 index 0000000000..e8a2bd1a3f --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogAggregationUtils.java @@ -0,0 +1,107 @@ +/** +* 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.logaggregation; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +public class LogAggregationUtils { + + /** + * Constructs the full filename for an application's log file per node. + * @param remoteRootLogDir + * @param appId + * @param user + * @param nodeId + * @param suffix + * @return the remote log file. + */ + public static Path getRemoteNodeLogFileForApp(Path remoteRootLogDir, + ApplicationId appId, String user, NodeId nodeId, String suffix) { + return new Path(getRemoteAppLogDir(remoteRootLogDir, appId, user, suffix), + getNodeString(nodeId)); + } + + /** + * Gets the remote app log dir. + * @param remoteRootLogDir + * @param appId + * @param user + * @param suffix + * @return the remote application specific log dir. + */ + public static Path getRemoteAppLogDir(Path remoteRootLogDir, + ApplicationId appId, String user, String suffix) { + return new Path(getRemoteLogSuffixedDir(remoteRootLogDir, user, suffix), + appId.toString()); + } + + /** + * Gets the remote suffixed log dir for the user. + * @param remoteRootLogDir + * @param user + * @param suffix + * @return the remote suffixed log dir. + */ + public static Path getRemoteLogSuffixedDir(Path remoteRootLogDir, + String user, String suffix) { + if (suffix == null || suffix.isEmpty()) { + return getRemoteLogUserDir(remoteRootLogDir, user); + } + // TODO Maybe support suffix to be more than a single file. + return new Path(getRemoteLogUserDir(remoteRootLogDir, user), suffix); + } + + // TODO Add a utility method to list available log files. Ignore the + // temporary ones. + + /** + * Gets the remote log user dir. + * @param remoteRootLogDir + * @param user + * @return the remote per user log dir. + */ + public static Path getRemoteLogUserDir(Path remoteRootLogDir, String user) { + return new Path(remoteRootLogDir, user); + } + + /** + * Returns the suffix component of the log dir. + * @param conf + * @return the suffix which will be appended to the user log dir. + */ + public static String getRemoteNodeLogDirSuffix(Configuration conf) { + return conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, + YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX); + } + + + /** + * Converts a nodeId to a form used in the app log file name. + * @param nodeId + * @return the node string to be used to construct the file name. + */ + private static String getNodeString(NodeId nodeId) { + return nodeId.toString().replace(":", "_"); + } + +} 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/logaggregation/LogDumper.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java similarity index 93% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogDumper.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java index f5e5640c52..20206d6f94 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogDumper.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation; +package org.apache.hadoop.yarn.logaggregation; import java.io.DataInputStream; import java.io.DataOutputStream; @@ -41,8 +41,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogKey; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogReader; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader; import org.apache.hadoop.yarn.util.ConverterUtils; public class LogDumper extends Configured implements Tool { @@ -117,7 +117,7 @@ public int run(String[] args) throws Exception { YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); AggregatedLogFormat.LogReader reader = new AggregatedLogFormat.LogReader(getConf(), - LogAggregationService.getRemoteNodeLogFileForApp( + LogAggregationUtils.getRemoteNodeLogFileForApp( remoteRootLogDir, appId, appOwner, @@ -135,10 +135,10 @@ public void dumpAContainersLogs(String appId, String containerId, Path remoteRootLogDir = new Path(getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); - String suffix = LogAggregationService.getRemoteNodeLogDirSuffix(getConf()); + String suffix = LogAggregationUtils.getRemoteNodeLogDirSuffix(getConf()); AggregatedLogFormat.LogReader reader = new AggregatedLogFormat.LogReader(getConf(), - LogAggregationService.getRemoteNodeLogFileForApp(remoteRootLogDir, + LogAggregationUtils.getRemoteNodeLogFileForApp(remoteRootLogDir, ConverterUtils.toApplicationId(appId), jobOwner, ConverterUtils.toNodeId(nodeId), suffix)); DataOutputStream out = new DataOutputStream(System.out); @@ -185,7 +185,7 @@ private void dumpAllContainersLogs(ApplicationId appId, String appOwner, YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX); //TODO Change this to get a list of files from the LAS. Path remoteAppLogDir = - LogAggregationService.getRemoteAppLogDir(remoteRootLogDir, appId, user, + LogAggregationUtils.getRemoteAppLogDir(remoteRootLogDir, appId, user, logDirSuffix); RemoteIterator nodeFiles = FileContext.getFileContext().listStatus(remoteAppLogDir); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java similarity index 100% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebParams.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java similarity index 73% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebParams.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java index eac16f4ccf..fbc0887e12 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebParams.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java @@ -16,13 +16,13 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.nodemanager.webapp; +package org.apache.hadoop.yarn.webapp; -public interface NMWebParams { +public interface YarnWebParams { String NM_NODENAME = "nm.id"; - String APPLICATION_ID = "nm.appId"; - String CONTAINER_ID = "nm.containerId"; - String CONTAINER_LOG_TYPE= "nm.containerLogType"; - String ENTITY_STRING = "nm.entityString"; - String APP_OWNER = "nm.appOwner"; + String APPLICATION_ID = "app.id"; + String CONTAINER_ID = "container.id"; + String CONTAINER_LOG_TYPE= "log.type"; + String ENTITY_STRING = "entity.string"; + String APP_OWNER = "app.owner"; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java new file mode 100644 index 0000000000..5698e04dc8 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java @@ -0,0 +1,183 @@ +package org.apache.hadoop.yarn.webapp.log; + +import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_OWNER; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.NM_NODENAME; + +import java.io.DataInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationAccessType; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey; +import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils; +import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; + +import com.google.inject.Inject; + +public class AggregatedLogsBlock extends HtmlBlock { + + private final Configuration conf; + + @Inject + AggregatedLogsBlock(Configuration conf) { + this.conf = conf; + } + + @Override + protected void render(Block html) { + ContainerId containerId = verifyAndGetContainerId(html); + NodeId nodeId = verifyAndGetNodeId(html); + String appOwner = verifyAndGetAppOwner(html); + if (containerId == null || nodeId == null || appOwner == null + || appOwner.isEmpty()) { + return; + } + + ApplicationId applicationId = + containerId.getApplicationAttemptId().getApplicationId(); + String logEntity = $(ENTITY_STRING); + if (logEntity == null || logEntity.isEmpty()) { + logEntity = containerId.toString(); + } + + if (!conf.getBoolean(YarnConfiguration.NM_LOG_AGGREGATION_ENABLED, + YarnConfiguration.DEFAULT_NM_LOG_AGGREGATION_ENABLED)) { + html.h1() + ._("Aggregation is not enabled. Try the nodemanager at " + nodeId) + ._(); + return; + } + + Path remoteRootLogDir = + new Path(conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, + YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); + AggregatedLogFormat.LogReader reader = null; + try { + reader = + new AggregatedLogFormat.LogReader(conf, + LogAggregationUtils.getRemoteNodeLogFileForApp( + remoteRootLogDir, applicationId, appOwner, nodeId, + LogAggregationUtils.getRemoteNodeLogDirSuffix(conf))); + } catch (FileNotFoundException e) { + // ACLs not available till the log file is opened. + html.h1() + ._("Logs not available for " + + logEntity + + ". Aggregation may not be complete, " + + "Check back later or try the nodemanager at " + + nodeId)._(); + return; + } catch (IOException e) { + html.h1()._("Error getting logs for " + logEntity)._(); + LOG.error("Error getting logs for " + logEntity, e); + return; + } + + String owner = null; + Map appAcls = null; + try { + owner = reader.getApplicationOwner(); + appAcls = reader.getApplicationAcls(); + } catch (IOException e) { + html.h1()._("Error getting logs for " + logEntity)._(); + LOG.error("Error getting logs for " + logEntity, e); + return; + } + ApplicationACLsManager aclsManager = new ApplicationACLsManager(conf); + aclsManager.addApplication(applicationId, appAcls); + + String remoteUser = request().getRemoteUser(); + UserGroupInformation callerUGI = null; + if (remoteUser != null) { + callerUGI = UserGroupInformation.createRemoteUser(remoteUser); + } + if (callerUGI != null + && !aclsManager.checkAccess(callerUGI, ApplicationAccessType.VIEW_APP, + owner, applicationId)) { + html.h1() + ._("User [" + remoteUser + + "] is not authorized to view the logs for " + logEntity)._(); + return; + } + + DataInputStream valueStream; + LogKey key = new LogKey(); + try { + valueStream = reader.next(key); + while (valueStream != null + && !key.toString().equals(containerId.toString())) { + valueStream = reader.next(key); + } + if (valueStream == null) { + html.h1()._( + "Logs not available for " + logEntity + + ". Could be caused by the rentention policy")._(); + return; + } + writer().write("
");
+      AggregatedLogFormat.LogReader.readAcontainerLogs(valueStream, writer());
+      writer().write("
"); + return; + } catch (IOException e) { + html.h1()._("Error getting logs for " + logEntity)._(); + LOG.error("Error getting logs for " + logEntity, e); + return; + } + } + + private ContainerId verifyAndGetContainerId(Block html) { + String containerIdStr = $(CONTAINER_ID); + if (containerIdStr == null || containerIdStr.isEmpty()) { + html.h1()._("Cannot get container logs without a ContainerId")._(); + return null; + } + ContainerId containerId = null; + try { + containerId = ConverterUtils.toContainerId(containerIdStr); + } catch (IllegalArgumentException e) { + html.h1() + ._("Cannot get container logs for invalid containerId: " + + containerIdStr)._(); + return null; + } + return containerId; + } + + private NodeId verifyAndGetNodeId(Block html) { + String nodeIdStr = $(NM_NODENAME); + if (nodeIdStr == null || nodeIdStr.isEmpty()) { + html.h1()._("Cannot get container logs without a NodeId")._(); + return null; + } + NodeId nodeId = null; + try { + nodeId = ConverterUtils.toNodeId(nodeIdStr); + } catch (IllegalArgumentException e) { + html.h1()._("Cannot get container logs. Invalid nodeId: " + nodeIdStr) + ._(); + return null; + } + return nodeId; + } + + private String verifyAndGetAppOwner(Block html) { + String appOwner = $(APP_OWNER); + if (appOwner == null || appOwner.isEmpty()) { + html.h1()._("Cannot get container logs without an app owner")._(); + } + return appOwner; + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsNavBlock.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsNavBlock.java similarity index 88% rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsNavBlock.java rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsNavBlock.java index bde6db93c8..2b785525ad 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsNavBlock.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsNavBlock.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.nodemanager.webapp; +package org.apache.hadoop.yarn.webapp.log; import org.apache.hadoop.yarn.webapp.view.HtmlBlock; -public class AggregatedLogsNavBlock extends HtmlBlock implements NMWebParams { +public class AggregatedLogsNavBlock extends HtmlBlock { @Override protected void render(Block html) { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsPage.java new file mode 100644 index 0000000000..add4fc6853 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsPage.java @@ -0,0 +1,45 @@ +package org.apache.hadoop.yarn.webapp.log; + +import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING; +import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.THEMESWITCHER_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; + + +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout; + + +public class AggregatedLogsPage extends TwoColumnLayout { + + /* (non-Javadoc) + * @see org.apache.hadoop.yarn.server.nodemanager.webapp.NMView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) + */ + @Override + protected void preHead(Page.HTML<_> html) { + String logEntity = $(ENTITY_STRING); + if (logEntity == null || logEntity.isEmpty()) { + logEntity = $(CONTAINER_ID); + } + if (logEntity == null || logEntity.isEmpty()) { + logEntity = "UNKNOWN"; + } + set(TITLE, join("Logs for ", logEntity)); + set(ACCORDION_ID, "nav"); + set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}"); + set(THEMESWITCHER_ID, "themeswitcher"); + } + + @Override + protected Class content() { + return AggregatedLogsBlock.class; + } + + @Override + protected Class nav() { + return AggregatedLogsNavBlock.class; + } +} 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/application/ApplicationImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java index 9cc5d6f786..afdbe5b082 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType; @@ -41,7 +42,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.ContainerLogsRetentionPolicy; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; 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/localizer/LocalResourcesTrackerImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java index 98f665404f..7127db97c0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java @@ -101,13 +101,14 @@ public boolean remove(LocalizedResource rem, DeletionService delService) { // internal error LOG.error("Attempt to remove resource: " + rsrc + " with non-zero refcount"); - assert false; return false; + } else { // ResourceState is LOCALIZED or INIT + localrsrc.remove(rem.getRequest()); + if (ResourceState.LOCALIZED.equals(rsrc.getState())) { + delService.delete(getUser(), getPathToDelete(rsrc.getLocalPath())); + } + return true; } - if (ResourceState.LOCALIZED.equals(rsrc.getState())) { - delService.delete(getUser(), getPathToDelete(rsrc.getLocalPath())); - } - return true; } 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/logaggregation/AppLogAggregatorImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java index 5db1b5de50..c41162bbec 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java @@ -35,12 +35,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogWriter; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogKey; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogValue; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogWriter; import org.apache.hadoop.yarn.util.ConverterUtils; public class AppLogAggregatorImpl implements AppLogAggregator { 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/logaggregation/LogAggregationService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java index d651cb9853..95885d4e07 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java @@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy; +import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler; @@ -138,83 +140,7 @@ public synchronized void stop() { super.stop(); } - /** - * Constructs the full filename for an application's log file per node. - * @param remoteRootLogDir - * @param appId - * @param user - * @param nodeId - * @param suffix - * @return the remote log file. - */ - public static Path getRemoteNodeLogFileForApp(Path remoteRootLogDir, - ApplicationId appId, String user, NodeId nodeId, String suffix) { - return new Path(getRemoteAppLogDir(remoteRootLogDir, appId, user, suffix), - getNodeString(nodeId)); - } - - /** - * Gets the remote app log dir. - * @param remoteRootLogDir - * @param appId - * @param user - * @param suffix - * @return the remote application specific log dir. - */ - public static Path getRemoteAppLogDir(Path remoteRootLogDir, - ApplicationId appId, String user, String suffix) { - return new Path(getRemoteLogSuffixedDir(remoteRootLogDir, user, suffix), - appId.toString()); - } - - /** - * Gets the remote suffixed log dir for the user. - * @param remoteRootLogDir - * @param user - * @param suffix - * @return the remote suffixed log dir. - */ - private static Path getRemoteLogSuffixedDir(Path remoteRootLogDir, - String user, String suffix) { - if (suffix == null || suffix.isEmpty()) { - return getRemoteLogUserDir(remoteRootLogDir, user); - } - // TODO Maybe support suffix to be more than a single file. - return new Path(getRemoteLogUserDir(remoteRootLogDir, user), suffix); - } - - // TODO Add a utility method to list available log files. Ignore the - // temporary ones. - /** - * Gets the remote log user dir. - * @param remoteRootLogDir - * @param user - * @return the remote per user log dir. - */ - private static Path getRemoteLogUserDir(Path remoteRootLogDir, String user) { - return new Path(remoteRootLogDir, user); - } - - /** - * Returns the suffix component of the log dir. - * @param conf - * @return the suffix which will be appended to the user log dir. - */ - public static String getRemoteNodeLogDirSuffix(Configuration conf) { - return conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, - YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX); - } - - - /** - * Converts a nodeId to a form used in the app log file name. - * @param nodeId - * @return the node string to be used to construct the file name. - */ - private static String getNodeString(NodeId nodeId) { - return nodeId.toString().replace(":", "_"); - } @@ -268,7 +194,7 @@ private void verifyAndCreateRemoteLogDir(Configuration conf) { } Path getRemoteNodeLogFileForApp(ApplicationId appId, String user) { - return LogAggregationService.getRemoteNodeLogFileForApp( + return LogAggregationUtils.getRemoteNodeLogFileForApp( this.remoteRootLogDir, appId, user, this.nodeId, this.remoteRootLogDirSuffix); } @@ -299,7 +225,7 @@ public Object run() throws Exception { } try { userDir = - getRemoteLogUserDir( + LogAggregationUtils.getRemoteLogUserDir( LogAggregationService.this.remoteRootLogDir, user); userDir = userDir.makeQualified(remoteFS.getUri(), @@ -312,7 +238,7 @@ public Object run() throws Exception { } try { suffixDir = - getRemoteLogSuffixedDir( + LogAggregationUtils.getRemoteLogSuffixedDir( LogAggregationService.this.remoteRootLogDir, user, LogAggregationService.this.remoteRootLogDirSuffix); suffixDir = @@ -326,8 +252,8 @@ public Object run() throws Exception { } try { appDir = - getRemoteAppLogDir(LogAggregationService.this.remoteRootLogDir, - appId, user, + LogAggregationUtils.getRemoteAppLogDir( + LogAggregationService.this.remoteRootLogDir, appId, user, LogAggregationService.this.remoteRootLogDirSuffix); appDir = appDir.makeQualified(remoteFS.getUri(), 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/loghandler/NonAggregatingLogHandler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java index 96833f3d48..e0f843e245 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.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. + */ package org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler; import java.util.Map; 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/loghandler/event/LogHandlerAppStartedEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/event/LogHandlerAppStartedEvent.java index a598bed264..6c076748a4 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/event/LogHandlerAppStartedEvent.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/event/LogHandlerAppStartedEvent.java @@ -23,7 +23,7 @@ import org.apache.hadoop.security.Credentials; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.ContainerLogsRetentionPolicy; +import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy; public class LogHandlerAppStartedEvent extends LogHandlerEvent { 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/monitor/ContainersMonitorImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index d0e53a0c68..c633014c1c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree; import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin; -import com.google.inject.internal.Preconditions; +import com.google.common.base.Preconditions; public class ContainersMonitorImpl extends AbstractService implements ContainersMonitor { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/ProcessIdFileReader.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/ProcessIdFileReader.java index 74f37eed3a..279aa29858 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/ProcessIdFileReader.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/ProcessIdFileReader.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. + */ package org.apache.hadoop.yarn.server.nodemanager.util; import java.io.BufferedReader; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsBlock.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsBlock.java index 9a75aa24e4..5d5f1e35f1 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsBlock.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsBlock.java @@ -1,183 +1,17 @@ -package org.apache.hadoop.yarn.server.nodemanager.webapp; - -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.CONTAINER_ID; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.NM_NODENAME; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.ENTITY_STRING; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.APP_OWNER; - -import java.io.DataInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.Map; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.records.ApplicationAccessType; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogKey; -import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; -import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.hadoop.yarn.webapp.view.HtmlBlock; - -import com.google.inject.Inject; - -public class AggregatedLogsBlock extends HtmlBlock { - - private final Configuration conf; - - @Inject - AggregatedLogsBlock(Configuration conf) { - this.conf = conf; - } - - @Override - protected void render(Block html) { - ContainerId containerId = verifyAndGetContainerId(html); - NodeId nodeId = verifyAndGetNodeId(html); - String appOwner = verifyAndGetAppOwner(html); - if (containerId == null || nodeId == null || appOwner == null - || appOwner.isEmpty()) { - return; - } - - ApplicationId applicationId = - containerId.getApplicationAttemptId().getApplicationId(); - String logEntity = $(ENTITY_STRING); - if (logEntity == null || logEntity.isEmpty()) { - logEntity = containerId.toString(); - } - - if (!conf.getBoolean(YarnConfiguration.NM_LOG_AGGREGATION_ENABLED, - YarnConfiguration.DEFAULT_NM_LOG_AGGREGATION_ENABLED)) { - html.h1() - ._("Aggregation is not enabled. Try the nodemanager at " + nodeId) - ._(); - return; - } - - Path remoteRootLogDir = - new Path(conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, - YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); - AggregatedLogFormat.LogReader reader = null; - try { - reader = - new AggregatedLogFormat.LogReader(conf, - LogAggregationService.getRemoteNodeLogFileForApp( - remoteRootLogDir, applicationId, appOwner, nodeId, - LogAggregationService.getRemoteNodeLogDirSuffix(conf))); - } catch (FileNotFoundException e) { - // ACLs not available till the log file is opened. - html.h1() - ._("Logs not available for " - + logEntity - + ". Aggregation may not be complete, " - + "Check back later or try the nodemanager at " - + nodeId)._(); - return; - } catch (IOException e) { - html.h1()._("Error getting logs for " + logEntity)._(); - LOG.error("Error getting logs for " + logEntity, e); - return; - } - - String owner = null; - Map appAcls = null; - try { - owner = reader.getApplicationOwner(); - appAcls = reader.getApplicationAcls(); - } catch (IOException e) { - html.h1()._("Error getting logs for " + logEntity)._(); - LOG.error("Error getting logs for " + logEntity, e); - return; - } - ApplicationACLsManager aclsManager = new ApplicationACLsManager(conf); - aclsManager.addApplication(applicationId, appAcls); - - String remoteUser = request().getRemoteUser(); - UserGroupInformation callerUGI = null; - if (remoteUser != null) { - callerUGI = UserGroupInformation.createRemoteUser(remoteUser); - } - if (callerUGI != null - && !aclsManager.checkAccess(callerUGI, ApplicationAccessType.VIEW_APP, - owner, applicationId)) { - html.h1() - ._("User [" + remoteUser - + "] is not authorized to view the logs for " + logEntity)._(); - return; - } - - DataInputStream valueStream; - LogKey key = new LogKey(); - try { - valueStream = reader.next(key); - while (valueStream != null - && !key.toString().equals(containerId.toString())) { - valueStream = reader.next(key); - } - if (valueStream == null) { - html.h1()._( - "Logs not available for " + logEntity - + ". Could be caused by the rentention policy")._(); - return; - } - writer().write("
");
-      AggregatedLogFormat.LogReader.readAcontainerLogs(valueStream, writer());
-      writer().write("
"); - return; - } catch (IOException e) { - html.h1()._("Error getting logs for " + logEntity)._(); - LOG.error("Error getting logs for " + logEntity, e); - return; - } - } - - private ContainerId verifyAndGetContainerId(Block html) { - String containerIdStr = $(CONTAINER_ID); - if (containerIdStr == null || containerIdStr.isEmpty()) { - html.h1()._("Cannot get container logs without a ContainerId")._(); - return null; - } - ContainerId containerId = null; - try { - containerId = ConverterUtils.toContainerId(containerIdStr); - } catch (IllegalArgumentException e) { - html.h1() - ._("Cannot get container logs for invalid containerId: " - + containerIdStr)._(); - return null; - } - return containerId; - } - - private NodeId verifyAndGetNodeId(Block html) { - String nodeIdStr = $(NM_NODENAME); - if (nodeIdStr == null || nodeIdStr.isEmpty()) { - html.h1()._("Cannot get container logs without a NodeId")._(); - return null; - } - NodeId nodeId = null; - try { - nodeId = ConverterUtils.toNodeId(nodeIdStr); - } catch (IllegalArgumentException e) { - html.h1()._("Cannot get container logs. Invalid nodeId: " + nodeIdStr) - ._(); - return null; - } - return nodeId; - } - - private String verifyAndGetAppOwner(Block html) { - String appOwner = $(APP_OWNER); - if (appOwner == null || appOwner.isEmpty()) { - html.h1()._("Cannot get container logs without an app owner")._(); - } - return appOwner; - } -} \ No newline at end of file +/** + * 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. + */ diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsPage.java index e433b0a385..5d5f1e35f1 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AggregatedLogsPage.java @@ -1,44 +1,17 @@ -package org.apache.hadoop.yarn.server.nodemanager.webapp; - -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.CONTAINER_ID; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.ENTITY_STRING; -import static org.apache.hadoop.yarn.util.StringHelper.join; -import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; -import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID; -import static org.apache.hadoop.yarn.webapp.view.JQueryUI.THEMESWITCHER_ID; -import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; - - -import org.apache.hadoop.yarn.webapp.SubView; - - -public class AggregatedLogsPage extends NMView { - - /* (non-Javadoc) - * @see org.apache.hadoop.yarn.server.nodemanager.webapp.NMView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML) - */ - @Override - protected void preHead(Page.HTML<_> html) { - String logEntity = $(ENTITY_STRING); - if (logEntity == null || logEntity.isEmpty()) { - logEntity = $(CONTAINER_ID); - } - if (logEntity == null || logEntity.isEmpty()) { - logEntity = "UNKNOWN"; - } - set(TITLE, join("Logs for ", logEntity)); - set(ACCORDION_ID, "nav"); - set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}"); - set(THEMESWITCHER_ID, "themeswitcher"); - } - - @Override - protected Class content() { - return AggregatedLogsBlock.class; - } - - @Override - protected Class nav() { - return AggregatedLogsNavBlock.class; - } -} +/** + * 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. + */ diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java index a35f81ffb2..49b75373d7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java @@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.SubView; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.BODY; @@ -60,7 +61,7 @@ protected Class content() { } public static class AllApplicationsBlock extends HtmlBlock implements - NMWebParams { + YarnWebParams { private final Context nmContext; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java index 9c2dc47ba2..13c8951aa6 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java @@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.SubView; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.BODY; @@ -60,7 +61,7 @@ protected Class content() { } public static class AllContainersBlock extends HtmlBlock implements - NMWebParams { + YarnWebParams { private final Context nmContext; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java index 4fd26be49b..fc02120c6e 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java @@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.SubView; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE; @@ -42,7 +43,7 @@ import com.google.inject.Inject; -public class ApplicationPage extends NMView implements NMWebParams { +public class ApplicationPage extends NMView implements YarnWebParams { @Override protected void preHead(Page.HTML<_> html) { commonPreHead(html); @@ -61,7 +62,7 @@ protected Class content() { } public static class ApplicationBlock extends HtmlBlock implements - NMWebParams { + YarnWebParams { private final Context nmContext; private final Configuration conf; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java index 84d2a4a271..faf0cbc47f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java @@ -18,8 +18,8 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp; -import static org.apache.hadoop.yarn.server.nodemanager.webapp.NMWebParams.CONTAINER_ID; import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID; import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID; import static org.apache.hadoop.yarn.webapp.view.JQueryUI.THEMESWITCHER_ID; @@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.SubView; import org.apache.hadoop.yarn.webapp.view.HtmlBlock; @@ -84,7 +85,7 @@ protected Class content() { } public static class ContainersLogsBlock extends HtmlBlock implements - NMWebParams { + YarnWebParams { private final Configuration conf; private final LocalDirAllocator logsSelector; private final Context nmContext; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java index aec44295a7..6633aa0349 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.SubView; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV; @@ -36,7 +37,7 @@ import com.google.inject.Inject; -public class ContainerPage extends NMView implements NMWebParams { +public class ContainerPage extends NMView implements YarnWebParams { @Override protected void preHead(Page.HTML<_> html) { @@ -50,7 +51,7 @@ protected Class content() { return ContainerBlock.class; } - public static class ContainerBlock extends HtmlBlock implements NMWebParams { + public static class ContainerBlock extends HtmlBlock implements YarnWebParams { private final Context nmContext; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMController.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMController.java index cf69b0f4ee..033271afde 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMController.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMController.java @@ -28,10 +28,11 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.webapp.Controller; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import com.google.inject.Inject; -public class NMController extends Controller implements NMWebParams { +public class NMController extends Controller implements YarnWebParams { private Context nmContext; private Configuration nmConf; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java index 01ea4aa9ee..89fcc2047f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java @@ -18,9 +18,10 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.view.HtmlBlock; -public class NavBlock extends HtmlBlock implements NMWebParams { +public class NavBlock extends HtmlBlock implements YarnWebParams { @Override protected void render(Block html) { @@ -36,8 +37,12 @@ protected void render(Block html) { .li() .a(url("allContainers"), "List of Containers")._() ._() - .h3()._("Tools")._() - ._() + .h3("Tools") + .ul() + .li().a("/conf", "Configuration")._() + .li().a("/logs", "Local logs")._() + .li().a("/stacks", "Server stacks")._() + .li().a("/metrics", "Server metrics")._()._()._() .div("#themeswitcher")._(); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java index 5df3b9edbd..2573015877 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/WebServer.java @@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.server.nodemanager.ResourceView; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.service.AbstractService; +import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.WebApp; import org.apache.hadoop.yarn.webapp.WebApps; @@ -76,7 +77,7 @@ public synchronized void stop() { super.stop(); } - public static class NMWebApp extends WebApp implements NMWebParams { + public static class NMWebApp extends WebApp implements YarnWebParams { private final ResourceView resourceView; private final ApplicationACLsManager aclsManager; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java index 111d5c4e8f..236489b7f4 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.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. + */ package org.apache.hadoop.yarn.server.nodemanager.containermanager.application; import static org.junit.Assert.assertEquals; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java new file mode 100644 index 0000000000..7f0e353697 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java @@ -0,0 +1,166 @@ +package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.util.Iterator; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.DrainDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.nodemanager.DeletionService; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerEventType; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerResourceRequestEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceLocalizedEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceReleaseEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRequestEvent; +import org.apache.hadoop.yarn.util.BuilderUtils; +import org.junit.Test; + +public class TestLocalResourcesTrackerImpl { + + @Test + @SuppressWarnings("unchecked") + public void test() { + String user = "testuser"; + DrainDispatcher dispatcher = null; + try { + dispatcher = createDispatcher(new Configuration()); + EventHandler localizerEventHandler = + mock(EventHandler.class); + EventHandler containerEventHandler = + mock(EventHandler.class); + dispatcher.register(LocalizerEventType.class, localizerEventHandler); + dispatcher.register(ContainerEventType.class, containerEventHandler); + + DeletionService mockDelService = mock(DeletionService.class); + + ContainerId cId1 = BuilderUtils.newContainerId(1, 1, 1, 1); + LocalizerContext lc1 = new LocalizerContext(user, cId1, null); + ContainerId cId2 = BuilderUtils.newContainerId(1, 1, 1, 2); + LocalizerContext lc2 = new LocalizerContext(user, cId2, null); + + LocalResourceRequest req1 = + createLocalResourceRequest(user, 1, 1, LocalResourceVisibility.PUBLIC); + LocalResourceRequest req2 = + createLocalResourceRequest(user, 2, 1, LocalResourceVisibility.PUBLIC); + LocalizedResource lr1 = createLocalizedResource(req1, dispatcher); + LocalizedResource lr2 = createLocalizedResource(req2, dispatcher); + ConcurrentMap localrsrc = + new ConcurrentHashMap(); + localrsrc.put(req1, lr1); + localrsrc.put(req2, lr2); + LocalResourcesTracker tracker = + new LocalResourcesTrackerImpl(user, dispatcher, localrsrc); + + ResourceEvent req11Event = + new ResourceRequestEvent(req1, LocalResourceVisibility.PUBLIC, lc1); + ResourceEvent req12Event = + new ResourceRequestEvent(req1, LocalResourceVisibility.PUBLIC, lc2); + ResourceEvent req21Event = + new ResourceRequestEvent(req2, LocalResourceVisibility.PUBLIC, lc1); + + ResourceEvent rel11Event = new ResourceReleaseEvent(req1, cId1); + ResourceEvent rel12Event = new ResourceReleaseEvent(req1, cId2); + ResourceEvent rel21Event = new ResourceReleaseEvent(req2, cId1); + + // Localize R1 for C1 + tracker.handle(req11Event); + + // Localize R1 for C2 + tracker.handle(req12Event); + + // Localize R2 for C1 + tracker.handle(req21Event); + + dispatcher.await(); + verify(localizerEventHandler, times(3)).handle( + any(LocalizerResourceRequestEvent.class)); + // Verify refCount for R1 is 2 + Assert.assertEquals(2, lr1.getRefCount()); + // Verify refCount for R2 is 1 + Assert.assertEquals(1, lr2.getRefCount()); + + // Release R2 for C1 + tracker.handle(rel21Event); + + dispatcher.await(); + verifyTrackedResourceCount(tracker, 2); + + // Verify resources in state INIT with ref-count=0 is removed. + Assert.assertTrue(tracker.remove(lr2, mockDelService)); + verifyTrackedResourceCount(tracker, 1); + + // Verify resource with non zero ref count is not removed. + Assert.assertEquals(2, lr1.getRefCount()); + Assert.assertFalse(tracker.remove(lr1, mockDelService)); + verifyTrackedResourceCount(tracker, 1); + + // Localize resource1 + ResourceLocalizedEvent rle = + new ResourceLocalizedEvent(req1, new Path("file:///tmp/r1"), 1); + lr1.handle(rle); + Assert.assertTrue(lr1.getState().equals(ResourceState.LOCALIZED)); + + // Release resource1 + tracker.handle(rel11Event); + tracker.handle(rel12Event); + Assert.assertEquals(0, lr1.getRefCount()); + + // Verify resources in state LOCALIZED with ref-count=0 is removed. + Assert.assertTrue(tracker.remove(lr1, mockDelService)); + verifyTrackedResourceCount(tracker, 0); + } finally { + if (dispatcher != null) { + dispatcher.stop(); + } + } + } + + private void verifyTrackedResourceCount(LocalResourcesTracker tracker, + int expected) { + int count = 0; + Iterator iter = tracker.iterator(); + while (iter.hasNext()) { + iter.next(); + count++; + } + Assert.assertEquals("Tracker resource count does not match", expected, + count); + } + + private LocalResourceRequest createLocalResourceRequest(String user, int i, + long ts, LocalResourceVisibility vis) { + final LocalResourceRequest req = + new LocalResourceRequest(new Path("file:///tmp/" + user + "/rsrc" + i), + ts + i * 2000, LocalResourceType.FILE, vis); + return req; + } + + private LocalizedResource createLocalizedResource(LocalResourceRequest req, + Dispatcher dispatcher) { + LocalizedResource lr = new LocalizedResource(req, dispatcher); + return lr; + } + + private DrainDispatcher createDispatcher(Configuration conf) { + DrainDispatcher dispatcher = new DrainDispatcher(); + dispatcher.init(conf); + dispatcher.start(); + return dispatcher; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java index bc420e4f92..a4202a9462 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java @@ -63,13 +63,15 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat; +import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogKey; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogReader; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerContainerFinishedEvent; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/TestNonAggregatingLogHandler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/TestNonAggregatingLogHandler.java index 6eacb8aa72..5fa7bcb3b1 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/TestNonAggregatingLogHandler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/TestNonAggregatingLogHandler.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. + */ package org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler; import static org.mockito.Matchers.any; @@ -18,10 +35,10 @@ import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.ContainerLogsRetentionPolicy; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerContainerFinishedEvent; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestProcessIdFileReader.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestProcessIdFileReader.java index d9f60e6053..a8e3e8a989 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestProcessIdFileReader.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestProcessIdFileReader.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. + */ package org.apache.hadoop.yarn.server.nodemanager.util; import static org.junit.Assert.*; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index e5a90cab55..4c4334c4dc 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -278,7 +278,8 @@ protected synchronized void submitApplication( // Setup tokens for renewal if (UserGroupInformation.isSecurityEnabled()) { this.rmContext.getDelegationTokenRenewer().addApplication( - applicationId,parseCredentials(submissionContext) + applicationId,parseCredentials(submissionContext), + submissionContext.getCancelTokensWhenComplete() ); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java index 0d81f80121..71dd982b60 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java @@ -595,8 +595,13 @@ public void transition(RMAppAttemptImpl appAttempt, AM_CONTAINER_PRIORITY, "*", appAttempt.submissionContext .getAMContainerSpec().getResource(), 1); - appAttempt.scheduler.allocate(appAttempt.applicationAttemptId, - Collections.singletonList(request), EMPTY_CONTAINER_RELEASE_LIST); + Allocation amContainerAllocation = + appAttempt.scheduler.allocate(appAttempt.applicationAttemptId, + Collections.singletonList(request), EMPTY_CONTAINER_RELEASE_LIST); + if (amContainerAllocation != null + && amContainerAllocation.getContainers() != null) { + assert(amContainerAllocation.getContainers().size() == 0); + } } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index cf19b74829..40fbefe73f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -245,16 +245,45 @@ private synchronized void setupQueueConfigs( aclsString.append(e.getKey() + ":" + e.getValue().getAclString()); } - LOG.info("Initializing " + queueName + - ", capacity=" + capacity + - ", asboluteCapacity=" + absoluteCapacity + - ", maxCapacity=" + maxCapacity + - ", asboluteMaxCapacity=" + absoluteMaxCapacity + - ", userLimit=" + userLimit + ", userLimitFactor=" + userLimitFactor + - ", maxApplications=" + maxApplications + - ", maxApplicationsPerUser=" + maxApplicationsPerUser + - ", state=" + state + - ", acls=" + aclsString); + LOG.info("Initializing " + queueName + "\n" + + "capacity = " + capacity + + " [= (float) configuredCapacity / 100 ]" + "\n" + + "asboluteCapacity = " + absoluteCapacity + + " [= parentAbsoluteCapacity * capacity ]" + "\n" + + "maxCapacity = " + maxCapacity + + " [= configuredMaxCapacity ]" + "\n" + + "absoluteMaxCapacity = " + absoluteMaxCapacity + + " [= Float.MAX_VALUE if maximumCapacity undefined, " + + "(parentAbsoluteCapacity * maximumCapacity) / 100 otherwise ]" + "\n" + + "userLimit = " + userLimit + + " [= configuredUserLimit ]" + "\n" + + "userLimitFactor = " + userLimitFactor + + " [= configuredUserLimitFactor ]" + "\n" + + "maxApplications = " + maxApplications + + " [= (int)(configuredMaximumSystemApplications * absoluteCapacity) ]" + "\n" + + "maxApplicationsPerUser = " + maxApplicationsPerUser + + " [= (int)(maxApplications * (userLimit / 100.0f) * userLimitFactor) ]" + "\n" + + "maxActiveApplications = " + maxActiveApplications + + " [= max(" + + "(int)((clusterResourceMemory / (float)DEFAULT_AM_RESOURCE) *" + + "maxAMResourcePercent * absoluteCapacity)," + + "1) ]" + "\n" + + "maxActiveApplicationsPerUser = " + maxActiveApplicationsPerUser + + " [= (int)(maxActiveApplications * (userLimit / 100.0f) * userLimitFactor) ]" + "\n" + + "utilization = " + utilization + + " [= usedResourcesMemory / queueLimit ]" + "\n" + + "usedCapacity = " + usedCapacity + + " [= usedResourcesMemory / (clusterResourceMemory * capacity) ]" + "\n" + + "maxAMResourcePercent = " + maxAMResourcePercent + + " [= configuredMaximumAMResourcePercent ]" + "\n" + + "minimumAllocationFactor = " + minimumAllocationFactor + + " [= (float)(maximumAllocationMemory - minimumAllocationMemory) / maximumAllocationMemory ]" + "\n" + + "numContainers = " + numContainers + + " [= currentNumContainers ]" + "\n" + + "state = " + state + + " [= configuredState ]" + "\n" + + "acls = " + aclsString + + " [= configuredAcls ]" + "\n"); } @Override diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index c61c7ab89f..977150520a 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -236,28 +236,30 @@ public Allocation allocate( RMContainerEventType.RELEASED); } - if (!ask.isEmpty()) { - LOG.debug("allocate: pre-update" + - " applicationId=" + applicationAttemptId + - " application=" + application); - application.showRequests(); + synchronized (application) { + if (!ask.isEmpty()) { + LOG.debug("allocate: pre-update" + + " applicationId=" + applicationAttemptId + + " application=" + application); + application.showRequests(); - // Update application requests - application.updateResourceRequests(ask); + // Update application requests + application.updateResourceRequests(ask); - LOG.debug("allocate: post-update" + - " applicationId=" + applicationAttemptId + - " application=" + application); - application.showRequests(); + LOG.debug("allocate: post-update" + + " applicationId=" + applicationAttemptId + + " application=" + application); + application.showRequests(); - LOG.debug("allocate:" + - " applicationId=" + applicationAttemptId + - " #ask=" + ask.size()); + LOG.debug("allocate:" + + " applicationId=" + applicationAttemptId + + " #ask=" + ask.size()); + } + + return new Allocation( + application.pullNewlyAllocatedContainers(), + application.getHeadroom()); } - - return new Allocation( - application.pullNewlyAllocatedContainers(), - application.getHeadroom()); } private SchedulerApp getApplication( diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java index b3ab9a1c4e..a7d2e4582d 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java @@ -108,15 +108,17 @@ private static class DelegationTokenToRenew { public final Configuration conf; public long expirationDate; public TimerTask timerTask; + public final boolean shouldCancelAtEnd; public DelegationTokenToRenew( ApplicationId jId, Token token, - Configuration conf, long expirationDate) { + Configuration conf, long expirationDate, boolean shouldCancelAtEnd) { this.token = token; this.applicationId = jId; this.conf = conf; this.expirationDate = expirationDate; this.timerTask = null; + this.shouldCancelAtEnd = shouldCancelAtEnd; if (this.token==null || this.applicationId==null || this.conf==null) { throw new IllegalArgumentException("Invalid params to renew token" + ";token=" + this.token + @@ -218,10 +220,12 @@ private void addTokenToList(DelegationTokenToRenew t) { * Add application tokens for renewal. * @param applicationId added application * @param ts tokens + * @param shouldCancelAtEnd true if tokens should be canceled when the app is + * done else false. * @throws IOException */ public synchronized void addApplication( - ApplicationId applicationId, Credentials ts) + ApplicationId applicationId, Credentials ts, boolean shouldCancelAtEnd) throws IOException { if (ts == null) { return; //nothing to add @@ -239,7 +243,8 @@ public synchronized void addApplication( // first renew happens immediately if (token.isManaged()) { DelegationTokenToRenew dtr = - new DelegationTokenToRenew(applicationId, token, getConfig(), now); + new DelegationTokenToRenew(applicationId, token, getConfig(), now, + shouldCancelAtEnd); addTokenToList(dtr); @@ -317,7 +322,11 @@ void setTimerForTokenRenewal(DelegationTokenToRenew token, // cancel a token private void cancelToken(DelegationTokenToRenew t) { - dtCancelThread.cancelToken(t.token, t.conf); + if(t.shouldCancelAtEnd) { + dtCancelThread.cancelToken(t.token, t.conf); + } else { + LOG.info("Did not cancel "+t); + } } /** diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java index 1a43c15519..32d8d97215 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.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. + */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import static org.junit.Assert.*; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java index 61870a817d..9448fe0ed4 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.IOException; import java.net.URI; @@ -243,16 +244,16 @@ static MyToken createTokens(Text renewer) /** * Basic idea of the test: * 1. create tokens. - * 2. Mark one of them to be renewed in 2 seconds (istead of - * 24 hourse) + * 2. Mark one of them to be renewed in 2 seconds (instead of + * 24 hours) * 3. register them for renewal * 4. sleep for 3 seconds * 5. count number of renewals (should 3 initial ones + one extra) * 6. register another token for 2 seconds * 7. cancel it immediately * 8. Sleep and check that the 2 seconds renew didn't happen - * (totally 5 reneals) - * 9. check cancelation + * (totally 5 renewals) + * 9. check cancellation * @throws IOException * @throws URISyntaxException */ @@ -287,7 +288,7 @@ public void testDTRenewal () throws Exception { // register the tokens for renewal ApplicationId applicationId_0 = BuilderUtils.newApplicationId(0, 0); - delegationTokenRenewer.addApplication(applicationId_0, ts); + delegationTokenRenewer.addApplication(applicationId_0, ts, true); // first 3 initial renewals + 1 real int numberOfExpectedRenewals = 3+1; @@ -326,7 +327,7 @@ public void testDTRenewal () throws Exception { ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1); - delegationTokenRenewer.addApplication(applicationId_1, ts); + delegationTokenRenewer.addApplication(applicationId_1, ts, true); delegationTokenRenewer.removeApplication(applicationId_1); numberOfExpectedRenewals = Renewer.counter; // number of renewals so far @@ -347,4 +348,49 @@ public void testDTRenewal () throws Exception { //expected } } + + /** + * Basic idea of the test: + * 1. register a token for 2 seconds with no cancel at the end + * 2. cancel it immediately + * 3. Sleep and check that the 2 seconds renew didn't happen + * (totally 5 renewals) + * 4. check cancellation + * @throws IOException + * @throws URISyntaxException + */ + @Test + public void testDTRenewalWithNoCancel () throws Exception { + MyFS dfs = (MyFS)FileSystem.get(conf); + LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode()); + + Credentials ts = new Credentials(); + MyToken token1 = dfs.getDelegationToken(new Text("user1")); + + //to cause this one to be set for renew in 2 secs + Renewer.tokenToRenewIn2Sec = token1; + LOG.info("token="+token1+" should be renewed for 2 secs"); + + String nn1 = DelegationTokenRenewer.SCHEME + "://host1:0"; + ts.addToken(new Text(nn1), token1); + + + ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1); + delegationTokenRenewer.addApplication(applicationId_1, ts, false); + delegationTokenRenewer.removeApplication(applicationId_1); + + int numberOfExpectedRenewals = Renewer.counter; // number of renewals so far + try { + Thread.sleep(6*1000); // sleep 6 seconds, so it has time to renew + } catch (InterruptedException e) {} + LOG.info("Counter = " + Renewer.counter + ";t="+ Renewer.lastRenewed); + + // counter and the token should still be the old ones + assertEquals("renew wasn't called as many times as expected", + numberOfExpectedRenewals, Renewer.counter); + + // also renewing of the canceled token should not fail, because it has not + // been canceled + token1.renew(conf); + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm index daf6a12c3a..88fb9a6c88 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm @@ -50,7 +50,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler that the available resources in the Hadoop cluster are shared among multiple organizations who collectively fund the cluster based on their computing needs. There is an added benefit that an organization can access - any excess capacity no being used by others. This provides elasticity for + any excess capacity not being used by others. This provides elasticity for the organizations in a cost-effective manner. Sharing clusters across organizations necessitates strong support for @@ -58,7 +58,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler safe-guards to ensure the shared cluster is impervious to single rouge application or user or sets thereof. The <<>> provides a stringent set of limits to ensure that a single application or user or queue - cannot consume dispropotionate amount of resources in the cluster. Also, the + cannot consume disproportionate amount of resources in the cluster. Also, the <<>> provides limits on initialized/pending applications from a single user and queue to ensure fairness and stability of the cluster. @@ -67,7 +67,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler economics of the shared cluster. To provide further control and predictability on sharing of resources, the - <<>> supports to ensure + <<>> supports to ensure resources are shared among the sub-queues of an organization before other queues are allowed to use free resources, there-by providing for sharing free resources among applications of a given organization. @@ -76,7 +76,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler The <<>> supports the following features: - * Heirarchical Queues - Heirarchy of queues is supported to ensure resources + * Hierarchical Queues - Hierarchy of queues is supported to ensure resources are shared among the sub-queues of an organization before other queues are allowed to use free resources, there-by providing more control and predictability. @@ -96,12 +96,12 @@ Hadoop MapReduce Next Generation - Capacity Scheduler capacity. When there is demand for these resources from queues running below capacity at a future point in time, as tasks scheduled on these resources complete, they will be assigned to applications on queues running below the - capacity. This ensures that resources are available in a predictable and - elastic manner to queues, thus preventing artifical silos of resources in - the cluster which helps utilization. + capacity (pre-emption is not supported). This ensures that resources are available + in a predictable and elastic manner to queues, thus preventing artifical silos + of resources in the cluster which helps utilization. * Multi-tenancy - Comprehensive set of limits are provided to prevent a - single application, user and queue from monpolizing resources of the queue + single application, user and queue from monopolizing resources of the queue or the cluster as a whole to ensure that the cluster isn't overwhelmed. * Operability @@ -110,8 +110,8 @@ Hadoop MapReduce Next Generation - Capacity Scheduler capacity, ACLs can be changed, at runtime, by administrators in a secure manner to minimize disruption to users. Also, a console is provided for users and administrators to view current allocation of resources to - various queues in the system. Administrators can also - at runtime. + various queues in the system. Administrators can + at runtime, but queues cannot be at runtime. * Drain applications - Administrators can queues at runtime to ensure that while existing applications run to completion, @@ -139,7 +139,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler || Property || Value | *--------------------------------------+--------------------------------------+ | <<>> | | -| | <<>> | +| | <<>> | *--------------------------------------+--------------------------------------+ * Setting up @@ -155,13 +155,14 @@ Hadoop MapReduce Next Generation - Capacity Scheduler child queues. The configuration for <<>> uses a concept called - to configure the heirarchy of queues. The is the - full path of the queue's heirarcy, starting at , with . (dot) as the + to configure the hierarchy of queues. The is the + full path of the queue's hierarchy, starting at , with . (dot) as the delimiter. A given queue's children can be defined with the configuration knob: - <<.queues>>> - + <<.queues>>>. Children do not + inherit properties directly from the parent. + Here is an example with three top-level child-queues <<>>, <<>> and <<>> and some sub-queues for <<>> and <<>>: @@ -197,52 +198,59 @@ Hadoop MapReduce Next Generation - Capacity Scheduler *--------------------------------------+--------------------------------------+ | <<.capacity>>> | | | | Queue in percentage (%). | -| | The sum of capacities for all queues, at each level, should be less than | -| | or equal to 100. | +| | The sum of capacities for all queues, at each level, must be equal | +| | to 100. | | | Applications in the queue may consume more resources than the queue's | | | capacity if there are free resources, providing elasticity. | *--------------------------------------+--------------------------------------+ | <<.maximum-capacity>>> | | | | Maximum queue capacity in percentage (%). | | | This limits the for applications in the queue. | +| | Defaults to -1 which disables it. | *--------------------------------------+--------------------------------------+ | <<.minimum-user-limit-percent>>> | | | | Each queue enforces a limit on the percentage of resources allocated to a | | | user at any given time, if there is demand for resources. The user limit | -| | can vary between a minimum and maximum value. The former depends on the | -| | number of users who have submitted applications, and the latter is set to | -| | this property value. For e.g., suppose the value of this property is 25. | +| | can vary between a minimum and maximum value. The the former | +| | (the minimum value) is set to this property value and the latter | +| | (the maximum value) depends on the number of users who have submitted | +| | applications. For e.g., suppose the value of this property is 25. | | | If two users have submitted applications to a queue, no single user can | | | use more than 50% of the queue resources. If a third user submits an | | | application, no single user can use more than 33% of the queue resources. | | | With 4 or more users, no user can use more than 25% of the queues | -| | resources. A value of 100 implies no user limits are imposed. | +| | resources. A value of 100 implies no user limits are imposed. The default | +| | is 100.| *--------------------------------------+--------------------------------------+ | <<.user-limit-factor>>> | | | | The multiple of the queue capacity which can be configured to allow a | | | single user to acquire more resources. By default this is set to 1 which | | | ensures that a single user can never take more than the queue's configured | -| | capacity irrespective of how idle th cluster is. | +| | capacity irrespective of how idle th cluster is. Value is specified as | +| | a float.| *--------------------------------------+--------------------------------------+ * Running and Pending Application Limits The <<>> supports the following parameters to control - the runnign and pending applications: + the running and pending applications: *--------------------------------------+--------------------------------------+ || Property || Description | *--------------------------------------+--------------------------------------+ | <<>> | | -| | Maximum number of jobs in the system which can be concurently active | -| | both running and pending. Limits on each queue are directly proportional | -| | to their queue capacities. | +| | Maximum number of applications in the system which can be concurrently | +| | active both running and pending. Limits on each queue are directly | +| | proportional to their queue capacities and user limits. This is a +| | hard limit and any applications submitted when this limit is reached will | +| | be rejected. Default is 10000.| *--------------------------------------+--------------------------------------+ | yarn.scheduler.capacity.maximum-am-resource-percent | | | | Maximum percent of resources in the cluster which can be used to run | | | application masters - controls number of concurrent running applications. | +| | Specified as a float - ie 0.5 = 50%. Default is 10%. | *--------------------------------------+--------------------------------------+ * Queue Administration & Permissions @@ -257,7 +265,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler | <<.state>>> | | | | The of the queue. Can be one of <<>> or <<>>. | | | If a queue is in <<>> state, new applications cannot be | -| | submitted to or . | +| | submitted to or . | | | Thus, if the queue is <<>> no applications can be | | | submitted to the entire cluster. | | | Existing applications continue to completion, thus the queue can be @@ -276,7 +284,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler An is of the form , , . The special value of <<*>> implies . The special value of - implies . + implies . The default is <<*>> if not specified. * Reviewing the configuration of the CapacityScheduler @@ -295,14 +303,14 @@ Hadoop MapReduce Next Generation - Capacity Scheduler * {Changing Queue Configuration} Changing queue properties and adding new queues is very simple. You need to - edit <> and run . + edit <> and run . ---- $ vi $HADOOP_CONF_DIR/capacity-scheduler.xml -$ $YARN_HOME/bin/rmadmin -refreshQueues +$ $YARN_HOME/bin/yarn rmadmin -refreshQueues ---- Queues cannot be , only addition of new queues is supported - the updated queue configuration should be a valid one i.e. queue-capacity at each should be equal to 100%. - \ No newline at end of file + diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SingleCluster.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SingleCluster.apt.vm index d3af411784..966f155b3a 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SingleCluster.apt.vm +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SingleCluster.apt.vm @@ -41,7 +41,7 @@ $ mvn clean install assembly:assembly -Pnative * Setting up the environment. Assuming you have installed hadoop-common/hadoop-hdfs and exported - <<$HADOOP_COMMON_HOME>>/<<$HADOOP_COMMON_HOME>>, untar hadoop mapreduce + <<$HADOOP_COMMON_HOME>>/<<$HADOOP_HDFS_HOME>>, untar hadoop mapreduce tarball and set environment variable <<$HADOOP_MAPRED_HOME>> to the untarred directory. Set <<$YARN_HOME>> the same as <<$HADOOP_MAPRED_HOME>>. @@ -79,15 +79,15 @@ $ mvn clean install assembly:assembly -Pnative Add the following configs to your <<>> +---+ - + yarn.resourcemanager.resource-tracker.address host:port host is the hostname of the resource manager and port is the port on which the NodeManagers contact the Resource Manager. - + - + yarn.resourcemanager.scheduler.address host:port host is the hostname of the resourcemanager and port is the port @@ -145,6 +145,32 @@ Add the following configs to your <<>> +---+ +** Setting up <<>> + + Make sure you populate the root queues in <<>>. + ++---+ + + yarn.scheduler.capacity.root.queues + unfunded,default + + + + yarn.scheduler.capacity.root.capacity + 100 + + + + yarn.scheduler.capacity.root.unfunded.capacity + 50 + + + + yarn.scheduler.capacity.root.default.capacity + 50 + ++---+ + * Create Symlinks. You will have to create the following symlinks: diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java index fcc6939c4e..b4a0e0b5e2 100644 --- a/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java @@ -123,7 +123,7 @@ public class Gridmix extends Configured implements Tool { summarizer = new Summarizer(args); } - Gridmix() { + public Gridmix() { summarizer = new Summarizer(); } diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithCustomInterval.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithCustomInterval.java new file mode 100644 index 0000000000..dd8e51edfd --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithCustomInterval.java @@ -0,0 +1,108 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test cpu emulation with default interval for gridmix jobs + * against different input data, submission policies and user resolvers. + * Verify the cpu resource metrics of both maps and reduces phase of + * Gridmix jobs with their corresponding original job in the input trace. + */ +public class TestCPUEmulationForMapsAndReducesWithCustomInterval + extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog("TestCPUEmulationWithUncompressedInput.class"); + int execMode = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue(); + + /** + * Generate compressed input and run {@link Gridmix} by turning on the + * cpu emulation feature with default setting. The {@link Gridmix} + * should use the following runtime parameters. + * Submission Policy : STRESS, UserResovler: RoundRobinUserResolver. + * Once the {@link Gridmix} run is complete, verify cpu resource metrics of + * {@link Gridmix} jobs with their corresponding original job in a trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testCPUEmulationForMapsAndReducesWithCompressedInputCase7() + throws Exception { + final long inputSizeInMB = 1024 * 7; + String tracePath = getTraceFile("cpu_emul_case2"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + RoundRobinUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false", + "-D", GridMixConfig.GRIDMIX_CPU_CUSTOM_INTERVAL + "=0.35F", + "-D", GridMixConfig.GRIDMIX_CPU_EMULATON + "=" + + GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on the + * cpu emulation feature with default setting. The {@link Gridmix} + * should use the following runtime parameters. + * Submission Policy : SERIAL, UserResovler: SubmitterUserResolver + * Once the {@link Gridmix} run is complete, verify cpu resource metrics of + * {@link Gridmix} jobs with their corresponding original job in a trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testCPUEmulatonForMapsAndReducesWithUncompressedInputCase8() + throws Exception { + final long inputSizeInMB = cSize * 300; + String tracePath = getTraceFile("cpu_emul_case2"); + Assert.assertNotNull("Trace file not found.", tracePath); + String [] runtimeValues = + { "LOADJOB", + SubmitterUserResolver.class.getName(), + "SERIAL", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false", + "-D", GridMixConfig.GRIDMIX_CPU_CUSTOM_INTERVAL + "=0.4F", + "-D", GridMixConfig.GRIDMIX_CPU_EMULATON + "=" + + GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN }; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); + } +} + + diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithDefaultInterval.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithDefaultInterval.java new file mode 100644 index 0000000000..edd14a6bcb --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithDefaultInterval.java @@ -0,0 +1,105 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.apache.hadoop.mapreduce.JobContext; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test cpu emulation with default interval for gridmix jobs + * against different input data, submission policies and user resolvers. + * Verify the cpu resource metrics for both maps and reduces of + * Gridmix jobs with their corresponding original job in the input trace. + */ +public class TestCPUEmulationForMapsAndReducesWithDefaultInterval + extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog( + "TestCPUEmulationForMapsAndReducesWithDefaultInterval.class"); + int execMode = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue(); + + /** + * Generate compressed input and run {@link Gridmix} by turning on the + * cpu emulation feature with default setting. The {@link Gridmix} + * should use the following runtime parameters. + * Submission Policy : REPLAY, UserResovler: RoundRobinUserResolver. + * Once the {@link Gridmix} run is complete, verify cpu resource metrics of + * {@link Gridmix} jobs with their corresponding original jobs in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testCPUEmulationForMapsAndReducesWithCompressedInputCase5() + throws Exception { + final long inputSizeInMB = 7168; + String tracePath = getTraceFile("cpu_emul_case2"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + RoundRobinUserResolver.class.getName(), + "REPLAY", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false", + "-D", GridMixConfig.GRIDMIX_CPU_EMULATON + "=" + + GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on the + * cpu emulation feature with default settings. The {@link Gridmix} + * should use the following runtime parameters. + * Submission Policy : STRESS, UserResovler: SubmitterUserResolver + * Once the Gridmix run is complete, verify cpu resource metrics of + * {@link Gridmix} jobs with their corresponding original jobs in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testCPUEmulatonForMapsAndReducesWithUncompressedInputCase6() + throws Exception { + final long inputSizeInMB = cSize * 400; + String tracePath = getTraceFile("cpu_emul_case2"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + SubmitterUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false", + "-D", GridMixConfig.GRIDMIX_CPU_EMULATON + "=" + + GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN }; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); + } +} diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithCustomInterval.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithCustomInterval.java new file mode 100644 index 0000000000..ac5205818c --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithCustomInterval.java @@ -0,0 +1,105 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test the {@link Gridmix} cpu emulation with custom interval for + * gridmix jobs against different input data, submission policies and + * user resolvers. Verify the map phase cpu metrics of gridmix jobs + * against their original job in the trace. + */ +public class TestCPUEmulationForMapsWithCustomInterval + extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog("TestCPUEmulationForMapsWithCustomInterval.class"); + int execMode = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue(); + + /** + * Generate compressed input and run {@link Gridmix} by turning on + * cpu emulation feature with custom setting. The {@link Gridmix} should + * use the following runtime parameters while running gridmix jobs. + * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver + * Once {@link Gridmix} run is complete, verify maps phase cpu resource + * metrics of {@link Gridmix} jobs with their corresponding original + * in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testCPUEmulatonForMapsWithCompressedInputCase3() + throws Exception { + final long inputSizeInMB = 1024 * 7; + String tracePath = getTraceFile("cpu_emul_case1"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = {"LOADJOB", + SubmitterUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_CPU_EMULATON + "=" + + GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN, + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_CPU_CUSTOM_INTERVAL + "=0.25F"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on + * cpu emulation feature with custom settings. The {@link Gridmix} + * should use the following runtime paramters while running gridmix jobs. + * Submission Policy: REPLAY User Resolver Mode: RoundRobinUserResolver + * Once {@link Gridmix} run is complete, verify the map phase cpu resource + * metrics of {@link Gridmix} jobs with their corresponding jobs + * in the original trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testCPUEmulatonForMapsUnCompressedInputCase4() + throws Exception { + final long inputSizeInMB = cSize * 200; + String tracePath = getTraceFile("cpu_emul_case1"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + {"LOADJOB", + RoundRobinUserResolver.class.getName(), + "REPLAY", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_CPU_EMULATON + "=" + + GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN, + "-D", GridMixConfig.GRIDMIX_CPU_CUSTOM_INTERVAL + "=0.35F"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); + } +} + diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithDefaultInterval.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithDefaultInterval.java new file mode 100644 index 0000000000..6eabc53838 --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithDefaultInterval.java @@ -0,0 +1,103 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test the {@link Gridmix} cpu emulation with default settings for + * gridmix jobs against different input data, submission policies and + * user resolvers. Verify the map phase cpu metrics of gridmix jobs + * against their original jobs in the trace. + */ +public class TestCPUEmulationForMapsWithDefaultInterval + extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog("TestCPUEmulationForMapsWithDefaultInterval.class"); + int execMode = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue(); + + /** + * Generate compressed input and run {@link Gridmix} by turning on cpu + * emulation feature with default settings. The {@link Gridmix} should + * use the following runtime parameters while running the gridmix jobs. + * Submission Policy: STRESS, UserResolver: SubmitterUserResolver. + * Once the {@link Gridmix} run is complete, verify map phase cpu metrics of + * {@link Gridmix} jobs with their corresponding original job in a trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testCPUEmulatonForMapsWithCompressedInputCase1() + throws Exception { + final long inputSizeInMB = 1024 * 6; + String tracePath = getTraceFile("cpu_emul_case1"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = { "LOADJOB", + SubmitterUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_CPU_EMULATON + "=" + + GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN, + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_HIGH_RAM_JOB_ENABLE + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on + * cpu emulation feature with default settings. The {@link Gridmix} + * should use the following runtime parameters while running Gridmix jobs. + * Submission Policy: REPLAY, UserResolver: RoundRobinUserResolver + * Once the Gridmix run is complete, verify cpu resource metrics of + * {@link Gridmix} jobs with their corresponding original job in a trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testCPUEmulatonForMapsWithUnCompressedInputCase2() + throws Exception { + final long inputSizeInMB = cSize * 200; + String tracePath = getTraceFile("cpu_emul_case1"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + RoundRobinUserResolver.class.getName(), + "REPLAY", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_HIGH_RAM_JOB_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_CPU_EMULATON + "=" + + GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } +} diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithCustomIntrvl.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithCustomIntrvl.java new file mode 100644 index 0000000000..a82e806059 --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithCustomIntrvl.java @@ -0,0 +1,106 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test the {@link Gridmix} memory emulation feature for the jobs with + * custom progress interval, different input data, submission policies + * and user resolver modes. Verify the total heap usage of map and reduce + * tasks of the jobs with corresponding original job in the trace. + */ +public class TestMemEmulForMapsAndReducesWithCustomIntrvl + extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog("TestMemEmulForMapsAndReducesWithCustomIntrvl.class"); + /** + * Generate compressed input and run {@link Gridmix} by turning on the + * memory emulation with custom progress interval. The {@link Gridmix} + * should use the following runtime parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver + * Verify maps and reduces total heap memory usage of {@link Gridmix} jobs + * with corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForReducesWithCompressedInputCase7() + throws Exception { + final long inputSizeInMB = 1024 * 7; + String tracePath = getTraceFile("mem_emul_case2"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + RoundRobinUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL + "=0.3F", + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on the + * memory emulation with custom progress interval. The {@link Gridmix} + * should use the following runtime parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver + * Verify maps and reduces total heap memory usage of {@link Gridmix} jobs + * with corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForReducesWithUncompressedInputCase8() + throws Exception { + final long inputSizeInMB = cSize * 300; + String tracePath = getTraceFile("mem_emul_case2"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + SubmitterUserResolver.class.getName(), + "REPLAY", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL + "=0.2F", + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } +} diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithDefaultIntrvl.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithDefaultIntrvl.java new file mode 100644 index 0000000000..e1f211a11f --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithDefaultIntrvl.java @@ -0,0 +1,106 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test the {@link Gridmix} memory emulation feature for gridmix jobs + * with default progress interval, different input data, submission + * policies and user resolver modes. Verify the total heap usage of + * map and reduce tasks of the jobs with corresponding original + * job in the trace. + */ +public class TestMemEmulForMapsAndReducesWithDefaultIntrvl + extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog("TestMemEmulForMapsAndReducesWithDefaultIntrvl.class"); + + /** + * Generate compressed input and run {@link Gridmix} by turning on the + * memory emulation with default progress interval. The {@link Gridmix} + * should use the following runtime parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver + * Verify maps and reduces total heap memory usage of {@link Gridmix} jobs + * with corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForReducesWithCompressedInputCase5() + throws Exception { + final long inputSizeInMB = 1024 * 7; + String tracePath = getTraceFile("mem_emul_case2"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + RoundRobinUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on the + * memory emulation with default progress interval. The {@link Gridmix} + * should use the following runtime parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver + * Verify maps and reduces total heap memory usage of {@link Gridmix} jobs + * with corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForReducesWithUncompressedInputCase6() + throws Exception { + final long inputSizeInMB = cSize * 300; + String tracePath = getTraceFile("mem_emul_case2"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + SubmitterUserResolver.class.getName(), + "REPLAY", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } +} diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomHeapMemoryRatio.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomHeapMemoryRatio.java new file mode 100644 index 0000000000..da48ad4538 --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomHeapMemoryRatio.java @@ -0,0 +1,108 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test the {@link Gridmix} memory emulation feature for {@link Gridmix} jobs + * with default progress interval, custom heap memory ratio, different input + * data, submission policies and user resolver modes. Verify the total heap + * usage of map and reduce tasks of the jobs with corresponding the original job + * in the trace. + */ +public class TestMemEmulForMapsWithCustomHeapMemoryRatio + extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog("TestMemEmulForMapsWithCustomHeapMemoryRatio.class"); + + /** + * Generate compressed input and run {@link Gridmix} by turning on the + * memory emulation. The {@link Gridmix} should use the following runtime + * parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver + * Verify total heap memory usage of the tasks of {@link Gridmix} jobs with + * corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForMapsWithCompressedInputCase1() + throws Exception { + final long inputSizeInMB = 1024 * 7; + String tracePath = getTraceFile("mem_emul_case2"); + Assert.assertNotNull("Trace file has not found.", tracePath); + String [] runtimeValues = + { "LOADJOB", + SubmitterUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false", + "-D", GridMixConfig.GRIDMIX_HEAP_FREE_MEMORY_RATIO + "=0.5F"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on the + * memory emulation. The {@link Gridmix} should use the following runtime + * parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : RoundRobinUserResolver + * Verify total heap memory usage of tasks of {@link Gridmix} jobs with + * corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForMapsWithUncompressedInputCase2() + throws Exception { + final long inputSizeInMB = cSize * 300; + String tracePath = getTraceFile("mem_emul_case2"); + Assert.assertNotNull("Trace file has not found.", tracePath); + String [] runtimeValues = + { "LOADJOB", + RoundRobinUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false", + "-D", GridMixConfig.GRIDMIX_HEAP_FREE_MEMORY_RATIO + "=0.4F"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } +} diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomIntrvl.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomIntrvl.java new file mode 100644 index 0000000000..5d1d452b8d --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomIntrvl.java @@ -0,0 +1,106 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test the {@link Gridmix} memory emulation feature for {@link Gridmix} jobs + * with custom progress interval, different input data, submission policies + * and user resolver modes. Verify the total heap usage of map tasks of + * the jobs with corresponding the original job in the trace. + */ +public class TestMemEmulForMapsWithCustomIntrvl extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog("TestMemEmulForMapsWithCustomIntrvl.class"); + + /** + * Generate compressed input and run {@link Gridmix} by turning on the + * memory emulation with custom progress interval. The {@link Gridmix} + * should use the following runtime parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver + * Verify maps total heap memory usage of {@link Gridmix} jobs with + * corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForMapsWithCompressedInputCase3() + throws Exception { + final long inputSizeInMB = 1024 * 7; + String tracePath = getTraceFile("mem_emul_case1"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + SubmitterUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL + "=0.2F", + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on the + * memory emulation with custom progress interval. The {@link Gridmix} + * should use the following runtime parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : RoundRobinUserResolver + * Verify maps total heap memory usage of {@link Gridmix} jobs with + * corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForMapsWithUncompressedInputCase4() + throws Exception { + final long inputSizeInMB = cSize * 300; + String tracePath = getTraceFile("mem_emul_case1"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + RoundRobinUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL + "=0.3F", + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } +} diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithDefaultIntrvl.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithDefaultIntrvl.java new file mode 100644 index 0000000000..ff136b89c7 --- /dev/null +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithDefaultIntrvl.java @@ -0,0 +1,104 @@ +/** + * 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.gridmix; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig; +import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode; +import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.junit.Test; +import org.junit.Assert; + +/** + * Test the {@link Gridmix} memory emulation feature for {@link Gridmix} jobs + * with default progress interval, different input data, submission policies + * and user resolver modes. Verify the total heap usage of map tasks of the + * jobs with corresponding original job in the trace. + */ +public class TestMemEmulForMapsWithDefaultIntrvl extends GridmixSystemTestCase { + private static final Log LOG = + LogFactory.getLog("TestMemEmulForMapsWithDefaultIntrvl.class"); + + /** + * Generate compressed input and run {@link Gridmix} by turning on the + * memory emulation with default progress interval. The {@link Gridmix} + * should use the following runtime parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver + * Verify maps total heap memory usage of {@link Gridmix} jobs with + * corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForMapsWithCompressedInputCase1() + throws Exception { + final long inputSizeInMB = 1024 * 7; + String tracePath = getTraceFile("mem_emul_case1"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + SubmitterUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } + + /** + * Generate uncompressed input and run {@link Gridmix} by turning on the + * memory emulation with default progress interval. The {@link Gridmix} + * should use the following runtime parameters while running the jobs. + * Submission Policy : STRESS, User Resolver Mode : RoundRobinUserResolver + * Verify maps total heap memory usage of {@link Gridmix} jobs with + * corresponding original job in the trace. + * @throws Exception - if an error occurs. + */ + @Test + public void testMemoryEmulationForMapsWithUncompressedInputCase2() + throws Exception { + final long inputSizeInMB = cSize * 300; + String tracePath = getTraceFile("mem_emul_case1"); + Assert.assertNotNull("Trace file not found!", tracePath); + String [] runtimeValues = + { "LOADJOB", + RoundRobinUserResolver.class.getName(), + "STRESS", + inputSizeInMB + "m", + "file://" + UtilsForGridmix.getProxyUsersFile(conf), + tracePath}; + + String [] otherArgs = { + "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATON + "=" + + GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN, + "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", + "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", + "-D", MRJobConfig.JOB_CANCEL_DELEGATION_TOKEN + "=false"}; + + runGridmixAndVerify(runtimeValues, otherArgs, tracePath, + GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()); + } +} diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridMixConfig.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridMixConfig.java index 13cc67c18f..fc99162bd8 100644 --- a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridMixConfig.java +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridMixConfig.java @@ -23,6 +23,7 @@ import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; +import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.*; /** * Gridmix system tests configurations. @@ -218,4 +219,67 @@ public class GridMixConfig { */ public static final String CLUSTER_MAX_REDUCE_MEMORY = JTConfig.JT_MAX_REDUCEMEMORY_MB; + + /** + * Gridmix cpu emulation. + */ + public static final String GRIDMIX_CPU_EMULATON = + ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS; + + /** + * Gridmix cpu usage emulation plugin. + */ + public static final String GRIDMIX_CPU_USAGE_PLUGIN = + CumulativeCpuUsageEmulatorPlugin.class.getName(); + + /** + * Gridmix cpu emulation custom interval. + */ + public static final String GRIDMIX_CPU_CUSTOM_INTERVAL = + CumulativeCpuUsageEmulatorPlugin.CPU_EMULATION_PROGRESS_INTERVAL; + + /** + * Gridmix cpu emulation lower limit. + */ + public static int GRIDMIX_CPU_EMULATION_LOWER_LIMIT = 55; + + /** + * Gridmix cpu emulation upper limit. + */ + public static int GRIDMIX_CPU_EMULATION_UPPER_LIMIT = 130; + + /** + * Gridmix heap memory custom interval + */ + public static final String GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL = + TotalHeapUsageEmulatorPlugin.HEAP_EMULATION_PROGRESS_INTERVAL; + + /** + * Gridmix heap free memory ratio + */ + public static final String GRIDMIX_HEAP_FREE_MEMORY_RATIO = + TotalHeapUsageEmulatorPlugin.MIN_HEAP_FREE_RATIO; + + /** + * Gridmix memory emulation plugin + */ + public static final String GRIDMIX_MEMORY_EMULATION_PLUGIN = + TotalHeapUsageEmulatorPlugin.class.getName(); + + /** + * Gridmix memory emulation + */ + public static final String GRIDMIX_MEMORY_EMULATON = + ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS; + + /** + * Gridmix memory emulation lower limit. + */ + public static int GRIDMIX_MEMORY_EMULATION_LOWER_LIMIT = 55; + + /** + * Gridmix memory emulation upper limit. + */ + public static int GRIDMIX_MEMORY_EMULATION_UPPER_LIMIT = 130; + } diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobVerification.java b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobVerification.java index ae71ec5764..e448412a39 100644 --- a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobVerification.java +++ b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobVerification.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.TaskCounter; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.CounterGroup; @@ -105,7 +106,7 @@ public GridmixJobVerification(Path path, Configuration conf, * @throws ParseException - if an parse error occurs. */ public void verifyGridmixJobsWithJobStories(List jobids) - throws IOException, ParseException { + throws Exception { SortedMap origSubmissionTime = new TreeMap (); SortedMap simuSubmissionTime = new TreeMap(); @@ -147,6 +148,8 @@ public void verifyGridmixJobsWithJobStories(List jobids) setJobDistributedCacheInfo(simuJobId.toString(), simuJobConf, zombieJob.getJobConf()); verifyHighRamMemoryJobs(zombieJob, simuJobConf); + verifyCPUEmulationOfJobs(zombieJob, jhInfo, simuJobConf); + verifyMemoryEmulationOfJobs(zombieJob, jhInfo, simuJobConf); LOG.info("Done."); } verifyDistributedCacheBetweenJobs(simuAndOrigJobsInfo); @@ -353,6 +356,229 @@ public JobHistoryParser.JobInfo getSimulatedJobHistory(JobID simulatedJobID) fs.close(); } } + + /** + * It verifies the heap memory resource usage of gridmix jobs with + * corresponding original job in the trace. + * @param zombieJob - Original job history. + * @param jhInfo - Simulated job history. + * @param simuJobConf - simulated job configuration. + */ + public void verifyMemoryEmulationOfJobs(ZombieJob zombieJob, + JobHistoryParser.JobInfo jhInfo, + JobConf simuJobConf) throws Exception { + long origJobMapsTHU = 0; + long origJobReducesTHU = 0; + long simuJobMapsTHU = 0; + long simuJobReducesTHU = 0; + boolean isMemEmulOn = false; + if (simuJobConf.get(GridMixConfig.GRIDMIX_MEMORY_EMULATON) != null) { + isMemEmulOn = + simuJobConf.get(GridMixConfig.GRIDMIX_MEMORY_EMULATON). + contains(GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN); + } + + if (isMemEmulOn) { + for (int index = 0; index < zombieJob.getNumberMaps(); index ++) { + TaskInfo mapTask = zombieJob.getTaskInfo(TaskType.MAP, index); + if (mapTask.getResourceUsageMetrics().getHeapUsage() > 0) { + origJobMapsTHU += + mapTask.getResourceUsageMetrics().getHeapUsage(); + } + } + LOG.info("Original Job Maps Total Heap Usage: " + origJobMapsTHU); + + for (int index = 0; index < zombieJob.getNumberReduces(); index ++) { + TaskInfo reduceTask = zombieJob.getTaskInfo(TaskType.REDUCE, index); + if (reduceTask.getResourceUsageMetrics().getHeapUsage() > 0) { + origJobReducesTHU += + reduceTask.getResourceUsageMetrics().getHeapUsage(); + } + } + LOG.info("Original Job Reduces Total Heap Usage: " + origJobReducesTHU); + + simuJobMapsTHU = + getCounterValue(jhInfo.getMapCounters(), + TaskCounter.COMMITTED_HEAP_BYTES.toString()); + LOG.info("Simulated Job Maps Total Heap Usage: " + simuJobMapsTHU); + + simuJobReducesTHU = + getCounterValue(jhInfo.getReduceCounters(), + TaskCounter.COMMITTED_HEAP_BYTES.toString()); + LOG.info("Simulated Jobs Reduces Total Heap Usage: " + simuJobReducesTHU); + + long mapCount = jhInfo.getTotalMaps(); + long reduceCount = jhInfo.getTotalReduces(); + + String strHeapRatio = + simuJobConf.get(GridMixConfig.GRIDMIX_HEAP_FREE_MEMORY_RATIO); + if (strHeapRatio == null) { + strHeapRatio = "0.3F"; + } + + if (mapCount > 0) { + double mapEmulFactor = (simuJobMapsTHU * 100) / origJobMapsTHU; + long mapEmulAccuracy = Math.round(mapEmulFactor); + LOG.info("Maps memory emulation accuracy of a job:" + + mapEmulAccuracy + "%"); + Assert.assertTrue("Map phase total memory emulation had crossed the " + + "configured max limit.", mapEmulAccuracy + <= GridMixConfig.GRIDMIX_MEMORY_EMULATION_UPPER_LIMIT); + Assert.assertTrue("Map phase total memory emulation had not crossed " + + "the configured min limit.", mapEmulAccuracy + >= GridMixConfig.GRIDMIX_MEMORY_EMULATION_LOWER_LIMIT); + double expHeapRatio = Double.parseDouble(strHeapRatio); + LOG.info("expHeapRatio for maps:" + expHeapRatio); + double actHeapRatio = + ((double)Math.abs(origJobMapsTHU - simuJobMapsTHU)) ; + actHeapRatio /= origJobMapsTHU; + LOG.info("actHeapRatio for maps:" + actHeapRatio); + Assert.assertTrue("Simulate job maps heap ratio not matched.", + actHeapRatio <= expHeapRatio); + } + + if (reduceCount >0) { + double reduceEmulFactor = (simuJobReducesTHU * 100) / origJobReducesTHU; + long reduceEmulAccuracy = Math.round(reduceEmulFactor); + LOG.info("Reduces memory emulation accuracy of a job:" + + reduceEmulAccuracy + "%"); + Assert.assertTrue("Reduce phase total memory emulation had crossed " + + "configured max limit.", reduceEmulAccuracy + <= GridMixConfig.GRIDMIX_MEMORY_EMULATION_UPPER_LIMIT); + Assert.assertTrue("Reduce phase total memory emulation had not " + + "crosssed configured min limit.", reduceEmulAccuracy + >= GridMixConfig.GRIDMIX_MEMORY_EMULATION_LOWER_LIMIT); + double expHeapRatio = Double.parseDouble(strHeapRatio); + LOG.info("expHeapRatio for reduces:" + expHeapRatio); + double actHeapRatio = + ((double)Math.abs(origJobReducesTHU - simuJobReducesTHU)); + actHeapRatio /= origJobReducesTHU; + LOG.info("actHeapRatio for reduces:" + actHeapRatio); + Assert.assertTrue("Simulate job reduces heap ratio not matched.", + actHeapRatio <= expHeapRatio); + } + } + } + + /** + * It verifies the cpu resource usage of a gridmix job against + * their original job. + * @param origJobHistory - Original job history. + * @param simuJobHistoryInfo - Simulated job history. + * @param simuJobConf - simulated job configuration. + */ + public void verifyCPUEmulationOfJobs(ZombieJob origJobHistory, + JobHistoryParser.JobInfo simuJobHistoryInfo, + JobConf simuJobConf) throws Exception { + + boolean isCpuEmulOn = false; + if (simuJobConf.get(GridMixConfig.GRIDMIX_CPU_EMULATON) != null) { + isCpuEmulOn = + simuJobConf.get(GridMixConfig.GRIDMIX_CPU_EMULATON). + contains(GridMixConfig.GRIDMIX_CPU_USAGE_PLUGIN); + } + + if (isCpuEmulOn) { + Map origJobMetrics = + getOriginalJobCPUMetrics(origJobHistory); + Map simuJobMetrics = + getSimulatedJobCPUMetrics(simuJobHistoryInfo); + + long origMapUsage = origJobMetrics.get("MAP"); + LOG.info("Maps cpu usage of original job:" + origMapUsage); + + long origReduceUsage = origJobMetrics.get("REDUCE"); + LOG.info("Reduces cpu usage of original job:" + origReduceUsage); + + long simuMapUsage = simuJobMetrics.get("MAP"); + LOG.info("Maps cpu usage of simulated job:" + simuMapUsage); + + long simuReduceUsage = simuJobMetrics.get("REDUCE"); + LOG.info("Reduces cpu usage of simulated job:"+ simuReduceUsage); + + long mapCount = simuJobHistoryInfo.getTotalMaps(); + long reduceCount = simuJobHistoryInfo.getTotalReduces(); + + if (mapCount > 0) { + double mapEmulFactor = (simuMapUsage * 100) / origMapUsage; + long mapEmulAccuracy = Math.round(mapEmulFactor); + LOG.info("CPU emulation accuracy for maps in job " + + simuJobHistoryInfo.getJobId() + + ":"+ mapEmulAccuracy + "%"); + Assert.assertTrue("Map-side cpu emulaiton inaccurate!" + + " Actual cpu usage: " + simuMapUsage + + " Expected cpu usage: " + origMapUsage, mapEmulAccuracy + >= GridMixConfig.GRIDMIX_CPU_EMULATION_LOWER_LIMIT + && mapEmulAccuracy + <= GridMixConfig.GRIDMIX_CPU_EMULATION_UPPER_LIMIT); + } + + if (reduceCount >0) { + double reduceEmulFactor = (simuReduceUsage * 100) / origReduceUsage; + long reduceEmulAccuracy = Math.round(reduceEmulFactor); + LOG.info("CPU emulation accuracy for reduces in job " + + simuJobHistoryInfo.getJobId() + + ": " + reduceEmulAccuracy + "%"); + Assert.assertTrue("Reduce side cpu emulaiton inaccurate!" + + " Actual cpu usage:" + simuReduceUsage + + "Expected cpu usage: " + origReduceUsage, + reduceEmulAccuracy + >= GridMixConfig.GRIDMIX_CPU_EMULATION_LOWER_LIMIT + && reduceEmulAccuracy + <= GridMixConfig.GRIDMIX_CPU_EMULATION_UPPER_LIMIT); + } + } + } + + /** + * Get the simulated job cpu metrics. + * @param jhInfo - Simulated job history + * @return - cpu metrics as a map. + * @throws Exception - if an error occurs. + */ + private Map getSimulatedJobCPUMetrics( + JobHistoryParser.JobInfo jhInfo) throws Exception { + Map resourceMetrics = new HashMap(); + long mapCPUUsage = + getCounterValue(jhInfo.getMapCounters(), + TaskCounter.CPU_MILLISECONDS.toString()); + resourceMetrics.put("MAP", mapCPUUsage); + long reduceCPUUsage = + getCounterValue(jhInfo.getReduceCounters(), + TaskCounter.CPU_MILLISECONDS.toString()); + resourceMetrics.put("REDUCE", reduceCPUUsage); + return resourceMetrics; + } + + /** + * Get the original job cpu metrics. + * @param zombieJob - original job history. + * @return - cpu metrics as map. + */ + private Map getOriginalJobCPUMetrics(ZombieJob zombieJob) { + long mapTotalCPUUsage = 0; + long reduceTotalCPUUsage = 0; + Map resourceMetrics = new HashMap(); + + for (int index = 0; index < zombieJob.getNumberMaps(); index ++) { + TaskInfo mapTask = zombieJob.getTaskInfo(TaskType.MAP, index); + if (mapTask.getResourceUsageMetrics().getCumulativeCpuUsage() > 0) { + mapTotalCPUUsage += + mapTask.getResourceUsageMetrics().getCumulativeCpuUsage(); + } + } + resourceMetrics.put("MAP", mapTotalCPUUsage); + + for (int index = 0; index < zombieJob.getNumberReduces(); index ++) { + TaskInfo reduceTask = zombieJob.getTaskInfo(TaskType.REDUCE, index); + if (reduceTask.getResourceUsageMetrics().getCumulativeCpuUsage() > 0) { + reduceTotalCPUUsage += + reduceTask.getResourceUsageMetrics().getCumulativeCpuUsage(); + } + } + resourceMetrics.put("REDUCE", reduceTotalCPUUsage); + return resourceMetrics; + } /** * Get the user resolver of a job. diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/cpu_emul_case1.json.gz b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/cpu_emul_case1.json.gz new file mode 100644 index 0000000000..211773857d Binary files /dev/null and b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/cpu_emul_case1.json.gz differ diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/cpu_emul_case2.json.gz b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/cpu_emul_case2.json.gz new file mode 100644 index 0000000000..b230610515 Binary files /dev/null and b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/cpu_emul_case2.json.gz differ diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/mem_emul_case1.json.gz b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/mem_emul_case1.json.gz new file mode 100644 index 0000000000..5f7fcab1d5 Binary files /dev/null and b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/mem_emul_case1.json.gz differ diff --git a/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/mem_emul_case2.json.gz b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/mem_emul_case2.json.gz new file mode 100644 index 0000000000..d0ea21e6b4 Binary files /dev/null and b/hadoop-mapreduce-project/src/contrib/gridmix/src/test/system/resources/mem_emul_case2.json.gz differ diff --git a/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java b/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java index b68e73e34b..2da171aecf 100644 --- a/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java +++ b/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java @@ -984,19 +984,6 @@ protected String getJobTrackerHostPort() { return jobConf_.get(JTConfig.JT_IPC_ADDRESS); } - protected void jobInfo() { - if (isLocalHadoop()) { - LOG.info("Job running in-process (local Hadoop)"); - } else { - String hp = getJobTrackerHostPort(); - LOG.info("To kill this job, run:"); - LOG.info(getHadoopClientHome() + "/bin/hadoop job -D" + JTConfig.JT_IPC_ADDRESS + "=" + hp + " -kill " - + jobId_); - //LOG.info("Job file: " + running_.getJobFile()); - LOG.info("Tracking URL: " + StreamUtil.qualifyHost(running_.getTrackingURL())); - } - } - // Based on JobClient public int submitAndMonitorJob() throws IOException { @@ -1012,7 +999,6 @@ public int submitAndMonitorJob() throws IOException { try { running_ = jc_.submitJob(jobConf_); jobId_ = running_.getID(); - jobInfo(); if (background_) { LOG.info("Job is running in background."); } else if (!jc_.monitorAndPrintJob(jobConf_, running_)) { diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/fs/slive/SlivePartitioner.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/fs/slive/SlivePartitioner.java index dfe602a936..4016124afb 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/fs/slive/SlivePartitioner.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/fs/slive/SlivePartitioner.java @@ -34,6 +34,6 @@ public void configure(JobConf conf) {} @Override // Partitioner public int getPartition(Text key, Text value, int numPartitions) { OperationOutput oo = new OperationOutput(key, value); - return oo.getOperationType().hashCode() % numPartitions; + return (oo.getOperationType().hashCode() & Integer.MAX_VALUE) % numPartitions; } } diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestAuditLogger.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestAuditLogger.java index 145442071d..a6aebb0e4e 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestAuditLogger.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestAuditLogger.java @@ -21,6 +21,7 @@ import java.net.InetSocketAddress; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.ProtocolInfo; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.TestRPC.TestImpl; @@ -123,6 +124,7 @@ public void testAuditLoggerWithoutIP() throws Exception { * A special extension of {@link TestImpl} RPC server with * {@link TestImpl#ping()} testing the audit logs. */ + @ProtocolInfo(protocolName = "org.apache.hadoop.ipc.TestRPC$TestProtocol") private class MyTestRPCServer extends TestImpl { @Override public void ping() { @@ -135,10 +137,8 @@ public void ping() { /** * Test {@link AuditLogger} with IP set. */ + @SuppressWarnings("deprecation") public void testAuditLoggerWithIP() throws Exception { - /* - // TODO - // Disable test to address build failures. Configuration conf = new Configuration(); // start the IPC server Server server = RPC.getServer(new MyTestRPCServer(), "0.0.0.0", 0, conf); @@ -153,6 +153,5 @@ public void testAuditLoggerWithIP() throws Exception { proxy.ping(); server.stop(); - */ } } diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java index c24ff38d5b..c32ead05b8 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java @@ -17,44 +17,33 @@ */ package org.apache.hadoop.mapred; -import static org.junit.Assert.*; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; -import java.io.DataOutputStream; import java.io.IOException; import java.net.URI; import java.security.PrivilegedExceptionAction; -import java.util.HashMap; -import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol; import org.apache.hadoop.hdfs.server.namenode.NameNode; -import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.mapred.lib.IdentityMapper; -import org.apache.hadoop.mapred.lib.IdentityReducer; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.SleepJob; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; -import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ToolRunner; - -import org.junit.After; -import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; /** @@ -68,42 +57,10 @@ public class TestSubmitJob { static final Log LOG = LogFactory.getLog(TestSubmitJob.class); - private MiniMRCluster mrCluster; - - private MiniDFSCluster dfsCluster; - private JobTracker jt; - private FileSystem fs; private static Path TEST_DIR = new Path(System.getProperty("test.build.data","/tmp"), "job-submission-testing"); - private static int numSlaves = 1; - @Before - public void startCluster() throws Exception { - Configuration conf = new Configuration(); - dfsCluster = new MiniDFSCluster(conf, numSlaves, true, null); - JobConf jConf = new JobConf(conf); - jConf.setLong("mapred.job.submission.expiry.interval", 6 * 1000); - mrCluster = new MiniMRCluster(0, 0, numSlaves, - dfsCluster.getFileSystem().getUri().toString(), 1, null, null, null, - jConf); - jt = mrCluster.getJobTrackerRunner().getJobTracker(); - fs = FileSystem.get(mrCluster.createJobConf()); - } - - @After - public void stopCluster() throws Exception { - if (mrCluster != null) { - mrCluster.shutdown(); - mrCluster = null; - } - if (dfsCluster != null) { - dfsCluster.shutdown(); - dfsCluster = null; - } - jt = null; - fs = null; - } /** * Test to verify that jobs with invalid memory requirements are killed at the @@ -111,51 +68,53 @@ public void stopCluster() throws Exception { * * @throws Exception */ + @SuppressWarnings("deprecation") @Test - public void testJobWithInvalidMemoryReqs() - throws Exception { - JobConf jtConf = new JobConf(); - jtConf - .setLong(MRConfig.MAPMEMORY_MB, 1 * 1024L); - jtConf.setLong(MRConfig.REDUCEMEMORY_MB, - 2 * 1024L); - jtConf.setLong(JTConfig.JT_MAX_MAPMEMORY_MB, - 3 * 1024L); - jtConf.setLong(JTConfig.JT_MAX_REDUCEMEMORY_MB, - 4 * 1024L); + public void testJobWithInvalidMemoryReqs() throws Exception { + MiniMRCluster mrCluster = null; + try { + JobConf jtConf = new JobConf(); + jtConf.setLong(MRConfig.MAPMEMORY_MB, 1 * 1024L); + jtConf.setLong(MRConfig.REDUCEMEMORY_MB, 2 * 1024L); + jtConf.setLong(JTConfig.JT_MAX_MAPMEMORY_MB, 3 * 1024L); + jtConf.setLong(JTConfig.JT_MAX_REDUCEMEMORY_MB, 4 * 1024L); - mrCluster = new MiniMRCluster(0, "file:///", 0, null, null, jtConf); + mrCluster = new MiniMRCluster(0, "file:///", 0, null, null, jtConf); - JobConf clusterConf = mrCluster.createJobConf(); + JobConf clusterConf = mrCluster.createJobConf(); - // No map-memory configuration - JobConf jobConf = new JobConf(clusterConf); - jobConf.setMemoryForReduceTask(1 * 1024L); - runJobAndVerifyFailure(jobConf, JobConf.DISABLED_MEMORY_LIMIT, 1 * 1024L, - "Invalid job requirements."); + // No map-memory configuration + JobConf jobConf = new JobConf(clusterConf); + jobConf.setMemoryForReduceTask(1 * 1024L); + runJobAndVerifyFailure(jobConf, JobConf.DISABLED_MEMORY_LIMIT, 1 * 1024L, + "Invalid job requirements."); - // No reduce-memory configuration - jobConf = new JobConf(clusterConf); - jobConf.setMemoryForMapTask(1 * 1024L); - runJobAndVerifyFailure(jobConf, 1 * 1024L, JobConf.DISABLED_MEMORY_LIMIT, - "Invalid job requirements."); + // No reduce-memory configuration + jobConf = new JobConf(clusterConf); + jobConf.setMemoryForMapTask(1 * 1024L); + runJobAndVerifyFailure(jobConf, 1 * 1024L, JobConf.DISABLED_MEMORY_LIMIT, + "Invalid job requirements."); - // Invalid map-memory configuration - jobConf = new JobConf(clusterConf); - jobConf.setMemoryForMapTask(4 * 1024L); - jobConf.setMemoryForReduceTask(1 * 1024L); - runJobAndVerifyFailure(jobConf, 4 * 1024L, 1 * 1024L, - "Exceeds the cluster's max-memory-limit."); + // Invalid map-memory configuration + jobConf = new JobConf(clusterConf); + jobConf.setMemoryForMapTask(4 * 1024L); + jobConf.setMemoryForReduceTask(1 * 1024L); + runJobAndVerifyFailure(jobConf, 4 * 1024L, 1 * 1024L, + "Exceeds the cluster's max-memory-limit."); - // No reduce-memory configuration - jobConf = new JobConf(clusterConf); - jobConf.setMemoryForMapTask(1 * 1024L); - jobConf.setMemoryForReduceTask(5 * 1024L); - runJobAndVerifyFailure(jobConf, 1 * 1024L, 5 * 1024L, - "Exceeds the cluster's max-memory-limit."); - + // No reduce-memory configuration + jobConf = new JobConf(clusterConf); + jobConf.setMemoryForMapTask(1 * 1024L); + jobConf.setMemoryForReduceTask(5 * 1024L); + runJobAndVerifyFailure(jobConf, 1 * 1024L, 5 * 1024L, + "Exceeds the cluster's max-memory-limit."); + } finally { + if (mrCluster != null) + mrCluster.shutdown(); + } } + @SuppressWarnings("deprecation") private void runJobAndVerifyFailure(JobConf jobConf, long memForMapTasks, long memForReduceTasks, String expectedMsg) throws Exception, @@ -180,6 +139,7 @@ private void runJobAndVerifyFailure(JobConf jobConf, long memForMapTasks, .contains(overallExpectedMsg)); } + @SuppressWarnings("deprecation") static ClientProtocol getJobSubmitClient(JobConf conf, UserGroupInformation ugi) throws IOException { @@ -188,24 +148,23 @@ static ClientProtocol getJobSubmitClient(JobConf conf, conf, NetUtils.getSocketFactory(conf, ClientProtocol.class)); } - static org.apache.hadoop.hdfs.protocol.ClientProtocol getDFSClient( + static ClientNamenodeWireProtocol getDFSClient( Configuration conf, UserGroupInformation ugi) throws IOException { - return (org.apache.hadoop.hdfs.protocol.ClientProtocol) - RPC.getProxy(org.apache.hadoop.hdfs.protocol.ClientProtocol.class, - org.apache.hadoop.hdfs.protocol.ClientProtocol.versionID, + return (ClientNamenodeWireProtocol) + RPC.getProxy(ClientNamenodeWireProtocol.class, + ClientNamenodeWireProtocol.versionID, NameNode.getAddress(conf), ugi, conf, NetUtils.getSocketFactory(conf, - org.apache.hadoop.hdfs.protocol.ClientProtocol.class)); + ClientNamenodeWireProtocol.class)); } /** * Submit a job and check if the files are accessible to other users. - * TODO fix testcase */ + @SuppressWarnings("deprecation") @Test - @Ignore public void testSecureJobExecution() throws Exception { LOG.info("Testing secure job submission/execution"); MiniMRCluster mr = null; @@ -227,7 +186,6 @@ public FileSystem run() throws IOException { mr = new MiniMRCluster(0, 0, 1, dfs.getFileSystem().getUri().toString(), 1, null, null, MR_UGI); JobTracker jt = mr.getJobTrackerRunner().getJobTracker(); - String jobTrackerName = "localhost:" + mr.getJobTrackerPort(); // cleanup dfs.getFileSystem().delete(TEST_DIR, true); @@ -268,7 +226,7 @@ public RunningJob run() throws IOException { UserGroupInformation user2 = TestMiniMRWithDFSWithDistinctUsers.createUGI("user2", false); JobConf conf_other = mr.createJobConf(); - org.apache.hadoop.hdfs.protocol.ClientProtocol client = + ClientNamenodeWireProtocol client = getDFSClient(conf_other, user2); // try accessing mapred.system.dir/jobid/* diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java index 583572505b..1116d4cda6 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java @@ -56,16 +56,15 @@ private Job submitAndValidateJob(Configuration conf, int numMaps, int numReds) FileSystem fs = FileSystem.get(conf); assertTrue("Job output directory doesn't exit!", fs.exists(outDir)); - // TODO - /* - // Disabling check for now to address builds until we fix underlying issue - // output still in temporary as job commit only seems - // to be called during job cleanup - FileStatus[] list = fs.listStatus(outDir, new OutputFilter()); + // job commit done only in cleanup + // therefore output should still be in temp location + String tempWorkingPathStr = outDir + Path.SEPARATOR + "_temporary" + + Path.SEPARATOR + "0"; + Path tempWorkingPath = new Path(tempWorkingPathStr); + FileStatus[] list = fs.listStatus(tempWorkingPath, new OutputFilter()); int numPartFiles = numReds == 0 ? numMaps : numReds; assertTrue("Number of part-files is " + list.length + " and not " + numPartFiles, list.length == numPartFiles); - */ return job; } diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java index 0ea698330c..4c1e34da94 100644 --- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java +++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java @@ -29,7 +29,6 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import static org.junit.Assert.*; @@ -48,26 +47,27 @@ public void setup() throws Exception { cluster = new MiniMRCluster(0,0,1,"file:///",1); } + @SuppressWarnings("deprecation") @Test - @Ignore public void testDelegationToken() throws Exception { - JobClient client; + final JobClient client; client = user1.doAs(new PrivilegedExceptionAction(){ - @Override public JobClient run() throws Exception { return new JobClient(cluster.createJobConf()); - }}); - JobClient bobClient; - bobClient = user2.doAs(new PrivilegedExceptionAction(){ - - @Override - public JobClient run() throws Exception { - return new JobClient(cluster.createJobConf()); - }}); + } + }); - Token token = + final JobClient bobClient; + bobClient = user2.doAs(new PrivilegedExceptionAction(){ + @Override + public JobClient run() throws Exception { + return new JobClient(cluster.createJobConf()); + } + }); + + final Token token = client.getDelegationToken(new Text(user1.getUserName())); DataInputBuffer inBuf = new DataInputBuffer(); @@ -85,26 +85,58 @@ public JobClient run() throws Exception { System.out.println("max time: " + maxTime); assertTrue("createTime < current", createTime < currentTime); assertTrue("current < maxTime", currentTime < maxTime); - client.renewDelegationToken(token); - client.renewDelegationToken(token); - try { - bobClient.renewDelegationToken(token); - Assert.fail("bob renew"); - } catch (AccessControlException ace) { - // PASS - } - try { - bobClient.cancelDelegationToken(token); - Assert.fail("bob renew"); - } catch (AccessControlException ace) { - // PASS - } - client.cancelDelegationToken(token); - try { - client.cancelDelegationToken(token); - Assert.fail("second alice cancel"); - } catch (InvalidToken it) { - // PASS - } + + // renew should work as user alice + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + client.renewDelegationToken(token); + client.renewDelegationToken(token); + return null; + } + }); + + // bob should fail to renew + user2.doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + try { + bobClient.renewDelegationToken(token); + Assert.fail("bob renew"); + } catch (AccessControlException ace) { + // PASS + } + return null; + } + }); + + // bob should fail to cancel + user2.doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + try { + bobClient.cancelDelegationToken(token); + Assert.fail("bob cancel"); + } catch (AccessControlException ace) { + // PASS + } + return null; + } + }); + + // alice should be able to cancel but only cancel once + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + client.cancelDelegationToken(token); + try { + client.cancelDelegationToken(token); + Assert.fail("second alice cancel"); + } catch (InvalidToken it) { + // PASS + } + return null; + } + }); } } diff --git a/hadoop-mapreduce-project/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java b/hadoop-mapreduce-project/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java index fde7d0545e..5437a15003 100644 --- a/hadoop-mapreduce-project/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java +++ b/hadoop-mapreduce-project/src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java @@ -328,11 +328,27 @@ public String getHostName() { return hostName; } - + void setHostName(String hostName) { + this.hostName = hostName; + } + // hostName is saved in the format rackName/NodeName void setHostName(String hostName, String rackName) { - this.hostName = hostName == null || rackName == null ? null - : rackName.intern() + "/" + hostName.intern(); + if (hostName == null || hostName.length() == 0) { + throw new RuntimeException("Invalid entry! Missing hostname"); + } else if (rackName == null || rackName.length() == 0) { + setHostName(hostName); + } else { + // make sure that the rackname is prefixed with a '/' + if (!rackName.startsWith("/")) { + rackName = "/" + rackName; + } + // make sure that the hostname is prefixed with a '/' + if (!hostName.startsWith("/")) { + hostName = "/" + hostName; + } + setHostName(rackName.intern() + hostName.intern()); + } } public long getHdfsBytesRead() { diff --git a/pom.xml b/pom.xml index 2a923cd238..23b9d69157 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ - apache.releases.https + apache.staging.https Apache Release Distribution Repository https://repository.apache.org/service/local/staging/deploy/maven2 @@ -299,5 +299,85 @@ + + dist + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + module-javadocs + package + + jar + + + + + default-cli + + aggregate + + false + + hadoop-common-project/hadoop-common/src/main/java/overview.html + + + + + + org.apache.maven.plugins + maven-source-plugin + + + + hadoop-java-sources + package + + jar-no-fork + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + package + + enforce + + + + + + + + + sign + + + + org.apache.maven.plugins + maven-gpg-plugin + + + sign-artifacts + verify + + sign + + + + + + + +