Skip to content

Commit

Permalink
HDFS-7270. Add congestion signaling capability to DataNode write prot…
Browse files Browse the repository at this point in the history
…ocol. Contributed by Haohui Mai.
  • Loading branch information
Haohui Mai committed Feb 5, 2015
1 parent d27439f commit c4980a2
Show file tree
Hide file tree
Showing 8 changed files with 147 additions and 40 deletions.
3 changes: 3 additions & 0 deletions hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
Expand Up @@ -571,6 +571,9 @@ Release 2.7.0 - UNRELEASED

HDFS-7712. Switch blockStateChangeLog to use slf4j. (wang)

HDFS-7270. Add congestion signaling capability to DataNode write protocol.
(wheat9)

OPTIMIZATIONS

HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
Expand Down
Expand Up @@ -764,4 +764,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String NNTOP_WINDOWS_MINUTES_KEY =
"dfs.namenode.top.windows.minutes";
public static final String[] NNTOP_WINDOWS_MINUTES_DEFAULT = {"1","5","25"};
public static final String DFS_PIPELINE_ECN_ENABLED = "dfs.pipeline.ecn";
public static final boolean DFS_PIPELINE_ECN_ENABLED_DEFAULT = false;
}
Expand Up @@ -892,7 +892,8 @@ public void run() {
long seqno = ack.getSeqno();
// processes response status from datanodes.
for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) {
final Status reply = ack.getReply(i);
final Status reply = PipelineAck.getStatusFromHeader(ack
.getReply(i));
// Restart will not be treated differently unless it is
// the local node or the only one in the pipeline.
if (PipelineAck.isRestartOOBStatus(reply) &&
Expand Down
Expand Up @@ -22,17 +22,21 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;

import com.google.common.collect.Lists;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_DEFAULT;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos;

import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import com.google.protobuf.TextFormat;
import org.apache.hadoop.hdfs.util.LongBitFormat;

/** Pipeline Acknowledgment **/
@InterfaceAudience.Private
Expand All @@ -46,6 +50,55 @@ public class PipelineAck {
// place holder for timeout value of each OOB type
final static long[] OOB_TIMEOUT;

public enum ECN {
DISABLED(0),
SUPPORTED(1),
SUPPORTED2(2),
CONGESTED(3);

private final int value;
private static final ECN[] VALUES = values();
static ECN valueOf(int value) {
return VALUES[value];
}

ECN(int value) {
this.value = value;
}

public int getValue() {
return value;
}
}

private enum StatusFormat {
STATUS(null, 4),
RESERVED(STATUS.BITS, 1),
ECN_BITS(RESERVED.BITS, 2);

private final LongBitFormat BITS;

StatusFormat(LongBitFormat prev, int bits) {
BITS = new LongBitFormat(name(), prev, bits, 0);
}

static Status getStatus(int header) {
return Status.valueOf((int) STATUS.BITS.retrieve(header));
}

static ECN getECN(int header) {
return ECN.valueOf((int) ECN_BITS.BITS.retrieve(header));
}

public static int setStatus(int old, Status status) {
return (int) STATUS.BITS.combine(status.getNumber(), old);
}

public static int setECN(int old, ECN ecn) {
return (int) ECN_BITS.BITS.combine(ecn.getValue(), old);
}
}

static {
OOB_TIMEOUT = new long[NUM_OOB_TYPES];
HdfsConfiguration conf = new HdfsConfiguration();
Expand All @@ -65,7 +118,7 @@ public PipelineAck() {
* @param seqno sequence number
* @param replies an array of replies
*/
public PipelineAck(long seqno, Status[] replies) {
public PipelineAck(long seqno, int[] replies) {
this(seqno, replies, 0L);
}

Expand All @@ -75,10 +128,15 @@ public PipelineAck(long seqno, Status[] replies) {
* @param replies an array of replies
* @param downstreamAckTimeNanos ack RTT in nanoseconds, 0 if no next DN in pipeline
*/
public PipelineAck(long seqno, Status[] replies, long downstreamAckTimeNanos) {
public PipelineAck(long seqno, int[] replies,
long downstreamAckTimeNanos) {
ArrayList<Integer> replyList = Lists.newArrayList();
for (int r : replies) {
replyList.add(r);
}
proto = PipelineAckProto.newBuilder()
.setSeqno(seqno)
.addAllStatus(Arrays.asList(replies))
.addAllReply(replyList)
.setDownstreamAckTimeNanos(downstreamAckTimeNanos)
.build();
}
Expand All @@ -96,15 +154,15 @@ public long getSeqno() {
* @return the number of replies
*/
public short getNumOfReplies() {
return (short)proto.getStatusCount();
return (short)proto.getReplyCount();
}

/**
* get the ith reply
* @return the the ith reply
*/
public Status getReply(int i) {
return proto.getStatus(i);
public int getReply(int i) {
return proto.getReply(i);
}

/**
Expand All @@ -120,8 +178,8 @@ public long getDownstreamAckTimeNanos() {
* @return true if all statuses are SUCCESS
*/
public boolean isSuccess() {
for (Status reply : proto.getStatusList()) {
if (reply != Status.SUCCESS) {
for (int reply : proto.getReplyList()) {
if (StatusFormat.getStatus(reply) != Status.SUCCESS) {
return false;
}
}
Expand All @@ -138,11 +196,12 @@ public Status getOOBStatus() {
if (getSeqno() != UNKOWN_SEQNO) {
return null;
}
for (Status reply : proto.getStatusList()) {
for (int reply : proto.getReplyList()) {
// The following check is valid because protobuf guarantees to
// preserve the ordering of enum elements.
if (reply.getNumber() >= OOB_START && reply.getNumber() <= OOB_END) {
return reply;
Status s = StatusFormat.getStatus(reply);
if (s.getNumber() >= OOB_START && s.getNumber() <= OOB_END) {
return s;
}
}
return null;
Expand Down Expand Up @@ -184,4 +243,19 @@ public void write(OutputStream out) throws IOException {
public String toString() {
return TextFormat.shortDebugString(proto);
}

public static Status getStatusFromHeader(int header) {
return StatusFormat.getStatus(header);
}

public static int setStatusForHeader(int old, Status status) {
return StatusFormat.setStatus(old, status);
}

public static int combineHeader(ECN ecn, Status status) {
int header = 0;
header = StatusFormat.setStatus(header, status);
header = StatusFormat.setECN(header, ecn);
return header;
}
}
Expand Up @@ -978,9 +978,7 @@ private Checksum computePartialChunkCrc(long blkoff, long ckoff)
private static enum PacketResponderType {
NON_PIPELINE, LAST_IN_PIPELINE, HAS_DOWNSTREAM_IN_PIPELINE
}

private static final Status[] MIRROR_ERROR_STATUS = {Status.SUCCESS, Status.ERROR};


/**
* Processes responses from downstream datanodes in the pipeline
* and sends back replies to the originator.
Expand Down Expand Up @@ -1084,7 +1082,7 @@ void sendOOBResponse(final Status ackStatus) throws IOException,
LOG.info("Sending an out of band ack of type " + ackStatus);
try {
sendAckUpstreamUnprotected(null, PipelineAck.UNKOWN_SEQNO, 0L, 0L,
ackStatus);
PipelineAck.combineHeader(datanode.getECN(), ackStatus));
} finally {
// Let others send ack. Unless there are miltiple OOB send
// calls, there can be only one waiter, the responder thread.
Expand Down Expand Up @@ -1167,7 +1165,8 @@ public void run() {
if (oobStatus != null) {
LOG.info("Relaying an out of band ack of type " + oobStatus);
sendAckUpstream(ack, PipelineAck.UNKOWN_SEQNO, 0L, 0L,
Status.SUCCESS);
PipelineAck.combineHeader(datanode.getECN(),
Status.SUCCESS));
continue;
}
seqno = ack.getSeqno();
Expand Down Expand Up @@ -1241,9 +1240,10 @@ public void run() {
finalizeBlock(startTime);
}

Status myStatus = pkt != null ? pkt.ackStatus : Status.SUCCESS;
sendAckUpstream(ack, expected, totalAckTimeNanos,
(pkt != null ? pkt.offsetInBlock : 0),
(pkt != null ? pkt.ackStatus : Status.SUCCESS));
(pkt != null ? pkt.offsetInBlock : 0),
PipelineAck.combineHeader(datanode.getECN(), myStatus));
if (pkt != null) {
// remove the packet from the ack queue
removeAckHead();
Expand Down Expand Up @@ -1303,11 +1303,11 @@ private void finalizeBlock(long startTime) throws IOException {
* @param totalAckTimeNanos total ack time including all the downstream
* nodes
* @param offsetInBlock offset in block for the data in packet
* @param myStatus the local ack status
* @param myHeader the local ack header
*/
private void sendAckUpstream(PipelineAck ack, long seqno,
long totalAckTimeNanos, long offsetInBlock,
Status myStatus) throws IOException {
int myHeader) throws IOException {
try {
// Wait for other sender to finish. Unless there is an OOB being sent,
// the responder won't have to wait.
Expand All @@ -1321,7 +1321,7 @@ private void sendAckUpstream(PipelineAck ack, long seqno,
try {
if (!running) return;
sendAckUpstreamUnprotected(ack, seqno, totalAckTimeNanos,
offsetInBlock, myStatus);
offsetInBlock, myHeader);
} finally {
synchronized(this) {
sending = false;
Expand All @@ -1341,32 +1341,34 @@ private void sendAckUpstream(PipelineAck ack, long seqno,
* @param totalAckTimeNanos total ack time including all the downstream
* nodes
* @param offsetInBlock offset in block for the data in packet
* @param myStatus the local ack status
* @param myHeader the local ack header
*/
private void sendAckUpstreamUnprotected(PipelineAck ack, long seqno,
long totalAckTimeNanos, long offsetInBlock, Status myStatus)
long totalAckTimeNanos, long offsetInBlock, int myHeader)
throws IOException {
Status[] replies = null;
final int[] replies;
if (ack == null) {
// A new OOB response is being sent from this node. Regardless of
// downstream nodes, reply should contain one reply.
replies = new Status[1];
replies[0] = myStatus;
replies = new int[] { myHeader };
} else if (mirrorError) { // ack read error
replies = MIRROR_ERROR_STATUS;
int h = PipelineAck.combineHeader(datanode.getECN(), Status.SUCCESS);
int h1 = PipelineAck.combineHeader(datanode.getECN(), Status.ERROR);
replies = new int[] {h, h1};
} else {
short ackLen = type == PacketResponderType.LAST_IN_PIPELINE ? 0 : ack
.getNumOfReplies();
replies = new Status[1 + ackLen];
replies[0] = myStatus;
for (int i = 0; i < ackLen; i++) {
replies = new int[ackLen + 1];
replies[0] = myHeader;
for (int i = 0; i < ackLen; ++i) {
replies[i + 1] = ack.getReply(i);
}
// If the mirror has reported that it received a corrupt packet,
// do self-destruct to mark myself bad, instead of making the
// do self-destruct to mark myself bad, instead of making the
// mirror node bad. The mirror is guaranteed to be good without
// corrupt data on disk.
if (ackLen > 0 && replies[1] == Status.ERROR_CHECKSUM) {
if (ackLen > 0 && PipelineAck.getStatusFromHeader(replies[1]) ==
Status.ERROR_CHECKSUM) {
throw new IOException("Shutting down writer and responder "
+ "since the down streams reported the data sent by this "
+ "thread is corrupt");
Expand All @@ -1392,7 +1394,8 @@ private void sendAckUpstreamUnprotected(PipelineAck ack, long seqno,
}

// If a corruption was detected in the received data, terminate after
// sending ERROR_CHECKSUM back.
// sending ERROR_CHECKSUM back.
Status myStatus = PipelineAck.getStatusFromHeader(myHeader);
if (myStatus == Status.ERROR_CHECKSUM) {
throw new IOException("Shutting down writer and responder "
+ "due to a checksum error in received data. The error "
Expand All @@ -1412,7 +1415,7 @@ private void removeAckHead() {
}
}
}

/**
* This information is cached by the Datanode in the ackQueue.
*/
Expand Down
Expand Up @@ -124,6 +124,7 @@
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
Expand Down Expand Up @@ -333,6 +334,7 @@ public static InetSocketAddress createSocketAddr(String target) {
private Configuration conf;
private final String confVersion;
private final long maxNumberOfBlocksToLog;
private final boolean pipelineSupportECN;

private final List<String> usersWithLocalPathAccess;
private final boolean connectToDnViaHostname;
Expand Down Expand Up @@ -366,6 +368,7 @@ public static InetSocketAddress createSocketAddr(String target) {
this.connectToDnViaHostname = false;
this.getHdfsBlockLocationsEnabled = false;
this.blockScanner = new BlockScanner(this, conf);
this.pipelineSupportECN = false;
}

/**
Expand Down Expand Up @@ -394,6 +397,9 @@ public static InetSocketAddress createSocketAddr(String target) {
this.isPermissionEnabled = conf.getBoolean(
DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY,
DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT);
this.pipelineSupportECN = conf.getBoolean(
DFSConfigKeys.DFS_PIPELINE_ECN_ENABLED,
DFSConfigKeys.DFS_PIPELINE_ECN_ENABLED_DEFAULT);

confVersion = "core-" +
conf.get("hadoop.common.configuration.version", "UNSPECIFIED") +
Expand Down Expand Up @@ -468,6 +474,19 @@ public Collection<String> getReconfigurableProperties() {
return reconfigurable;
}

/**
* The ECN bit for the DataNode. The DataNode should return:
* <ul>
* <li>ECN.DISABLED when ECN is disabled.</li>
* <li>ECN.SUPPORTED when ECN is enabled but the DN still has capacity.</li>
* <li>ECN.CONGESTED when ECN is enabled and the DN is congested.</li>
* </ul>
*/
public PipelineAck.ECN getECN() {
return pipelineSupportECN ? PipelineAck.ECN.SUPPORTED : PipelineAck.ECN
.DISABLED;
}

/**
* Contains the StorageLocations for changed data volumes.
*/
Expand Down

0 comments on commit c4980a2

Please sign in to comment.