Merge trunk into HA branch
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1202013 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
0d171f6513
@ -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."
|
||||
|
@ -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
|
||||
|
@ -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 ]
|
||||
|
@ -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) {
|
||||
|
@ -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 <code>param</code>, to the IPC server running at
|
||||
* <code>address</code>, 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 <code>param</code>, 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 <code>param</code>, 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 <code>param</code>, to the IPC server running at
|
||||
* <code>address</code> which is servicing the <code>protocol</code> 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 <code>param</code>, to the IPC server defined by
|
||||
* <code>remoteId</code>, 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
* <pre>
|
||||
* 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.
|
||||
* </pre>
|
||||
* <p>
|
||||
* <b>Note this header does NOT have its own version number,
|
||||
* it used the version number from the connection header. </b>
|
||||
*/
|
||||
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();
|
||||
}
|
||||
}
|
@ -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<Call> 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<Call> 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<Call> 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) {
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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 "
|
||||
|
@ -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): "
|
||||
|
@ -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): "
|
||||
|
@ -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): "
|
||||
|
@ -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"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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<BlockTokenIdentifier> 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<BlockTokenIdentifier> 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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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";
|
||||
|
@ -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,
|
||||
|
@ -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<CreateFlag> 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();
|
||||
|
@ -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. */
|
||||
|
@ -50,27 +50,13 @@
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
|
||||
/** This is a wrapper around connection to datanode
|
||||
* and understands checksum, offset etc.
|
||||
*
|
||||
* Terminology:
|
||||
* <dl>
|
||||
* <dt>block</dt>
|
||||
* <dd>The hdfs block, typically large (~64MB).
|
||||
* </dd>
|
||||
* <dt>chunk</dt>
|
||||
* <dd>A block is divided into chunks, each comes with a checksum.
|
||||
* We want transfers to be chunk-aligned, to be able to
|
||||
* verify checksums.
|
||||
* </dd>
|
||||
* <dt>packet</dt>
|
||||
* <dd>A grouping of chunks used for transport. It contains a
|
||||
* header, followed by checksum data, followed by real data.
|
||||
* </dd>
|
||||
* </dl>
|
||||
* 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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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:
|
||||
* <dl>
|
||||
* <dt>block</dt>
|
||||
* <dd>The hdfs block, typically large (~64MB).
|
||||
* </dd>
|
||||
* <dt>chunk</dt>
|
||||
* <dd>A block is divided into chunks, each comes with a checksum.
|
||||
* We want transfers to be chunk-aligned, to be able to
|
||||
* verify checksums.
|
||||
* </dd>
|
||||
* <dt>packet</dt>
|
||||
* <dd>A grouping of chunks used for transport. It contains a
|
||||
* header, followed by checksum data, followed by real data.
|
||||
* </dd>
|
||||
* </dl>
|
||||
* 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<BlockTokenIdentifier> 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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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
|
||||
|
@ -134,7 +134,7 @@ public String toString() {
|
||||
|
||||
private DelayQueue<RenewAction<T>> queue = new DelayQueue<RenewAction<T>>();
|
||||
|
||||
public DelegationTokenRenewer(final Class<?> clazz) {
|
||||
public DelegationTokenRenewer(final Class<T> clazz) {
|
||||
super(clazz.getSimpleName() + "-" + DelegationTokenRenewer.class.getSimpleName());
|
||||
setDaemon(true);
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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<Block>
|
||||
//
|
||||
public final Map<String, Collection<Block>> excessReplicateMap =
|
||||
new TreeMap<String, Collection<Block>>();
|
||||
public final Map<String, LightWeightLinkedSet<Block>> excessReplicateMap =
|
||||
new TreeMap<String, LightWeightLinkedSet<Block>>();
|
||||
|
||||
//
|
||||
// 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<blocksToReplicate.size(); i++) {
|
||||
for(Block block : blocksToReplicate.get(i)) {
|
||||
if (computeReplicationWorkForBlock(block, i)) {
|
||||
scheduledReplicationCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return scheduledReplicationCount;
|
||||
return computeReplicationWorkForBlocks(blocksToReplicate);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -1001,170 +995,201 @@ private List<List<Block>> 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<List<Block>> blocksToReplicate) {
|
||||
int requiredReplication, numEffectiveReplicas;
|
||||
List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
|
||||
DatanodeDescriptor srcNode;
|
||||
INodeFile fileINode = null;
|
||||
int additionalReplRequired;
|
||||
|
||||
int scheduledWork = 0;
|
||||
List<ReplicationWork> work = new LinkedList<ReplicationWork>();
|
||||
|
||||
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<DatanodeDescriptor>();
|
||||
liveReplicaNodes = new ArrayList<DatanodeDescriptor>();
|
||||
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<DatanodeDescriptor>();
|
||||
liveReplicaNodes = new ArrayList<DatanodeDescriptor>();
|
||||
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<Node, Node> excludedNodes = new HashMap<Node, Node>();
|
||||
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<Node, Node> excludedNodes
|
||||
= new HashMap<Node, Node>();
|
||||
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<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
|
||||
while(it.hasNext()) {
|
||||
DatanodeDescriptor node = it.next();
|
||||
Collection<Block> excessBlocks =
|
||||
LightWeightLinkedSet<Block> 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<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
|
||||
it.hasNext();) {
|
||||
DatanodeDescriptor cur = it.next();
|
||||
Collection<Block> excessBlocks = excessReplicateMap.get(cur
|
||||
LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(cur
|
||||
.getStorageID());
|
||||
if (excessBlocks == null || !excessBlocks.contains(block)) {
|
||||
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
|
||||
@ -1989,9 +2018,9 @@ private void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess,
|
||||
|
||||
private void addToExcessReplicate(DatanodeInfo dn, Block block) {
|
||||
assert namesystem.hasWriteLock();
|
||||
Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
|
||||
LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
|
||||
if (excessBlocks == null) {
|
||||
excessBlocks = new TreeSet<Block>();
|
||||
excessBlocks = new LightWeightLinkedSet<Block>();
|
||||
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<Block> excessBlocks = excessReplicateMap.get(node
|
||||
LightWeightLinkedSet<Block> 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<Block> blocksExcess =
|
||||
excessReplicateMap.get(node.getStorageID());
|
||||
LightWeightLinkedSet<Block> 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<DatanodeDescriptor> containingNodes;
|
||||
private List<DatanodeDescriptor> liveReplicaNodes;
|
||||
private int additionalReplRequired;
|
||||
|
||||
private DatanodeDescriptor targets[];
|
||||
private int priority;
|
||||
|
||||
public ReplicationWork(Block block,
|
||||
INodeFile fileINode,
|
||||
DatanodeDescriptor srcNode,
|
||||
List<DatanodeDescriptor> containingNodes,
|
||||
List<DatanodeDescriptor> 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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<BlockInfoUnderConstruction> recoverBlocks =
|
||||
new BlockQueue<BlockInfoUnderConstruction>();
|
||||
/** A set of blocks to be invalidated by this datanode */
|
||||
private Set<Block> invalidateBlocks = new TreeSet<Block>();
|
||||
private LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
|
||||
|
||||
/* 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<Block> 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<Block> 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 */
|
||||
|
@ -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<String, Collection<Block>> node2blocks =
|
||||
new TreeMap<String, Collection<Block>>();
|
||||
private final Map<String, LightWeightHashSet<Block>> node2blocks =
|
||||
new TreeMap<String, LightWeightHashSet<Block>>();
|
||||
/** 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<Block> set = node2blocks.get(datanode.getStorageID());
|
||||
LightWeightHashSet<Block> set = node2blocks.get(datanode.getStorageID());
|
||||
if (set == null) {
|
||||
set = new HashSet<Block>();
|
||||
set = new LightWeightHashSet<Block>();
|
||||
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<Block> blocks = node2blocks.remove(storageID);
|
||||
final LightWeightHashSet<Block> 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<Block> v = node2blocks.get(storageID);
|
||||
final LightWeightHashSet<Block> 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<String,Collection<Block>> entry : node2blocks.entrySet()) {
|
||||
final Collection<Block> blocks = entry.getValue();
|
||||
for(Map.Entry<String,LightWeightHashSet<Block>> entry : node2blocks.entrySet()) {
|
||||
final LightWeightHashSet<Block> 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<Block> invalidateWork(
|
||||
final String storageId, final DatanodeDescriptor dn) {
|
||||
final Collection<Block> set = node2blocks.get(storageId);
|
||||
final LightWeightHashSet<Block> 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<Block> toInvalidate = new ArrayList<Block>(limit);
|
||||
final Iterator<Block> it = set.iterator();
|
||||
for(int count = 0; count < limit && it.hasNext(); count++) {
|
||||
toInvalidate.add(it.next());
|
||||
it.remove();
|
||||
}
|
||||
final List<Block> toInvalidate = set.pollN(limit);
|
||||
|
||||
// If we send everything in this message, remove this node entry
|
||||
if (!it.hasNext()) {
|
||||
if (set.isEmpty()) {
|
||||
remove(storageId);
|
||||
}
|
||||
|
||||
|
@ -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<Block> {
|
||||
/** The queue for corrupt blocks: {@value} */
|
||||
static final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
|
||||
/** the queues themselves */
|
||||
private final List<NavigableSet<Block>> priorityQueues
|
||||
= new ArrayList<NavigableSet<Block>>(LEVEL);
|
||||
private List<LightWeightLinkedSet<Block>> priorityQueues
|
||||
= new ArrayList<LightWeightLinkedSet<Block>>();
|
||||
|
||||
/** Create an object. */
|
||||
UnderReplicatedBlocks() {
|
||||
for (int i = 0; i < LEVEL; i++) {
|
||||
priorityQueues.add(new TreeSet<Block>());
|
||||
priorityQueues.add(new LightWeightLinkedSet<Block>());
|
||||
}
|
||||
}
|
||||
|
||||
@ -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<Block> set : priorityQueues) {
|
||||
for(LightWeightLinkedSet<Block> set : priorityQueues) {
|
||||
if (set.contains(block)) {
|
||||
return true;
|
||||
}
|
||||
|
@ -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<DatanodeInfo, NodeRecord> map =
|
||||
new HashMap<DatanodeInfo, NodeRecord>();
|
||||
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<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
|
||||
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<DelegationTokenIdentifier> 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);
|
||||
|
@ -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 <em>not</em> 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 {
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>();
|
||||
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<Response>() {
|
||||
@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<Response>() {
|
||||
@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<Response>() {
|
||||
@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:
|
||||
|
@ -4050,7 +4050,7 @@ public String toString() {
|
||||
* @throws IOException
|
||||
*/
|
||||
Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
|
||||
String startBlockAfter) throws IOException {
|
||||
String[] cookieTab) throws IOException {
|
||||
|
||||
readLock();
|
||||
try {
|
||||
@ -4059,23 +4059,27 @@ Collection<CorruptFileBlockInfo> 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<CorruptFileBlockInfo> corruptFiles = new ArrayList<CorruptFileBlockInfo>();
|
||||
|
||||
if (startBlockAfter != null) {
|
||||
startBlockId = Block.filename2id(startBlockAfter);
|
||||
}
|
||||
|
||||
final Iterator<Block> 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<CorruptFileBlockInfo> 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
|
||||
*/
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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<FSNamesystem.CorruptFileBlockInfo> 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]);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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<FSNamesystem.CorruptFileBlockInfo> 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
|
||||
|
@ -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<Response>() {
|
||||
@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<Response>() {
|
||||
@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<Response>() {
|
||||
@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<Response>() {
|
||||
@Override
|
||||
|
@ -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();
|
||||
|
@ -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<Integer, Queue<WeakReference<ByteBuffer>>> buffersBySize =
|
||||
new ConcurrentHashMap<Integer, Queue<WeakReference<ByteBuffer>>>();
|
||||
|
||||
/**
|
||||
* 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<WeakReference<ByteBuffer>> list = buffersBySize.get(size);
|
||||
if (list == null) {
|
||||
// no available buffers for this size
|
||||
return ByteBuffer.allocateDirect(size);
|
||||
}
|
||||
|
||||
WeakReference<ByteBuffer> 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<WeakReference<ByteBuffer>> list = buffersBySize.get(size);
|
||||
if (list == null) {
|
||||
list = new ConcurrentLinkedQueue<WeakReference<ByteBuffer>>();
|
||||
Queue<WeakReference<ByteBuffer>> 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<ByteBuffer>(buf));
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the number of available buffers of a given size.
|
||||
* This is used only for tests.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
int countBuffersOfSize(int size) {
|
||||
Queue<WeakReference<ByteBuffer>> list = buffersBySize.get(size);
|
||||
if (list == null) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
return list.size();
|
||||
}
|
||||
}
|
@ -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<T> implements Collection<T> {
|
||||
/**
|
||||
* Elements of {@link LightWeightLinkedSet}.
|
||||
*/
|
||||
static class LinkedElement<T> {
|
||||
protected final T element;
|
||||
|
||||
// reference to the next entry within a bucket linked list
|
||||
protected LinkedElement<T> 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<T>[] 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<T> 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<T> le = new LinkedElement<T>(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<T> 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<T> removeElem(final T key) {
|
||||
LinkedElement<T> 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<T> 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<T> pollN(int n) {
|
||||
if (n >= size) {
|
||||
return pollAll();
|
||||
}
|
||||
List<T> retList = new ArrayList<T>(n);
|
||||
if (n == 0) {
|
||||
return retList;
|
||||
}
|
||||
boolean done = false;
|
||||
int currentBucketIndex = 0;
|
||||
|
||||
while (!done) {
|
||||
LinkedElement<T> 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<T> pollAll() {
|
||||
List<T> retList = new ArrayList<T>(size);
|
||||
for (int i = 0; i < entries.length; i++) {
|
||||
LinkedElement<T> 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<T> 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<T>[] temp = entries;
|
||||
entries = new LinkedElement[capacity];
|
||||
for (int i = 0; i < temp.length; i++) {
|
||||
LinkedElement<T> curr = temp[i];
|
||||
while (curr != null) {
|
||||
LinkedElement<T> 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<T> 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<T> 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<T> {
|
||||
/** 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<T> next = nextNonemptyEntry();
|
||||
|
||||
private LinkedElement<T> 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<T> 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> 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<T> 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.");
|
||||
}
|
||||
}
|
@ -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<T> extends LightWeightHashSet<T> {
|
||||
/**
|
||||
* Elements of {@link LightWeightLinkedSet}.
|
||||
*/
|
||||
static class DoubleLinkedElement<T> extends LinkedElement<T> {
|
||||
// references to elements within all-element linked list
|
||||
private DoubleLinkedElement<T> before;
|
||||
private DoubleLinkedElement<T> after;
|
||||
|
||||
public DoubleLinkedElement(T elem, int hashCode) {
|
||||
super(elem, hashCode);
|
||||
this.before = null;
|
||||
this.after = null;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return super.toString();
|
||||
}
|
||||
}
|
||||
|
||||
private DoubleLinkedElement<T> head;
|
||||
private DoubleLinkedElement<T> 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<T> le = new DoubleLinkedElement<T>(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<T> removeElem(final T key) {
|
||||
DoubleLinkedElement<T> found = (DoubleLinkedElement<T>) (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<T> pollN(int n) {
|
||||
if (n >= size) {
|
||||
// if we need to remove all elements then do fast polling
|
||||
return pollAll();
|
||||
}
|
||||
List<T> retList = new ArrayList<T>(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<T> pollAll() {
|
||||
List<T> retList = new ArrayList<T>(size);
|
||||
while (head != null) {
|
||||
retList.add(head.element);
|
||||
head = head.after;
|
||||
}
|
||||
this.clear();
|
||||
return retList;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public <U> 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<T> current = head;
|
||||
while (current != null) {
|
||||
T curr = current.element;
|
||||
a[currentIndex++] = (U) curr;
|
||||
current = current.after;
|
||||
}
|
||||
return a;
|
||||
}
|
||||
|
||||
public Iterator<T> iterator() {
|
||||
return new LinkedSetIterator();
|
||||
}
|
||||
|
||||
private class LinkedSetIterator implements Iterator<T> {
|
||||
/** The starting modification for fail-fast. */
|
||||
private final int startModification = modification;
|
||||
/** The next element to return. */
|
||||
private DoubleLinkedElement<T> 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;
|
||||
}
|
||||
}
|
@ -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<String, String[]> original = (Map<String, String[]>)request.getParameterMap();
|
||||
if (!ParamFilter.containsUpperCase(original.keySet())) {
|
||||
return request;
|
||||
}
|
||||
|
||||
final Map<String, List<String>> m = new HashMap<String, List<String>>();
|
||||
for(Map.Entry<String, String[]> entry : original.entrySet()) {
|
||||
final String key = entry.getKey().toLowerCase();
|
||||
List<String> strings = m.get(key);
|
||||
if (strings == null) {
|
||||
strings = new ArrayList<String>();
|
||||
m.put(key, strings);
|
||||
}
|
||||
for(String v : entry.getValue()) {
|
||||
strings.add(v);
|
||||
}
|
||||
}
|
||||
|
||||
return new HttpServletRequestWrapper(request) {
|
||||
private Map<String, String[]> parameters = null;
|
||||
|
||||
@Override
|
||||
public Map<String, String[]> getParameterMap() {
|
||||
if (parameters == null) {
|
||||
parameters = new HashMap<String, String[]>();
|
||||
for(Map.Entry<String, List<String>> entry : m.entrySet()) {
|
||||
final List<String> a = entry.getValue();
|
||||
parameters.put(entry.getKey(), a.toArray(new String[a.size()]));
|
||||
}
|
||||
}
|
||||
return parameters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getParameter(String name) {
|
||||
final List<String> a = m.get(name);
|
||||
return a == null? null: a.get(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] getParameterValues(String name) {
|
||||
return getParameterMap().get(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Enumeration<String> getParameterNames() {
|
||||
final Iterator<String> i = m.keySet().iterator();
|
||||
return new Enumeration<String>() {
|
||||
@Override
|
||||
public boolean hasMoreElements() {
|
||||
return i.hasNext();
|
||||
}
|
||||
@Override
|
||||
public String nextElement() {
|
||||
return i.next();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
@ -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<String, Object> m = new TreeMap<String, Object>();
|
||||
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));
|
||||
}
|
||||
|
||||
|
@ -59,7 +59,7 @@ public ContainerResponseFilter getResponseFilter() {
|
||||
}
|
||||
|
||||
/** Do the strings contain upper case letters? */
|
||||
private static boolean containsUpperCase(final Iterable<String> strings) {
|
||||
static boolean containsUpperCase(final Iterable<String> strings) {
|
||||
for(String s : strings) {
|
||||
for(int i = 0; i < s.length(); i++) {
|
||||
if (Character.isUpperCase(s.charAt(i))) {
|
||||
|
@ -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<DelegationTokenIdentifier> DT_SELECTOR
|
||||
= new AbstractDelegationTokenSelector<DelegationTokenIdentifier>(TOKEN_KIND) {};
|
||||
|
||||
private static final DelegationTokenRenewer<WebHdfsFileSystem> dtRenewer
|
||||
= new DelegationTokenRenewer<WebHdfsFileSystem>(WebHdfsFileSystem.class);
|
||||
static {
|
||||
dtRenewer.start();
|
||||
private static DelegationTokenRenewer<WebHdfsFileSystem> DT_RENEWER = null;
|
||||
|
||||
private static synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
|
||||
if (DT_RENEWER == null) {
|
||||
DT_RENEWER = new DelegationTokenRenewer<WebHdfsFileSystem>(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<DelegationTokenIdentifier> getDelegationToken(final String renewer
|
||||
final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN;
|
||||
final Map<?, ?> m = run(op, null, new RenewerParam(renewer));
|
||||
final Token<DelegationTokenIdentifier> token = JsonUtil.toDelegationToken(m);
|
||||
token.setService(new Text(getCanonicalServiceName()));
|
||||
SecurityUtil.setTokenService(token, nnAddr);
|
||||
return token;
|
||||
}
|
||||
|
||||
@ -590,23 +700,14 @@ public List<Token<?>> getDelegationTokens(final String renewer
|
||||
|
||||
@Override
|
||||
public Token<?> getRenewToken() {
|
||||
return renewToken;
|
||||
return delegationToken;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T extends TokenIdentifier> void setDelegationToken(
|
||||
final Token<T> 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<T>(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<DelegationTokenIdentifier> {
|
||||
private DtSelector() {
|
||||
super(TOKEN_KIND);
|
||||
}
|
||||
}
|
||||
|
||||
/** Delegation token renewer. */
|
||||
public static class DtRenewer extends TokenRenewer {
|
||||
@Override
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
@ -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<Exception> {
|
||||
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) {
|
||||
|
@ -24,15 +24,18 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
|
||||
/** 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;
|
||||
|
@ -17,6 +17,9 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.web.resources;
|
||||
|
||||
import javax.ws.rs.core.Response;
|
||||
|
||||
|
||||
/** Http operation parameter. */
|
||||
public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op>
|
||||
extends EnumParam<E> {
|
||||
@ -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<E> domain, final E value) {
|
||||
super(domain, value);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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, "");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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());
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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<RemoteBlockReader> {
|
||||
public RemoteBlockReader reader = null;
|
||||
private class MockGetBlockReader implements Answer<RemoteBlockReader2> {
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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");
|
||||
|
@ -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();
|
||||
|
@ -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();
|
||||
|
@ -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<DelegationTokenIdentifier> 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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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<Block> blockList = new ArrayList<Block>(MAX_BLOCKS);
|
||||
ArrayList<BlockInfo> blockInfoList = new ArrayList<BlockInfo>();
|
||||
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<BlockInfo> 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());
|
||||
}
|
||||
}
|
||||
}
|
@ -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<DatanodeDescriptor> 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<Block> list_p1 = new ArrayList<Block>();
|
||||
list_p1.add(block);
|
||||
|
||||
// list of lists for each priority
|
||||
List<List<Block>> list_all = new ArrayList<List<Block>>();
|
||||
list_all.add(new ArrayList<Block>()); // 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<DatanodeDescriptor, BlockTargetPair> repls =
|
||||
getAllPendingReplications();
|
||||
|
||||
LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> repls = getAllPendingReplications();
|
||||
assertEquals(1, repls.size());
|
||||
Entry<DatanodeDescriptor, BlockTargetPair> repl = repls.entries().iterator().next();
|
||||
Entry<DatanodeDescriptor, BlockTargetPair> 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;
|
||||
}
|
||||
|
||||
|
@ -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) {
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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));
|
||||
|
@ -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<FSNamesystem.CorruptFileBlockInfo> 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
|
||||
|
@ -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<ByteBuffer> 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);
|
||||
}
|
||||
}
|
@ -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<Integer> list = new ArrayList<Integer>();
|
||||
private final int NUM = 100;
|
||||
private LightWeightHashSet<Integer> 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<Integer>(initCapacity, maxF, minF);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyBasic() {
|
||||
LOG.info("Test empty basic");
|
||||
Iterator<Integer> 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<Integer> 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<Integer> 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<Integer> 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<Integer> 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<Integer> 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<Integer> 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<Integer> 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<Integer> 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<Integer>(1, maxF, minF);
|
||||
assertEquals(LightWeightHashSet.MINIMUM_CAPACITY, set.getCapacity());
|
||||
|
||||
// capacity not a power of two
|
||||
set = new LightWeightHashSet<Integer>(30, maxF, minF);
|
||||
assertEquals(Math.max(LightWeightHashSet.MINIMUM_CAPACITY, 32),
|
||||
set.getCapacity());
|
||||
|
||||
// capacity valid
|
||||
set = new LightWeightHashSet<Integer>(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<Integer> sub = new LinkedList<Integer>();
|
||||
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<Integer> sub2 = new LinkedList<Integer>();
|
||||
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");
|
||||
}
|
||||
|
||||
}
|
@ -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<Integer> list = new ArrayList<Integer>();
|
||||
private final int NUM = 100;
|
||||
private LightWeightLinkedSet<Integer> set;
|
||||
private Random rand;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
float maxF = LightWeightLinkedSet.DEFAULT_MAX_LOAD_FACTOR;
|
||||
float minF = LightWeightLinkedSet.DEFAUT_MIN_LOAD_FACTOR;
|
||||
int initCapacity = LightWeightLinkedSet.MINIMUM_CAPACITY;
|
||||
rand = new Random(System.currentTimeMillis());
|
||||
list.clear();
|
||||
for (int i = 0; i < NUM; i++) {
|
||||
list.add(rand.nextInt());
|
||||
}
|
||||
set = new LightWeightLinkedSet<Integer>(initCapacity, maxF, minF);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyBasic() {
|
||||
LOG.info("Test empty basic");
|
||||
Iterator<Integer> iter = set.iterator();
|
||||
// iterator should not have next
|
||||
assertFalse(iter.hasNext());
|
||||
assertEquals(0, set.size());
|
||||
assertTrue(set.isEmpty());
|
||||
|
||||
// poll should return nothing
|
||||
assertNull(set.pollFirst());
|
||||
assertEquals(0, set.pollAll().size());
|
||||
assertEquals(0, set.pollN(10).size());
|
||||
|
||||
LOG.info("Test empty - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneElementBasic() {
|
||||
LOG.info("Test one element basic");
|
||||
set.add(list.get(0));
|
||||
// set should be non-empty
|
||||
assertEquals(1, set.size());
|
||||
assertFalse(set.isEmpty());
|
||||
|
||||
// iterator should have next
|
||||
Iterator<Integer> 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<Integer> iter = set.iterator();
|
||||
int num = 0;
|
||||
while (iter.hasNext()) {
|
||||
assertEquals(list.get(num++), iter.next());
|
||||
}
|
||||
// check the number of element from the iterator
|
||||
assertEquals(list.size(), num);
|
||||
LOG.info("Test multi element basic - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveOne() {
|
||||
LOG.info("Test remove one");
|
||||
assertTrue(set.add(list.get(0)));
|
||||
assertEquals(1, set.size());
|
||||
|
||||
// remove from the head/tail
|
||||
assertTrue(set.remove(list.get(0)));
|
||||
assertEquals(0, set.size());
|
||||
|
||||
// check the iterator
|
||||
Iterator<Integer> iter = set.iterator();
|
||||
assertFalse(iter.hasNext());
|
||||
|
||||
// poll should return nothing
|
||||
assertNull(set.pollFirst());
|
||||
assertEquals(0, set.pollAll().size());
|
||||
assertEquals(0, set.pollN(10).size());
|
||||
|
||||
// add the element back to the set
|
||||
assertTrue(set.add(list.get(0)));
|
||||
assertEquals(1, set.size());
|
||||
|
||||
iter = set.iterator();
|
||||
assertTrue(iter.hasNext());
|
||||
LOG.info("Test remove one - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveMulti() {
|
||||
LOG.info("Test remove multi");
|
||||
for (Integer i : list) {
|
||||
assertTrue(set.add(i));
|
||||
}
|
||||
for (int i = 0; i < NUM / 2; i++) {
|
||||
assertTrue(set.remove(list.get(i)));
|
||||
}
|
||||
|
||||
// the deleted elements should not be there
|
||||
for (int i = 0; i < NUM / 2; i++) {
|
||||
assertFalse(set.contains(list.get(i)));
|
||||
}
|
||||
|
||||
// the rest should be there
|
||||
for (int i = NUM / 2; i < NUM; i++) {
|
||||
assertTrue(set.contains(list.get(i)));
|
||||
}
|
||||
|
||||
Iterator<Integer> iter = set.iterator();
|
||||
// the remaining elements should be in order
|
||||
int num = NUM / 2;
|
||||
while (iter.hasNext()) {
|
||||
assertEquals(list.get(num++), iter.next());
|
||||
}
|
||||
assertEquals(num, NUM);
|
||||
LOG.info("Test remove multi - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveAll() {
|
||||
LOG.info("Test remove all");
|
||||
for (Integer i : list) {
|
||||
assertTrue(set.add(i));
|
||||
}
|
||||
for (int i = 0; i < NUM; i++) {
|
||||
assertTrue(set.remove(list.get(i)));
|
||||
}
|
||||
// the deleted elements should not be there
|
||||
for (int i = 0; i < NUM; i++) {
|
||||
assertFalse(set.contains(list.get(i)));
|
||||
}
|
||||
|
||||
// iterator should not have next
|
||||
Iterator<Integer> iter = set.iterator();
|
||||
assertFalse(iter.hasNext());
|
||||
assertTrue(set.isEmpty());
|
||||
LOG.info("Test remove all - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPollOneElement() {
|
||||
LOG.info("Test poll one element");
|
||||
set.add(list.get(0));
|
||||
assertEquals(list.get(0), set.pollFirst());
|
||||
assertNull(set.pollFirst());
|
||||
LOG.info("Test poll one element - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPollMulti() {
|
||||
LOG.info("Test poll multi");
|
||||
for (Integer i : list) {
|
||||
assertTrue(set.add(i));
|
||||
}
|
||||
// remove half of the elements by polling
|
||||
for (int i = 0; i < NUM / 2; i++) {
|
||||
assertEquals(list.get(i), set.pollFirst());
|
||||
}
|
||||
assertEquals(NUM / 2, set.size());
|
||||
// the deleted elements should not be there
|
||||
for (int i = 0; i < NUM / 2; i++) {
|
||||
assertFalse(set.contains(list.get(i)));
|
||||
}
|
||||
// the rest should be there
|
||||
for (int i = NUM / 2; i < NUM; i++) {
|
||||
assertTrue(set.contains(list.get(i)));
|
||||
}
|
||||
Iterator<Integer> iter = set.iterator();
|
||||
// the remaining elements should be in order
|
||||
int num = NUM / 2;
|
||||
while (iter.hasNext()) {
|
||||
assertEquals(list.get(num++), iter.next());
|
||||
}
|
||||
assertEquals(num, NUM);
|
||||
|
||||
// add elements back
|
||||
for (int i = 0; i < NUM / 2; i++) {
|
||||
assertTrue(set.add(list.get(i)));
|
||||
}
|
||||
// order should be switched
|
||||
assertEquals(NUM, set.size());
|
||||
for (int i = NUM / 2; i < NUM; i++) {
|
||||
assertEquals(list.get(i), set.pollFirst());
|
||||
}
|
||||
for (int i = 0; i < NUM / 2; i++) {
|
||||
assertEquals(list.get(i), set.pollFirst());
|
||||
}
|
||||
assertEquals(0, set.size());
|
||||
assertTrue(set.isEmpty());
|
||||
LOG.info("Test poll multi - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPollAll() {
|
||||
LOG.info("Test poll all");
|
||||
for (Integer i : list) {
|
||||
assertTrue(set.add(i));
|
||||
}
|
||||
// remove all elements by polling
|
||||
while (set.pollFirst() != null);
|
||||
assertEquals(0, set.size());
|
||||
assertTrue(set.isEmpty());
|
||||
|
||||
// the deleted elements should not be there
|
||||
for (int i = 0; i < NUM; i++) {
|
||||
assertFalse(set.contains(list.get(i)));
|
||||
}
|
||||
|
||||
Iterator<Integer> iter = set.iterator();
|
||||
assertFalse(iter.hasNext());
|
||||
LOG.info("Test poll all - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPollNOne() {
|
||||
LOG.info("Test pollN one");
|
||||
set.add(list.get(0));
|
||||
List<Integer> l = set.pollN(10);
|
||||
assertEquals(1, l.size());
|
||||
assertEquals(list.get(0), l.get(0));
|
||||
LOG.info("Test pollN one - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPollNMulti() {
|
||||
LOG.info("Test pollN multi");
|
||||
|
||||
// use addAll
|
||||
set.addAll(list);
|
||||
|
||||
// poll existing elements
|
||||
List<Integer> l = set.pollN(10);
|
||||
assertEquals(10, l.size());
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
assertEquals(list.get(i), l.get(i));
|
||||
}
|
||||
|
||||
// poll more elements than present
|
||||
l = set.pollN(1000);
|
||||
assertEquals(NUM - 10, l.size());
|
||||
|
||||
// check the order
|
||||
for (int i = 10; i < NUM; i++) {
|
||||
assertEquals(list.get(i), l.get(i - 10));
|
||||
}
|
||||
// set is empty
|
||||
assertTrue(set.isEmpty());
|
||||
assertEquals(0, set.size());
|
||||
|
||||
LOG.info("Test pollN multi - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClear() {
|
||||
LOG.info("Test clear");
|
||||
// use addAll
|
||||
set.addAll(list);
|
||||
assertEquals(NUM, set.size());
|
||||
assertFalse(set.isEmpty());
|
||||
|
||||
// clear the set
|
||||
set.clear();
|
||||
assertEquals(0, set.size());
|
||||
assertTrue(set.isEmpty());
|
||||
|
||||
// poll should return an empty list
|
||||
assertEquals(0, set.pollAll().size());
|
||||
assertEquals(0, set.pollN(10).size());
|
||||
assertNull(set.pollFirst());
|
||||
|
||||
// iterator should be empty
|
||||
Iterator<Integer> iter = set.iterator();
|
||||
assertFalse(iter.hasNext());
|
||||
|
||||
LOG.info("Test clear - DONE");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOther() {
|
||||
LOG.info("Test other");
|
||||
assertTrue(set.addAll(list));
|
||||
// to array
|
||||
Integer[] array = set.toArray(new Integer[0]);
|
||||
assertEquals(NUM, array.length);
|
||||
for (int i = 0; i < array.length; i++) {
|
||||
assertTrue(list.contains(array[i]));
|
||||
}
|
||||
assertEquals(NUM, set.size());
|
||||
|
||||
// to array
|
||||
Object[] array2 = set.toArray();
|
||||
assertEquals(NUM, array2.length);
|
||||
for (int i = 0; i < array2.length; i++) {
|
||||
assertTrue(list.contains((Integer) array2[i]));
|
||||
}
|
||||
LOG.info("Test capacity - DONE");
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,137 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.web;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hdfs.TestByteRangeInputStream.MockHttpURLConnection;
|
||||
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem.OffsetUrlInputStream;
|
||||
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem.OffsetUrlOpener;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestOffsetUrlInputStream {
|
||||
@Test
|
||||
public void testRemoveOffset() throws IOException {
|
||||
{ //no offset
|
||||
String s = "http://test/Abc?Length=99";
|
||||
assertEquals(s, WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString());
|
||||
}
|
||||
|
||||
{ //no parameters
|
||||
String s = "http://test/Abc";
|
||||
assertEquals(s, WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString());
|
||||
}
|
||||
|
||||
{ //offset as first parameter
|
||||
String s = "http://test/Abc?offset=10&Length=99";
|
||||
assertEquals("http://test/Abc?Length=99",
|
||||
WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString());
|
||||
}
|
||||
|
||||
{ //offset as second parameter
|
||||
String s = "http://test/Abc?op=read&OFFset=10&Length=99";
|
||||
assertEquals("http://test/Abc?op=read&Length=99",
|
||||
WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString());
|
||||
}
|
||||
|
||||
{ //offset as last parameter
|
||||
String s = "http://test/Abc?Length=99&offset=10";
|
||||
assertEquals("http://test/Abc?Length=99",
|
||||
WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString());
|
||||
}
|
||||
|
||||
{ //offset as the only parameter
|
||||
String s = "http://test/Abc?offset=10";
|
||||
assertEquals("http://test/Abc",
|
||||
WebHdfsFileSystem.removeOffsetParam(new URL(s)).toString());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testByteRange() throws Exception {
|
||||
final Configuration conf = new Configuration();
|
||||
final String uri = WebHdfsFileSystem.SCHEME + "://localhost:50070/";
|
||||
final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)FileSystem.get(new URI(uri), conf);
|
||||
|
||||
OffsetUrlOpener ospy = spy(webhdfs.new OffsetUrlOpener(new URL("http://test/")));
|
||||
doReturn(new MockHttpURLConnection(ospy.getURL())).when(ospy)
|
||||
.openConnection();
|
||||
OffsetUrlOpener rspy = spy(webhdfs.new OffsetUrlOpener((URL) null));
|
||||
doReturn(new MockHttpURLConnection(rspy.getURL())).when(rspy)
|
||||
.openConnection();
|
||||
final OffsetUrlInputStream is = new OffsetUrlInputStream(ospy, rspy);
|
||||
|
||||
assertEquals("getPos wrong", 0, is.getPos());
|
||||
|
||||
is.read();
|
||||
|
||||
assertNull("Initial call made incorrectly (Range Check)", ospy
|
||||
.openConnection().getRequestProperty("Range"));
|
||||
|
||||
assertEquals("getPos should be 1 after reading one byte", 1, is.getPos());
|
||||
|
||||
is.read();
|
||||
|
||||
assertEquals("getPos should be 2 after reading two bytes", 2, is.getPos());
|
||||
|
||||
// No additional connections should have been made (no seek)
|
||||
|
||||
rspy.setURL(new URL("http://resolvedurl/"));
|
||||
|
||||
is.seek(100);
|
||||
is.read();
|
||||
|
||||
assertEquals("getPos should be 101 after reading one byte", 101,
|
||||
is.getPos());
|
||||
|
||||
verify(rspy, times(1)).openConnection();
|
||||
|
||||
is.seek(101);
|
||||
is.read();
|
||||
|
||||
verify(rspy, times(1)).openConnection();
|
||||
|
||||
// Seek to 101 should not result in another request"
|
||||
|
||||
is.seek(2500);
|
||||
is.read();
|
||||
|
||||
((MockHttpURLConnection) rspy.openConnection()).setResponseCode(206);
|
||||
is.seek(0);
|
||||
|
||||
try {
|
||||
is.read();
|
||||
fail("Exception should be thrown when 206 response is given "
|
||||
+ "but 200 is expected");
|
||||
} catch (IOException e) {
|
||||
WebHdfsFileSystem.LOG.info(e.toString());
|
||||
}
|
||||
}
|
||||
}
|
@ -23,9 +23,8 @@
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
|
||||
@ -34,12 +33,12 @@
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileSystemContractBaseTest;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.web.resources.DoAsParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.PutOpParam;
|
||||
@ -51,6 +50,8 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
|
||||
private static final Configuration conf = new Configuration();
|
||||
private static final MiniDFSCluster cluster;
|
||||
private String defaultWorkingDirectory;
|
||||
|
||||
private UserGroupInformation ugi;
|
||||
|
||||
static {
|
||||
conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
|
||||
@ -68,20 +69,11 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
|
||||
|
||||
@Override
|
||||
protected void setUp() throws Exception {
|
||||
final String uri = WebHdfsFileSystem.SCHEME + "://"
|
||||
+ conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
|
||||
|
||||
//get file system as a non-superuser
|
||||
final UserGroupInformation current = UserGroupInformation.getCurrentUser();
|
||||
final UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
|
||||
ugi = UserGroupInformation.createUserForTesting(
|
||||
current.getShortUserName() + "x", new String[]{"user"});
|
||||
fs = ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
|
||||
@Override
|
||||
public FileSystem run() throws Exception {
|
||||
return FileSystem.get(new URI(uri), conf);
|
||||
}
|
||||
});
|
||||
|
||||
fs = WebHdfsTestUtil.getWebHdfsFileSystemAs(ugi, conf);
|
||||
defaultWorkingDirectory = fs.getWorkingDirectory().toUri().getPath();
|
||||
}
|
||||
|
||||
@ -263,9 +255,29 @@ public void testRootDir() throws IOException {
|
||||
|
||||
public void testResponseCode() throws IOException {
|
||||
final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)fs;
|
||||
final Path root = new Path("/");
|
||||
final Path dir = new Path("/test/testUrl");
|
||||
assertTrue(webhdfs.mkdirs(dir));
|
||||
|
||||
{//test GETHOMEDIRECTORY
|
||||
final URL url = webhdfs.toUrl(GetOpParam.Op.GETHOMEDIRECTORY, root);
|
||||
final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
|
||||
final Map<?, ?> m = WebHdfsTestUtil.connectAndGetJson(
|
||||
conn, HttpServletResponse.SC_OK);
|
||||
assertEquals(WebHdfsFileSystem.getHomeDirectoryString(ugi),
|
||||
m.get(Path.class.getSimpleName()));
|
||||
conn.disconnect();
|
||||
}
|
||||
|
||||
{//test GETHOMEDIRECTORY with unauthorized doAs
|
||||
final URL url = webhdfs.toUrl(GetOpParam.Op.GETHOMEDIRECTORY, root,
|
||||
new DoAsParam(ugi.getShortUserName() + "proxy"));
|
||||
final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
|
||||
conn.connect();
|
||||
assertEquals(HttpServletResponse.SC_UNAUTHORIZED, conn.getResponseCode());
|
||||
conn.disconnect();
|
||||
}
|
||||
|
||||
{//test set owner with empty parameters
|
||||
final URL url = webhdfs.toUrl(PutOpParam.Op.SETOWNER, dir);
|
||||
final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
|
||||
@ -280,7 +292,7 @@ public void testResponseCode() throws IOException {
|
||||
final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
|
||||
conn.setRequestMethod(op.getType().toString());
|
||||
conn.connect();
|
||||
assertEquals(HttpServletResponse.SC_FORBIDDEN, conn.getResponseCode());
|
||||
assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
|
||||
|
||||
assertFalse(webhdfs.setReplication(dir, (short)1));
|
||||
conn.disconnect();
|
||||
|
@ -75,7 +75,7 @@ public void testDelegationTokenInUrl() throws IOException {
|
||||
+ "&token=" + tokenString, renewTokenUrl.getQuery());
|
||||
Token<DelegationTokenIdentifier> delegationToken = new Token<DelegationTokenIdentifier>(
|
||||
token);
|
||||
delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
|
||||
delegationToken.setKind(WebHdfsFileSystem.TOKEN_KIND);
|
||||
Assert.assertEquals(
|
||||
generateUrlQueryPrefix(PutOpParam.Op.CANCELDELEGATIONTOKEN,
|
||||
ugi.getUserName())
|
||||
|
@ -0,0 +1,90 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.web;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.net.URL;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.Param;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.junit.Assert;
|
||||
|
||||
public class WebHdfsTestUtil {
|
||||
public static final Log LOG = LogFactory.getLog(WebHdfsTestUtil.class);
|
||||
|
||||
public static WebHdfsFileSystem getWebHdfsFileSystem(final Configuration conf
|
||||
) throws IOException, URISyntaxException {
|
||||
final String uri = WebHdfsFileSystem.SCHEME + "://"
|
||||
+ conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
|
||||
return (WebHdfsFileSystem)FileSystem.get(new URI(uri), conf);
|
||||
}
|
||||
|
||||
public static WebHdfsFileSystem getWebHdfsFileSystemAs(
|
||||
final UserGroupInformation ugi, final Configuration conf
|
||||
) throws IOException, URISyntaxException, InterruptedException {
|
||||
return ugi.doAs(new PrivilegedExceptionAction<WebHdfsFileSystem>() {
|
||||
@Override
|
||||
public WebHdfsFileSystem run() throws Exception {
|
||||
return getWebHdfsFileSystem(conf);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static URL toUrl(final WebHdfsFileSystem webhdfs,
|
||||
final HttpOpParam.Op op, final Path fspath,
|
||||
final Param<?,?>... parameters) throws IOException {
|
||||
final URL url = webhdfs.toUrl(op, fspath, parameters);
|
||||
WebHdfsTestUtil.LOG.info("url=" + url);
|
||||
return url;
|
||||
}
|
||||
|
||||
public static Map<?, ?> connectAndGetJson(final HttpURLConnection conn,
|
||||
final int expectedResponseCode) throws IOException {
|
||||
conn.connect();
|
||||
Assert.assertEquals(expectedResponseCode, conn.getResponseCode());
|
||||
return WebHdfsFileSystem.jsonParse(conn.getInputStream());
|
||||
}
|
||||
|
||||
public static HttpURLConnection twoStepWrite(HttpURLConnection conn,
|
||||
final HttpOpParam.Op op) throws IOException {
|
||||
conn.setRequestMethod(op.getType().toString());
|
||||
conn = WebHdfsFileSystem.twoStepWrite(conn, op);
|
||||
conn.setDoOutput(true);
|
||||
conn.connect();
|
||||
return conn;
|
||||
}
|
||||
|
||||
public static FSDataOutputStream write(final WebHdfsFileSystem webhdfs,
|
||||
final HttpOpParam.Op op, final HttpURLConnection conn,
|
||||
final int bufferSize) throws IOException {
|
||||
return webhdfs.write(op, conn, bufferSize);
|
||||
}
|
||||
}
|
@ -10,6 +10,12 @@ Trunk (unreleased changes)
|
||||
(Plamen Jeliazkov via shv)
|
||||
|
||||
IMPROVEMENTS
|
||||
MAPREDUCE-3375. [Gridmix] Memory Emulation system tests.
|
||||
(Vinay Thota via amarrk)
|
||||
|
||||
MAPREDUCE-2733. [Gridmix] Gridmix3 cpu emulation system tests.
|
||||
(Vinay Thota via amarrk)
|
||||
|
||||
MAPREDUCE-3008. Improvements to cumulative CPU emulation for short running
|
||||
tasks in Gridmix. (amarrk)
|
||||
|
||||
@ -29,6 +35,8 @@ Trunk (unreleased changes)
|
||||
uri with no authority. (John George via jitendra)
|
||||
|
||||
BUG FIXES
|
||||
MAPREDUCE-3346. [Rumen] LoggedTaskAttempt#getHostName() returns null.
|
||||
(amarrk)
|
||||
|
||||
MAPREDUCE-2950. [Gridmix] TestUserResolve fails in trunk.
|
||||
(Ravi Gummadi via amarrk)
|
||||
@ -50,6 +58,80 @@ Trunk (unreleased changes)
|
||||
MAPREDUCE-3014. Rename and invert logic of '-cbuild' profile to 'native' and off
|
||||
by default. (tucu)
|
||||
|
||||
Release 0.23.1 - Unreleased
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
MAPREDUCE-3297. Moved log related components into yarn-common so that
|
||||
HistoryServer and clients can use them without depending on the
|
||||
yarn-server-nodemanager module. (Siddharth Seth via vinodkv)
|
||||
|
||||
MAPREDUCE-3336. Replaced guice internal.Preconditions api usage with the
|
||||
public Preconditions API. (Thomas Graves via vinodkv)
|
||||
|
||||
MAPREDUCE-3280. Removed the unnecessary job user-name configuration in
|
||||
mapred-site.xml. (vinodkv)
|
||||
|
||||
MAPREDUCE-3370. Fixed MiniMRYarnCluster and related tests to not use
|
||||
a hard-coded path for the mr-app jar. (Ahmed Radwan via vinodkv)
|
||||
|
||||
MAPREDUCE-3325. Improvements to CapacityScheduler doc. (Thomas Graves
|
||||
via mahadev)
|
||||
|
||||
MAPREDUCE-3341. Enhance logging of initalized queue limit values.
|
||||
(Anupam Seth via mahadev)
|
||||
|
||||
MAPREDUCE-3243. Invalid tracking URL for streaming jobs (Jonathan Eagles
|
||||
via mahadev)
|
||||
|
||||
MAPREDUCE-3331. Improvement to single node cluster setup documentation for
|
||||
0.23 (Anupam Seth via mahadev)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
||||
MAPREDUCE-3221. Reenabled the previously ignored test in TestSubmitJob
|
||||
and fixed bugs in it. (Devaraj K via vinodkv)
|
||||
|
||||
MAPREDUCE-3215. Reenabled and fixed bugs in the failing test
|
||||
TestNoJobSetupCleanup. (Hitesh Shah via vinodkv)
|
||||
|
||||
MAPREDUCE-3219. Reenabled and fixed bugs in the failing test
|
||||
TestDelegationToken. (Hitesh Shah via vinodkv)
|
||||
|
||||
MAPREDUCE-3217. Reenabled and fixed bugs in the failing ant test
|
||||
TestAuditLogger. (Devaraj K via vinodkv)
|
||||
|
||||
MAPREDUCE-3291. App fail to launch due to delegation token not
|
||||
found in cache (Robert Evans via mahadev)
|
||||
|
||||
MAPREDUCE-3344. o.a.h.mapreduce.Reducer since 0.21 blindly casts to
|
||||
ReduceContext.ValueIterator. (Brock Noland via tomwhite)
|
||||
|
||||
MAPREDUCE-3342. Fixed JobHistoryServer to also show the job's queue
|
||||
name. (Jonathan Eagles via vinodkv)
|
||||
|
||||
MAPREDUCE-3345. Fixed a race condition in ResourceManager that was causing
|
||||
TestContainerManagerSecurity to fail sometimes. (Hitesh Shah via vinodkv)
|
||||
|
||||
MAPREDUCE-3333. Fixed bugs in ContainerLauncher of MR AppMaster due to
|
||||
which per-container connections to NodeManager were lingering long enough
|
||||
to hit the ulimits on number of processes. (vinodkv)
|
||||
|
||||
MAPREDUCE-3392. Fixed Cluster's getDelegationToken's API to return null
|
||||
when there isn't a supported token. (John George via vinodkv)
|
||||
|
||||
MAPREDUCE-3379. Fixed LocalResourceTracker in NodeManager to remove deleted
|
||||
cache entries correctly. (Siddharth Seth via vinodkv)
|
||||
|
||||
MAPREDUCE-3324. Not All HttpServer tools links (stacks,logs,config,metrics) are
|
||||
accessible through all UI servers (Jonathan Eagles via mahadev)
|
||||
|
||||
Release 0.23.0 - 2011-11-01
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
@ -1912,6 +1994,8 @@ Release 0.23.0 - 2011-11-01
|
||||
MAPREDUCE-3332. contrib/raid compile breaks due to changes in hdfs/protocol/datatransfer/
|
||||
Sender#writeBlock related to checksum handling (Hitesh Shah via mahadev)
|
||||
|
||||
MAPREDUCE-3337. Added missing license headers. (acmurthy)
|
||||
|
||||
Release 0.22.0 - Unreleased
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
@ -2506,6 +2590,8 @@ Release 0.22.0 - Unreleased
|
||||
MAPREDUCE-2531. Fixed jobcontrol to downgrade JobID. (Robert Evans via
|
||||
acmurthy)
|
||||
|
||||
MAPREDUCE-3139. SlivePartitioner generates negative partitions. (jghoman)
|
||||
|
||||
Release 0.21.1 - Unreleased
|
||||
|
||||
NEW FEATURES
|
||||
|
@ -130,8 +130,8 @@ public TaskUmbilicalProtocol run() throws Exception {
|
||||
|
||||
// Initiate Java VM metrics
|
||||
JvmMetrics.initSingleton(jvmId.toString(), job.getSessionId());
|
||||
LOG.debug("Remote user: " + job.get("user.name"));
|
||||
childUGI = UserGroupInformation.createRemoteUser(job.get("user.name"));
|
||||
childUGI = UserGroupInformation.createRemoteUser(System
|
||||
.getenv(ApplicationConstants.Environment.USER.toString()));
|
||||
// Add tokens to new user so that it may execute its task correctly.
|
||||
for(Token<?> token : UserGroupInformation.getCurrentUser().getTokens()) {
|
||||
childUGI.addToken(token);
|
||||
|
@ -400,6 +400,7 @@ protected void handleEvent(JobHistoryEvent event) {
|
||||
JobSubmittedEvent jobSubmittedEvent =
|
||||
(JobSubmittedEvent) event.getHistoryEvent();
|
||||
mi.getJobIndexInfo().setSubmitTime(jobSubmittedEvent.getSubmitTime());
|
||||
mi.getJobIndexInfo().setQueueName(jobSubmittedEvent.getJobQueueName());
|
||||
}
|
||||
|
||||
// If this is JobFinishedEvent, close the writer and setup the job-index
|
||||
|
@ -55,6 +55,7 @@ public interface Job {
|
||||
int getCompletedReduces();
|
||||
boolean isUber();
|
||||
String getUserName();
|
||||
String getQueueName();
|
||||
|
||||
/**
|
||||
* @return a path to where the config file for this job is located.
|
||||
|
@ -142,6 +142,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
||||
private final EventHandler eventHandler;
|
||||
private final MRAppMetrics metrics;
|
||||
private final String userName;
|
||||
private final String queueName;
|
||||
private final long appSubmitTime;
|
||||
|
||||
private boolean lazyTasksCopyNeeded = false;
|
||||
@ -375,6 +376,7 @@ public JobImpl(JobId jobId, ApplicationAttemptId applicationAttemptId,
|
||||
this.completedTasksFromPreviousRun = completedTasksFromPreviousRun;
|
||||
this.amInfos = amInfos;
|
||||
this.userName = userName;
|
||||
this.queueName = conf.get(MRJobConfig.QUEUE_NAME, "default");
|
||||
this.appSubmitTime = appSubmitTime;
|
||||
this.oldJobId = TypeConverter.fromYarn(jobId);
|
||||
this.newApiCommitter = newApiCommitter;
|
||||
@ -766,6 +768,11 @@ public String getUserName() {
|
||||
return userName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getQueueName() {
|
||||
return queueName;
|
||||
}
|
||||
|
||||
/*
|
||||
* (non-Javadoc)
|
||||
* @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile()
|
||||
@ -829,7 +836,7 @@ public JobState transition(JobImpl job, JobEvent event) {
|
||||
job.conf.get(MRJobConfig.USER_NAME, "mapred"),
|
||||
job.appSubmitTime,
|
||||
job.remoteJobConfFile.toString(),
|
||||
job.jobACLs, job.conf.get(MRJobConfig.QUEUE_NAME, "default"));
|
||||
job.jobACLs, job.queueName);
|
||||
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse));
|
||||
//TODO JH Verify jobACLs, UserName via UGI?
|
||||
|
||||
|
@ -36,5 +36,5 @@ enum EventType {
|
||||
/**
|
||||
* Maximum of 1 minute timeout for a Node to react to the command
|
||||
*/
|
||||
static final int DEFAULT_NM__COMMAND_TIMEOUT = 60000;
|
||||
static final int DEFAULT_NM_COMMAND_TIMEOUT = 60000;
|
||||
}
|
||||
|
@ -21,9 +21,7 @@
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.Timer;
|
||||
import java.util.TimerTask;
|
||||
@ -36,6 +34,7 @@
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.ShuffleHandler;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
@ -59,11 +58,10 @@
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerToken;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.service.AbstractService;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
@ -84,29 +82,28 @@ public class ContainerLauncherImpl extends AbstractService implements
|
||||
private Thread eventHandlingThread;
|
||||
private BlockingQueue<ContainerLauncherEvent> eventQueue =
|
||||
new LinkedBlockingQueue<ContainerLauncherEvent>();
|
||||
private RecordFactory recordFactory;
|
||||
YarnRPC rpc;
|
||||
|
||||
// To track numNodes.
|
||||
Set<String> allNodes = new HashSet<String>();
|
||||
|
||||
// have a cache/map of proxies so as to avoid creating multiple RPC
|
||||
// client connection objects for the same container.
|
||||
private Map<ContainerId, ContainerManager> clientCache
|
||||
= new HashMap<ContainerId, ContainerManager>();
|
||||
|
||||
public ContainerLauncherImpl(AppContext context) {
|
||||
super(ContainerLauncherImpl.class.getName());
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void init(Configuration conf) {
|
||||
this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
|
||||
public synchronized void init(Configuration config) {
|
||||
Configuration conf = new Configuration(config);
|
||||
conf.setInt(
|
||||
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
|
||||
0);
|
||||
this.limitOnPoolSize = conf.getInt(
|
||||
MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT,
|
||||
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
|
||||
this.nmTimeOut = conf.getInt(ContainerLauncher.MR_AM_NM_COMMAND_TIMEOUT,
|
||||
ContainerLauncher.DEFAULT_NM__COMMAND_TIMEOUT);
|
||||
ContainerLauncher.DEFAULT_NM_COMMAND_TIMEOUT);
|
||||
this.rpc = YarnRPC.create(conf);
|
||||
super.init(conf);
|
||||
}
|
||||
|
||||
@ -180,37 +177,28 @@ protected ContainerManager getCMProxy(ContainerId containerID,
|
||||
|
||||
UserGroupInformation user = UserGroupInformation.getCurrentUser();
|
||||
|
||||
synchronized (this.clientCache) {
|
||||
this.allNodes.add(containerManagerBindAddr);
|
||||
|
||||
if (this.clientCache.containsKey(containerID)) {
|
||||
return this.clientCache.get(containerID);
|
||||
}
|
||||
|
||||
this.allNodes.add(containerManagerBindAddr);
|
||||
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
Token<ContainerTokenIdentifier> token = new Token<ContainerTokenIdentifier>(
|
||||
containerToken.getIdentifier().array(), containerToken
|
||||
.getPassword().array(), new Text(containerToken.getKind()),
|
||||
new Text(containerToken.getService()));
|
||||
// the user in createRemoteUser in this context has to be ContainerID
|
||||
user = UserGroupInformation.createRemoteUser(containerID.toString());
|
||||
user.addToken(token);
|
||||
}
|
||||
|
||||
ContainerManager proxy = user
|
||||
.doAs(new PrivilegedAction<ContainerManager>() {
|
||||
@Override
|
||||
public ContainerManager run() {
|
||||
YarnRPC rpc = YarnRPC.create(getConfig());
|
||||
return (ContainerManager) rpc.getProxy(ContainerManager.class,
|
||||
NetUtils.createSocketAddr(containerManagerBindAddr),
|
||||
getConfig());
|
||||
}
|
||||
});
|
||||
this.clientCache.put(containerID, proxy);
|
||||
return proxy;
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
Token<ContainerTokenIdentifier> token = new Token<ContainerTokenIdentifier>(
|
||||
containerToken.getIdentifier().array(), containerToken
|
||||
.getPassword().array(), new Text(containerToken.getKind()),
|
||||
new Text(containerToken.getService()));
|
||||
// the user in createRemoteUser in this context has to be ContainerID
|
||||
user = UserGroupInformation.createRemoteUser(containerID.toString());
|
||||
user.addToken(token);
|
||||
}
|
||||
|
||||
ContainerManager proxy = user
|
||||
.doAs(new PrivilegedAction<ContainerManager>() {
|
||||
@Override
|
||||
public ContainerManager run() {
|
||||
return (ContainerManager) rpc.getProxy(ContainerManager.class,
|
||||
NetUtils.createSocketAddr(containerManagerBindAddr),
|
||||
getConfig());
|
||||
}
|
||||
});
|
||||
return proxy;
|
||||
}
|
||||
|
||||
private static class CommandTimer extends TimerTask {
|
||||
@ -257,6 +245,8 @@ public void run() {
|
||||
|
||||
Timer timer = new Timer(true);
|
||||
|
||||
ContainerManager proxy = null;
|
||||
|
||||
switch(event.getType()) {
|
||||
|
||||
case CONTAINER_REMOTE_LAUNCH:
|
||||
@ -267,8 +257,8 @@ public void run() {
|
||||
timer.schedule(new CommandTimer(Thread.currentThread(), event),
|
||||
nmTimeOut);
|
||||
|
||||
ContainerManager proxy = getCMProxy(containerID,
|
||||
containerManagerBindAddr, containerToken);
|
||||
proxy = getCMProxy(containerID, containerManagerBindAddr,
|
||||
containerToken);
|
||||
|
||||
// Interruped during getProxy, but that didn't throw exception
|
||||
if (Thread.currentThread().isInterrupted()) {
|
||||
@ -284,8 +274,8 @@ public void run() {
|
||||
launchEvent.getContainer();
|
||||
|
||||
// Now launch the actual container
|
||||
StartContainerRequest startRequest = recordFactory
|
||||
.newRecordInstance(StartContainerRequest.class);
|
||||
StartContainerRequest startRequest = Records
|
||||
.newRecord(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
StartContainerResponse response = proxy.startContainer(startRequest);
|
||||
|
||||
@ -324,6 +314,7 @@ public void run() {
|
||||
sendContainerLaunchFailedMsg(taskAttemptID, message);
|
||||
} finally {
|
||||
timer.cancel();
|
||||
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
|
||||
}
|
||||
|
||||
break;
|
||||
@ -343,8 +334,8 @@ public void run() {
|
||||
timer.schedule(new CommandTimer(Thread.currentThread(), event),
|
||||
nmTimeOut);
|
||||
|
||||
ContainerManager proxy = getCMProxy(containerID,
|
||||
containerManagerBindAddr, containerToken);
|
||||
proxy = getCMProxy(containerID, containerManagerBindAddr,
|
||||
containerToken);
|
||||
|
||||
if (Thread.currentThread().isInterrupted()) {
|
||||
// The timer cancelled the command in the mean while. No need to
|
||||
@ -356,8 +347,8 @@ public void run() {
|
||||
// TODO:check whether container is launched
|
||||
|
||||
// kill the remote container if already launched
|
||||
StopContainerRequest stopRequest = recordFactory
|
||||
.newRecordInstance(StopContainerRequest.class);
|
||||
StopContainerRequest stopRequest = Records
|
||||
.newRecord(StopContainerRequest.class);
|
||||
stopRequest.setContainerId(event.getContainerID());
|
||||
proxy.stopContainer(stopRequest);
|
||||
}
|
||||
@ -373,6 +364,7 @@ public void run() {
|
||||
LOG.warn(message);
|
||||
} finally {
|
||||
timer.cancel();
|
||||
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig());
|
||||
}
|
||||
|
||||
// after killing, send killed event to taskattempt
|
||||
|
@ -492,6 +492,11 @@ public String getUserName() {
|
||||
return "mock";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getQueueName() {
|
||||
return "mockqueue";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getConfFile() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
|
@ -439,6 +439,11 @@ public String getName() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getQueueName() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getTotalMaps() {
|
||||
return mapTasks.size();
|
||||
|
@ -1,3 +1,20 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.mapreduce.v2.app.job.impl;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
@ -23,6 +23,9 @@
|
||||
import java.net.URLDecoder;
|
||||
import java.net.URLEncoder;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import org.apache.hadoop.mapreduce.JobID;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
@ -35,6 +38,10 @@ public class FileNameIndexUtils {
|
||||
static final String DELIMITER = "-";
|
||||
static final String DELIMITER_ESCAPE = "%2D";
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(FileNameIndexUtils.class);
|
||||
|
||||
// Job history file names need to be backwards compatible
|
||||
// Only append new elements to the end of this list
|
||||
private static final int JOB_ID_INDEX = 0;
|
||||
private static final int SUBMIT_TIME_INDEX = 1;
|
||||
private static final int USER_INDEX = 2;
|
||||
@ -43,7 +50,7 @@ public class FileNameIndexUtils {
|
||||
private static final int NUM_MAPS_INDEX = 5;
|
||||
private static final int NUM_REDUCES_INDEX = 6;
|
||||
private static final int JOB_STATUS_INDEX = 7;
|
||||
private static final int MAX_INDEX = JOB_STATUS_INDEX;
|
||||
private static final int QUEUE_NAME_INDEX = 8;
|
||||
|
||||
/**
|
||||
* Constructs the job history file name from the JobIndexInfo.
|
||||
@ -83,7 +90,11 @@ public static String getDoneFileName(JobIndexInfo indexInfo) throws IOException
|
||||
|
||||
//JobStatus
|
||||
sb.append(indexInfo.getJobStatus());
|
||||
sb.append(DELIMITER);
|
||||
|
||||
//QueueName
|
||||
sb.append(indexInfo.getQueueName());
|
||||
|
||||
sb.append(JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION);
|
||||
return encodeJobHistoryFileName(sb.toString());
|
||||
}
|
||||
@ -100,27 +111,60 @@ public static JobIndexInfo getIndexInfo(String jhFileName) throws IOException {
|
||||
JobIndexInfo indexInfo = new JobIndexInfo();
|
||||
|
||||
String[] jobDetails = fileName.split(DELIMITER);
|
||||
if (jobDetails.length != MAX_INDEX +1) {
|
||||
throw new IOException("Failed to parse file: [" + jhFileName + "]. Expected " + (MAX_INDEX + 1) + "parts.");
|
||||
}
|
||||
|
||||
JobID oldJobId = JobID.forName(decodeJobHistoryFileName(jobDetails[JOB_ID_INDEX]));
|
||||
JobId jobId = TypeConverter.toYarn(oldJobId);
|
||||
indexInfo.setJobId(jobId);
|
||||
//TODO Catch NumberFormatException - Do not fail if there's only a few fields missing.
|
||||
indexInfo.setSubmitTime(Long.parseLong(decodeJobHistoryFileName(jobDetails[SUBMIT_TIME_INDEX])));
|
||||
|
||||
indexInfo.setUser(decodeJobHistoryFileName(jobDetails[USER_INDEX]));
|
||||
|
||||
indexInfo.setJobName(decodeJobHistoryFileName(jobDetails[JOB_NAME_INDEX]));
|
||||
|
||||
indexInfo.setFinishTime(Long.parseLong(decodeJobHistoryFileName(jobDetails[FINISH_TIME_INDEX])));
|
||||
|
||||
indexInfo.setNumMaps(Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_MAPS_INDEX])));
|
||||
|
||||
indexInfo.setNumReduces(Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_REDUCES_INDEX])));
|
||||
|
||||
indexInfo.setJobStatus(decodeJobHistoryFileName(jobDetails[JOB_STATUS_INDEX]));
|
||||
|
||||
// Do not fail if there are some minor parse errors
|
||||
try {
|
||||
try {
|
||||
indexInfo.setSubmitTime(
|
||||
Long.parseLong(decodeJobHistoryFileName(jobDetails[SUBMIT_TIME_INDEX])));
|
||||
} catch (NumberFormatException e) {
|
||||
LOG.warn("Unable to parse submit time from job history file "
|
||||
+ jhFileName + " : " + e);
|
||||
}
|
||||
|
||||
indexInfo.setUser(
|
||||
decodeJobHistoryFileName(jobDetails[USER_INDEX]));
|
||||
|
||||
indexInfo.setJobName(
|
||||
decodeJobHistoryFileName(jobDetails[JOB_NAME_INDEX]));
|
||||
|
||||
try {
|
||||
indexInfo.setFinishTime(
|
||||
Long.parseLong(decodeJobHistoryFileName(jobDetails[FINISH_TIME_INDEX])));
|
||||
} catch (NumberFormatException e) {
|
||||
LOG.warn("Unable to parse finish time from job history file "
|
||||
+ jhFileName + " : " + e);
|
||||
}
|
||||
|
||||
try {
|
||||
indexInfo.setNumMaps(
|
||||
Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_MAPS_INDEX])));
|
||||
} catch (NumberFormatException e) {
|
||||
LOG.warn("Unable to parse num maps from job history file "
|
||||
+ jhFileName + " : " + e);
|
||||
}
|
||||
|
||||
try {
|
||||
indexInfo.setNumReduces(
|
||||
Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_REDUCES_INDEX])));
|
||||
} catch (NumberFormatException e) {
|
||||
LOG.warn("Unable to parse num reduces from job history file "
|
||||
+ jhFileName + " : " + e);
|
||||
}
|
||||
|
||||
indexInfo.setJobStatus(
|
||||
decodeJobHistoryFileName(jobDetails[JOB_STATUS_INDEX]));
|
||||
|
||||
indexInfo.setQueueName(
|
||||
decodeJobHistoryFileName(jobDetails[QUEUE_NAME_INDEX]));
|
||||
} catch (IndexOutOfBoundsException e) {
|
||||
LOG.warn("Parsing job history file with partial data encoded into name: "
|
||||
+ jhFileName);
|
||||
}
|
||||
|
||||
return indexInfo;
|
||||
}
|
||||
|
@ -28,6 +28,7 @@ public class JobIndexInfo {
|
||||
private long submitTime;
|
||||
private long finishTime;
|
||||
private String user;
|
||||
private String queueName;
|
||||
private String jobName;
|
||||
private JobId jobId;
|
||||
private int numMaps;
|
||||
@ -67,6 +68,12 @@ public String getUser() {
|
||||
public void setUser(String user) {
|
||||
this.user = user;
|
||||
}
|
||||
public String getQueueName() {
|
||||
return queueName;
|
||||
}
|
||||
public void setQueueName(String queueName) {
|
||||
this.queueName = queueName;
|
||||
}
|
||||
public String getJobName() {
|
||||
return jobName;
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user