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 extends TokenIdentifier> 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 extends TokenIdentifier> 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 extends TokenIdentifier> 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 extends T> 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