写入流程图
- 请求创建新文件。向NN发送create()请求。NN收到请求后在目录树的对应路径创建新文件并记在editlog。创建完成后返回FSDataOutputStream。
- 写入。
- client向NN申请block
- NN按照机架感知返回DN,第一个是离client近的节点,初始化pipline
- 把数据切分成packet发送,在多个DN间pipline传输,反向发送ACK
- DN写完后会向NN汇报块
- 写满一个block,再申请一个新的block
- 关闭流。向NN发送complete请求
基本概念
- block(文件块) hdfs操作文件的最小单元,默认128MB
- chunk(校验块) block实际存储/校验的最小单位, 严谨说chunk分为数据域和校验域两个部分组成, 但一般情况大家指的是数据域
- 数据域: 默认512字节, 比如你写513个字节数据, 它占据一个block (而此block内部是两个chunk组成) , 又称为chunk data
- 校验域: 固定4字节, 存放校验码, 它与数据域一一对应, 又称chunk checksum
- packet(数据包): HDFS各组件间数据传输的基本单位, 默认64KB, 类似网络传输中数据包的概念, 主要是 header + body/data 两个部分组成
- packet header : 存储传输数据过程中的一些基本信息(数据包长度/版本/标志位等), 变长
- packet body: 存储实际传输的数据(chunk), 同样packet里实际分布的最小单位也是chunk
一个block的结构
meta文件存储校验信息 ,包括校验版本/类型,本质是N个4字节的chunk校验域组成data部分由多个chunkdata组成,每个chunkdata存放二进制数据- HDFS组件间传输数据最小以packet为单位, 1个packet由多个chunk构成, 传输后取出其中的chunk(data+checksum), 最后构成block和
meta
create
DFSOutputStream##newStreamForCreate
- 向NN发送create请求
stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
new EnumSetWritable<CreateFlag>(flag), createParent, replication,
blockSize, SUPPORTED_CRYPTO_VERSIONS);
- 初始化DFSOutputStream
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
flag, progress, checksum, favoredNodes);
在DFSOutputStream初始化过程中,会初始化DataStreamer,并计算packetSize,这是实际传输的packet数据部分的大小。
/** Construct a new output stream for creating a file. */
private DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
EnumSet<CreateFlag> flag, Progressable progress,
DataChecksum checksum, String[] favoredNodes) throws IOException {
this(dfsClient, src, progress, stat, checksum);
this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum);
streamer = new DataStreamer(stat, null);
if (favoredNodes != null && favoredNodes.length != 0) {
streamer.setFavoredNodes(favoredNodes);
}
}
private void computePacketChunkSize(int psize, int csize) {
//减去packetHeader
final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
final int chunkSize = csize + getChecksumSize();
chunksPerPacket = Math.max(bodySize/chunkSize, 1);
packetSize = chunkSize*chunksPerPacket;
}
参数解释:
| 名称 | 数值 (默认) | 含义 |
|---|---|---|
| psize | 65536B (64K) | packetSizeMax的意思. 理论上可达65536B (64K) |
| csize | 512B | 每个chunk数据域的大小 |
| checkSumSize | 4B | 每个chunk校验域的大小 |
| chunkSize | 516B | 一个完整的chunk的大小 ( 数据域+ 校验域) |
| PKT_LENGTHS_LEN | 6B | PacketLength和HeaderLength之和 –> 4 + 2 |
| MAX_PROTO_SIZE | 27B | packet头中走protobuf序列化的最大占用(详见后文) |
| bodySize | 65503B | 一个packet的数据域部分最大值 (也称packet data) |
| chunksPerPacket | 126 | 每个packet可以承载多少个校验块(低版127个) |
| packetSize | 65016 | 这是当前实际传输时每个packet的大小, 约63.5K |
- 启动DataStreamer线程
out.start();
private synchronized void start() {
streamer.start();
}
写入数据
1. 生成packet
2. 发送packet(核心)
A. 整体结构
因为小于63.5K的数据不足发车要求, 这里假设写入128K的文件, 它应该占据三个packet, 但是只有前两个packet会入队发车, 最后一个在末班车才发.
满足1个packet大小的数据会被封装好, 然后Client依次发给DN, 这里Packet只是数据包的载体, 就像HTTP请求发数据, 除了数据载体本身也需要有一个请求头, 所以在发送Packet之前, 需要一个Sender发送写请求, 得到了所有DN的确认后, 才会开始发数据。
这里把每个Packet看做是一节载货车厢, 最后一节是空车厢(空尾包), 它们整体结构就像一辆载货的火车,“火车头“Sender并非是跟着货物(Packets)一起发送, 而是先单独发送, 车厢(packet)再进入铁轨(pipeline), 开始一个个传输。Sender在初始态变为就绪态时发送。
B. 流程图
C. 源码分析
Datastreamer:1. 负责开关pipline,维护pipline状态变化 2. 执行数据写入和错误恢复
packet包:
- 数据包
- 心跳包 当执行create方法,初始化DFSOutputStream后,DataStreamer线程就启动了,变为 DATA_STREAMING状态后,如果没数据时会每隔一段时间发送心跳包
- 空尾包 收到空尾包会一直等待所有packet都ack
- block结束
- 调用close方法
pipline的状态:
- PIPELINE_SETUP_CREATE 写入初始状态,此时向NN申请block,初始化pipline,变为就绪态
- DATA_STREAMING 就绪态,可以传输数据
- PIPELINE_SETUP_APPEND 追加写
- PIPELINE_CLOSE 关闭
pipline状态转移:
ResponseProcessor:接受DN1的reply,逆序遍历判断是否异常(由于数据是从DN1->DN2->DN3,节点先读取数据之后再转发给下个节点,如果DN2有问题,不会收到DN3的ack),异常处理逻辑是一次只处理一个故障节点,如果catch中没有确定故障节点,那么认为第一个是故障节点。
- 无异常 从ACKQueue中移除
- 有异常
- 本地节点或者唯一节点重启,setRestartingNodeIndex并直接抛出IO异常
- ack返回异常 抛出io异常 errorIndex记录第一个有异常的节点
DataStreamer##run
public void run() {
// 1. 等待, 直到有packet被放入dataQueue, 或者发心跳包
long lastPacket = Time.monotonicNow();
TraceScope scope = NullScope.INSTANCE;
while (!streamerClosed && dfsClient.clientRunning) {
// if the Responder encountered an error, shutdown Responder
if (hasError && response != null) {
try {
response.close();
response.join();
response = null;
} catch (InterruptedException e) {
DFSClient.LOG.warn("Caught exception ", e);
}
}
DFSPacket one;
try {
// process datanode IO errors if any
boolean doSleep = false;
if (hasError && (errorIndex >= 0 || restartingNodeIndex.get() >= 0)) {
doSleep = processDatanodeError();//失败返回false
}
synchronized (dataQueue) {
// wait for a packet to be sent.默认不等待 失败情况下也不等待
long now = Time.monotonicNow();
// 当队空, 且pipeline处于非就绪态(或距上一次发packet时间小于30s),等待一段时间
while ((!streamerClosed && !hasError && dfsClient.clientRunning
&& dataQueue.size() == 0 &&
(stage != BlockConstructionStage.DATA_STREAMING ||
stage == BlockConstructionStage.DATA_STREAMING &&
now - lastPacket < dfsClient.getConf().socketTimeout/2)) || doSleep ) {
long timeout = dfsClient.getConf().socketTimeout/2 - (now-lastPacket);
timeout = timeout <= 0 ? 1000 : timeout;
timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
timeout : 1000;
try {
dataQueue.wait(timeout);
} catch (InterruptedException e) {
DFSClient.LOG.warn("Caught exception ", e);
}
doSleep = false;
now = Time.monotonicNow();
}
if (streamerClosed || hasError || !dfsClient.clientRunning) {//失败情况下streamerClosed为true 跳出循环
continue;
}
// get packet to be sent.
// 2. 如果队不为空了, 或距上一次发packet时间大于30s
// 两种情况:1.队仍为空, 发心跳包 2.否则取出要被发送的packet
if (dataQueue.isEmpty()) {
one = createHeartbeatPacket();
assert one != null;
} else {
one = dataQueue.getFirst();
}
}
// get new block from namenode.create方法会分配一个新块
// 3. 若pipeline处于初始状态, 则给NN发addBlock()请求申请block, 然后初始化
if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
setPipeline(nextBlockOutputStream());
//启动ResponseProcessor线程,更新DataStreamer的状态为DATA_STREAMING
initDataStreaming();
} else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {//追加写/恢复逻辑
setupPipelineForAppendOrRecovery();
initDataStreaming();
}
// 4. 发送Packet大流程
// 4.1 如果当前为空尾包, 则需先等待之前的所有packet都acked
if (one.isLastPacketInBlock()) {
// wait for all data packets have been successfully acked
synchronized (dataQueue) {
while (!streamerClosed && !hasError &&
ackQueue.size() != 0 && dfsClient.clientRunning) {
try {
// wait for acks to arrive from datanodes
dataQueue.wait(1000);
} catch (InterruptedException e) {
DFSClient.LOG.warn("Caught exception ", e);
}
}
}
if (streamerClosed || hasError || !dfsClient.clientRunning) {
continue;
}
stage = BlockConstructionStage.PIPELINE_CLOSE;
}
// send the packet
// 4.2 普通发送packet 将非心跳包从dataQueue移到ackQueue 空尾包也会移到ackQueue
Span span = null;
synchronized (dataQueue) {
// move packet from dataQueue to ackQueue
if (!one.isHeartbeatPacket()) {
span = scope.detach();
one.setTraceSpan(span);
dataQueue.removeFirst();
ackQueue.addLast(one);
dataQueue.notifyAll();
}
}
// write out data to remote datanode
// 5. 真正写数据到通往DN的stream
TraceScope writeScope = Trace.startSpan("writeTo", span);
try {
one.writeTo(blockStream)
blockStream.flush();
} catch (IOException e) {
tryMarkPrimaryDatanodeFailed();
throw e;
} finally {
writeScope.close();
}
lastPacket = Time.monotonicNow();
// update bytesSent
// 6. 更新已发送字节数, 准备收尾
long tmpBytesSent = one.getLastByteOffsetBlock();
if (bytesSent < tmpBytesSent) {
bytesSent = tmpBytesSent;
}
if (streamerClosed || hasError || !dfsClient.clientRunning) {
continue;
}
// Is this block full?
// 6.1 如果当前packet是空尾包, 则重置pipeline为初始态
if (one.isLastPacketInBlock()) {
// wait for the close packet has been acked
synchronized (dataQueue) {
while (!streamerClosed && !hasError &&
ackQueue.size() != 0 && dfsClient.clientRunning) {
dataQueue.wait(1000);// wait for acks to arrive from datanodes
}
}
if (streamerClosed || hasError || !dfsClient.clientRunning) {
continue;
}
endBlock();
}
if (progress != null) { progress.progress(); }
// This is used by unit test to trigger race conditions.
if (artificialSlowdown != 0 && dfsClient.clientRunning) {
Thread.sleep(artificialSlowdown);
}
} catch (Throwable e) {
// Log warning if there was a real error.
if (restartingNodeIndex.get() == -1) {
DFSClient.LOG.warn("DataStreamer Exception", e);
}
if (e instanceof IOException) {
setLastException((IOException)e);
} else {
setLastException(new IOException("DataStreamer Exception: ",e));
}
hasError = true;
if (errorIndex == -1 && restartingNodeIndex.get() == -1) {
// Not a datanode issue
streamerClosed = true;
}
} finally {
scope.close();
}
}
closeInternal();
}
第三步中的初始化pipline
- 排除故障节点
- 向NN发送addBlock申请
- 和第一个block建连,获得写数据的输出流blockStream,和响应的输入流blockReplyStream
- 如果失败,返回故障节点
- 启动ResponseProcessor线程,更新DataStreamer的状态为DATA_STREAMING
private LocatedBlock nextBlockOutputStream() throws IOException {
LocatedBlock lb = null;
DatanodeInfo[] nodes = null;
StorageType[] storageTypes = null;
int count = dfsClient.getConf().nBlockWriteRetry;//默认最多重试三次
boolean success = false;
ExtendedBlock oldBlock = block;
do {
hasError = false;
lastException.set(null);
errorIndex = -1;
success = false;
//排除故障节点
DatanodeInfo[] excluded =
excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
.keySet()
.toArray(new DatanodeInfo[0]);
block = oldBlock;
//向NN发送addBlock请求
lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
block = lb.getBlock();
block.setNumBytes(0);
bytesSent = 0;
accessToken = lb.getBlockToken();
nodes = lb.getLocations();
storageTypes = lb.getStorageTypes();
//
// Connect to first DataNode in the list.
// 3. 创建pipeline, 连接DNList中的第一个DN, 成功返回true
//
success = createBlockOutputStream(nodes, storageTypes, 0L, false);
if (!success) {
DFSClient.LOG.info("Abandoning " + block);
// 创建失败发RPC放弃当前block,并把对应DN放入故障DN
dfsClient.namenode.abandonBlock(block, fileId, src,
dfsClient.clientName);
block = null;
DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
excludedNodes.put(nodes[errorIndex], nodes[errorIndex]);
}
} while (!success && --count >= 0);
if (!success) {
throw new IOException("Unable to create new block.");
}
return lb;
}
private boolean createBlockOutputStream(DatanodeInfo[] nodes,
StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
if (nodes.length == 0) {
DFSClient.LOG.info("nodes are empty for write pipeline of block "
+ block);
return false;
}
Status pipelineStatus = SUCCESS;
String firstBadLink = "";
boolean checkRestart = false;
// persist blocks on namenode on next flush
persistBlocks.set(true);
int refetchEncryptionKey = 1;
while (true) {
boolean result = false;
DataOutputStream out = null;
try {
assert null == s : "Previous socket unclosed";
assert null == blockReplyStream : "Previous blockReplyStream unclosed";
// 建立到第一个DN的socket连接
s = createSocketForPipeline(nodes[0], nodes.length, dfsClient);
long writeTimeout = dfsClient.getDatanodeWriteTimeout(nodes.length);
OutputStream unbufOut = NetUtils.getOutputStream(s, writeTimeout);
InputStream unbufIn = NetUtils.getInputStream(s);
IOStreamPair saslStreams = dfsClient.saslClient.socketSend(s,
unbufOut, unbufIn, dfsClient, accessToken, nodes[0]);
unbufOut = saslStreams.out;
unbufIn = saslStreams.in;
//封装outputstream
out = new DataOutputStream(new BufferedOutputStream(unbufOut,
HdfsConstants.SMALL_BUFFER_SIZE));
blockReplyStream = new DataInputStream(unbufIn);
//
// Xmit header info to datanode
//
BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
// We cannot change the block length in 'block' as it counts the number
// of bytes ack'ed.
ExtendedBlock blockCopy = new ExtendedBlock(block);
blockCopy.setNumBytes(blockSize);
boolean[] targetPinnings = getPinnings(nodes, true);
// send the request
new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
nodes.length, block.getNumBytes(), bytesSent, newGS,
checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
(targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
// receive ack for connect
// 接受写数据块后返回的ack
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
PBHelper.vintPrefixed(blockReplyStream));
pipelineStatus = resp.getStatus();
firstBadLink = resp.getFirstBadLink();
// Got an restart OOB ack.
// If a node is already restarting, this status is not likely from
// the same node. If it is from a different node, it is not
// from the local datanode. Thus it is safe to treat this as a
// regular node error.
if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
restartingNodeIndex.get() == -1) {
checkRestart = true;
throw new IOException("A datanode is restarting.");
}
String logInfo = "ack with firstBadLink as " + firstBadLink;
DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
assert null == blockStream : "Previous blockStream unclosed";
blockStream = out;
result = true; // success
restartingNodeIndex.set(-1);
hasError = false;
} catch (IOException ie) {
if (restartingNodeIndex.get() == -1) {
DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
}
// find the datanode that matches
//找到错误节点返回给client
if (firstBadLink.length() != 0) {
for (int i = 0; i < nodes.length; i++) {
// NB: Unconditionally using the xfer addr w/o hostname
if (firstBadLink.equals(nodes[i].getXferAddr())) {
errorIndex = i;
break;
}
}
} else {
assert checkRestart == false;
errorIndex = 0;
}
// Check whether there is a restart worth waiting for.
if (checkRestart && shouldWaitForRestart(errorIndex)) {
restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
Time.monotonicNow();
restartingNodeIndex.set(errorIndex);
errorIndex = -1;
DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
nodes[restartingNodeIndex.get()]);
}
hasError = true;
setLastException(ie);
result = false; // error
} finally {
if (!result) {
IOUtils.closeSocket(s);
s = null;
IOUtils.closeStream(out);
out = null;
IOUtils.closeStream(blockReplyStream);
blockReplyStream = null;
}
}
return result;
}
}
ResponseProcessor##run
public void run() {
setName("ResponseProcessor for block " + block);
PipelineAck ack = new PipelineAck();
TraceScope scope = NullScope.INSTANCE;
while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
// process responses from datanodes.
try {
// read an ack from the pipeline
long begin = Time.monotonicNow();
//1.获取DN的返回信息
ack.readFields(blockReplyStream);
long duration = Time.monotonicNow() - begin;
if (duration > dfsclientSlowLogThresholdMs
&& ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
DFSClient.LOG
.warn("Slow ReadProcessor read fields took " + duration
+ "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
+ ack + ", targets: " + Arrays.asList(targets));
} else if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("DFSClient " + ack);
}
long seqno = ack.getSeqno();
// processes response status from datanodes.
//2.逆序遍历 DNn-->DN2-->DN1
for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) {
final Status reply = PipelineAck.getStatusFromHeader(ack
.getHeaderFlag(i));
// Restart will not be treated differently unless it is
// the local node or the only one in the pipeline.
//仅当DN是本地节点或它是pipeline中的唯一节点是, 才会特殊处理它的重启状态(OOB)
if (PipelineAck.isRestartOOBStatus(reply) &&
shouldWaitForRestart(i)) {
restartDeadline = dfsClient.getConf().datanodeRestartTimeout
+ Time.monotonicNow();//重启过期时间
setRestartingNodeIndex(i);
String message = "A datanode is restarting: " + targets[i];
DFSClient.LOG.info(message);
throw new IOException(message);
}
// node error
//果ack返回有异常, 直接抛出到catch中再处理 (核心)
if (reply != SUCCESS) {
setErrorIndex(i); // first bad datanode
throw new IOException("Bad response " + reply +
" for block " + block +
" from datanode " +
targets[i]);
}
}
//3. 忽略由Client发出的心跳ack(细节, 需要注意, 心跳包是全过程中是如何处理的)
assert seqno != PipelineAck.UNKOWN_SEQNO :
"Ack for unknown seqno should be a failed ack: " + ack;
if (seqno == DFSPacket.HEART_BEAT_SEQNO) { // a heartbeat ack
continue;
}
// a success ack for a data packet
// 4.先判断ack队中的第一个packet和当前dn返回的序号相同
DFSPacket one;
synchronized (dataQueue) {
one = ackQueue.getFirst();
}
if (one.getSeqno() != seqno) {
throw new IOException("ResponseProcessor: Expecting seqno " +
" for block " + block +
one.getSeqno() + " but received " + seqno);
}
isLastPacketInBlock = one.isLastPacketInBlock();
// Fail the packet write for testing in order to force a
// pipeline recovery.
if (DFSClientFaultInjector.get().failPacket() &&
isLastPacketInBlock) {
failPacket = true;
throw new IOException(
"Failing the last packet for testing.");
}
// update bytesAcked
block.setNumBytes(one.getLastByteOffsetBlock());
// 5.确认一切无误后, 再从ack队中移除
synchronized (dataQueue) {
scope = Trace.continueSpan(one.getTraceSpan());
one.setTraceSpan(null);
lastAckedSeqno = seqno;
ackQueue.removeFirst();
dataQueue.notifyAll();
one.releaseBuffer(byteArrayManager);
}
} catch (Exception e) {
// 如果发现有DN返回失败, 在此处理异常, 立刻关闭response线程
if (!responderClosed) {
if (e instanceof IOException) {
setLastException((IOException)e);
}
hasError = true;
// If no explicit error report was received, mark the primary
// node as failed.
tryMarkPrimaryDatanodeFailed();
synchronized (dataQueue) {
dataQueue.notifyAll();
}
if (restartingNodeIndex.get() == -1) {
DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
+ " for block " + block, e);
}
responderClosed = true;
}
} finally {
scope.close();
}
}
}