前言
上篇文章Ozone Datanode的分佈式元數據管理,筆者闡述了關於Ozone Datanode的分佈式元數據相關的內容以及Datanode上的數據layout。既然我們瞭解了Datanode上元數據的結構,那麼一個很自然的問題:Datanode如何進行數據的寫入的呢?裏面的數據一致性是怎麼做的?中間寫數據發生錯誤,Datanode這邊怎麼來處理?本文我們來細聊Ozone數據寫入的內部過程。
Ozone Key(文件)數據的寫入
我們知道,Ozone作爲對象存儲系統,支持K-V模式的鍵值對存儲,文件的數據的put操作實質上代表的key的寫入。Ozone在這邊是通過OM返回給客戶端KeyOutputStream對象,來進行後續數據的寫入的。換句話說,client是通過向OM服務申請獲取目標key的KeyOutputStream,樣例代碼如下:
private void writeKey(String key1, OzoneBucket bucket) throws IOException {
OzoneOutputStream out = bucket.createKey(key1, 1024, STAND_ALONE,
ONE, new HashMap<>());
out.write(RandomStringUtils.random(1024).getBytes());
out.close();
}
OzoneOutputStream內部包裝的OutputStream對象即爲KeyOutputStream。
KeyOutputStream代表一個key下的數據文件,如果目標key對應要寫的數據文件比較大,就可能會出現需要多block存儲的情況。類似於HDFS大文件,需要多個block來存儲,每個block根據offset來分隔開。在Ozone中,每個block又對應有獨自的BlockOutputStream,這裏每個專屬的BlockOutputStream全程控制對應詞block的數據寫操作。
在前篇文章也提到了,Block在Ozone中是虛擬的概念,實際存儲的文件叫chunk文件,一個Block由1個或多個chunk文件組成。因此BlockOutputStream的數據寫出實質上是chunk文件的數據寫出。BlockOutputStream內部維護了一個Chunk Buffer池做臨時數據緩存,等數據達到flush觸發閾值,BlockOutputStream再進行chunk文件的數據寫Datanode操作。
然後執行client向Datanode發起putBlock的元數據更新操作,更新其Container db文件。此操作結束,意味着這個block數據成功地被寫出Datanode中了。然後BlockOutputStream內部對應的Chunk Buffer空間也能被釋放了。
上述數據的寫出過程如下圖所示:
KeyOutputStream的write方法代碼如下,通過Block池創建多個BlockOutputStream進行數據的寫入,
private void handleWrite(byte[] b, int off, long len, boolean retry)
throws IOException {
while (len > 0) {
// 如果當前剩餘寫入長度還未減少爲0,則意爲數據還未完全寫出到Block,則繼續進行循環內的數據寫出
try {
// 1.Block Pool新申請塊進行數據的寫入,返回的BlockOutputStream包裝對象
BlockOutputStreamEntry current =
blockOutputStreamEntryPool.allocateBlockIfNeeded();
// length(len) will be in int range if the call is happening through
// write API of blockOutputStream. Length can be in long range if it
// comes via Exception path.
// 2.計算得到應寫出的len數據長度,取當前BlockOutputStream和目標寫入長度的最小值
int writeLen = Math.min((int) len, (int) current.getRemaining());
long currentPos = current.getWrittenDataLength();
// 3.寫出字節數據到BlockOutputStream,數據範圍爲字節b從offset位置後的writeLen長度
// 此過程如果達到內部buffer觸發閾值,會進行chunk的flush寫出。
writeToOutputStream(current, retry, len, b, writeLen, off, currentPos);
// 4.如果寫完這批數據後,此BlockOutputStream達到最大寫入length限制,無剩餘,則close此stream
// close操作會flush出最後一個block chunk文件。
if (current.getRemaining() <= 0) {
// since the current block is already written close the stream.
handleFlushOrClose(StreamAction.FULL);
}
// 5.更新offset和len長度值
len -= writeLen;
off += writeLen;
} catch (Exception e) {
// 6.如果發生異常,關閉當前在寫的stream
markStreamClosed();
throw new IOException("Allocate any more blocks for write failed", e);
}
}
}
上述執行邏輯還算比較簡單,新申請一個BlockOutputStream來進行數據寫出,如果寫滿了,則進行close操作,意爲此BlockOutputStream代表的block數據已經成功寫出了。這裏我們重點關注BlockOutputStream的異常寫出情況,Ozone這邊會怎麼處理呢?
private void writeToOutputStream(BlockOutputStreamEntry current,
boolean retry, long len, byte[] b, int writeLen, int off, long currentPos)
throws IOException {
try {
if (retry) {
current.writeOnRetry(len);
} else {
// 調用BlockOutputStreamEntry的write寫出方法
current.write(b, off, writeLen);
offset += writeLen;
}
} catch (IOException ioe) {
...
LOG.debug("writeLen {}, total len {}", writeLen, len);
// 寫出過程失敗,進行異常處理
handleException(current, ioe);
}
}
private void handleException(BlockOutputStreamEntry streamEntry,
IOException exception) throws IOException {
Throwable t = HddsClientUtils.checkForException(exception);
Preconditions.checkNotNull(t);
boolean retryFailure = checkForRetryFailure(t);
boolean containerExclusionException = false;
if (!retryFailure) {
containerExclusionException = checkIfContainerToExclude(t);
}
Pipeline pipeline = streamEntry.getPipeline();
PipelineID pipelineId = pipeline.getId();
// 1.獲取當前stream寫出成功的數據長度
long totalSuccessfulFlushedData = streamEntry.getTotalAckDataLength();
// 2.設置stream當前的位置
streamEntry.setCurrentPosition(totalSuccessfulFlushedData);
long bufferedDataLen = blockOutputStreamEntryPool.computeBufferData();
...
// 3.清空當前stream內部的還未寫出的buffer數據,並關閉此stream
// 以此確保此block stream寫出的數據是成功寫出的數據(但不一定是滿的block size的)。
streamEntry.cleanup(retryFailure);
...
}
我們可以看到,上面的邏輯確保block當前寫出的數據是已經得到ack成功結果返回的數據,然後再關閉寫出異常BlockOutputStream,移除其內部的chunk buffer列表。
Ozone Block(Chunk)數據的寫出
下面我們繼續往內部過程進行分析,來看看BlockOutputStream內部數據寫出過程是怎樣的。
上文已經提到過,Ozone Block數據的寫出實質上是物理chunk文件的寫出過程。一個文件達到Block閾值大小限制時,會產生新的Block。同理一個Block內部,如果超過一個chunk大小時,會有新的chunk文件生成。因此,Block數據的write和Key文件的write操作十分類似,代碼如下:
public void write(byte[] b, int off, int len) throws IOException {
...
while (len > 0) {
int writeLen;
// 1.Buffer Pool分配ChunkBuffer進行數據寫入
final ChunkBuffer currentBuffer = bufferPool.allocateBufferIfNeeded(
bytesPerChecksum);
int pos = currentBuffer.position();
// 2.計算目標應寫入長度
writeLen =
Math.min(chunkSize - pos % chunkSize, len);
// 3.往chunk buffer寫入數據
currentBuffer.put(b, off, writeLen);
if (!currentBuffer.hasRemaining()) {
// 4.如果buffer數據滿了,則寫出buffer數據到chunk文件,調用WriteChunk請求
writeChunk(currentBuffer);
}
// 5.更新offset,剩餘數據長度值,已寫出數據長度
off += writeLen;
len -= writeLen;
writtenDataLength += writeLen;
if (shouldFlush()) {
// 6.如果達到flush長度,進行block metadata的更新,調用PutBlock請求
updateFlushLength();
executePutBlock();
}
// 7.如果BufferPool滿了,則進行阻塞等待,直到block數據已經成功寫出,收到Datanode的ack回覆
if (isBufferPoolFull()) {
handleFullBuffer();
}
}
}
在上述過程中,只有當收到Datanode寫數成功返回的Commit Index之後,BlockOutputStream纔會最終清空相應應的ChunkBuffer。
private void writeChunk(ChunkBuffer buffer)
throws IOException {
...
if (bufferList == null) {
bufferList = new ArrayList<>();
}
// 加入buffer list
bufferList.add(buffer);
// 寫出chunk數據到Datanode的Container中
writeChunkToContainer(buffer.duplicate(0, buffer.position()));
}
private void writeChunkToContainer(ChunkBuffer chunk) throws IOException {
int effectiveChunkSize = chunk.remaining();
final ByteString data = chunk.toByteString(
bufferPool.byteStringConversion());
Checksum checksum = new Checksum(checksumType, bytesPerChecksum);
ChecksumData checksumData = checksum.computeChecksum(chunk);
ChunkInfo chunkInfo = ChunkInfo.newBuilder()
.setChunkName(blockID.get().getLocalID() + "_chunk_" + ++chunkIndex)
.setOffset(0)
.setLen(effectiveChunkSize)
.setChecksumData(checksumData.getProtoBufMessage())
.build();
try {
// 向Datanode發起WriteChunk請求
XceiverClientReply asyncReply =
writeChunkAsync(xceiverClient, chunkInfo, blockID.get(), data);
CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
asyncReply.getResponse();
} catch (IOException | InterruptedException | ExecutionException e) {
throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e);
}
...
// 同時加入BlockData中,此對象會在後面被PutBlock用到
containerBlockData.addChunks(chunkInfo);
}
WriteChunk文件後是PutBlock請求,
private CompletableFuture<ContainerProtos.
ContainerCommandResponseProto> executePutBlock()
throws IOException {
checkOpen();
long flushPos = totalDataFlushedLength;
Preconditions.checkNotNull(bufferList);
final List<ChunkBuffer> byteBufferList = bufferList;
bufferList = null;
Preconditions.checkNotNull(byteBufferList);
CompletableFuture<ContainerProtos.
ContainerCommandResponseProto> flushFuture;
try {
// 發起PutBlock請求
XceiverClientReply asyncReply =
putBlockAsync(xceiverClient, containerBlockData.build());
CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
asyncReply.getResponse();
flushFuture = future.thenApplyAsync(e -> {
blockID.set(responseBlockID);
...
// 加入<Commit Index, buffer list> 到commitWatcher中
commitWatcher
.updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
}
return e;
}, responseExecutor).exceptionally(e -> {
...
commitWatcher.getFutureMap().put(flushPos, flushFuture);
return flushFuture;
}
public XceiverClientReply watchForCommit(long commitIndex)
throws IOException {
long index;
try {
// 阻塞等待Datanode的指定Commit Index
XceiverClientReply reply =
xceiverClient.watchForCommit(commitIndex, watchTimeout);
if (reply == null) {
index = 0;
} else {
index = reply.getLogIndex();
}
// 然後釋放Commit Index的chunk buffer
adjustBuffers(index);
return reply;
} catch (TimeoutException | InterruptedException | ExecutionException e) {
// 異常處理
releaseBuffersOnException();
throw ioException;
}
}
當然在此過程中,同樣可能會出現執行異常的情況導致Chunk數據寫失敗的情況,這邊BlockOutputStream將只會釋放那些已確保成功寫出的Chunk數據,邏輯如下:
// only contain data which have not been sufficiently replicated
void releaseBuffersOnException() {
// 獲取Datanode上最近一次成功提交的Commit Index,然後釋放這次commit之前的buffer
adjustBuffers(xceiverClient.getReplicatedMinCommitIndex());
}
/**
* 釋放提交成功的Commit Index值之前的buffer.
* @param commitIndex 給定的Commit Index值
*/
private void adjustBuffers(long commitIndex) {
List<Long> keyList = commitIndex2flushedDataMap.keySet().stream()
.filter(p -> p <= commitIndex).collect(Collectors.toList());
if (!keyList.isEmpty()) {
releaseBuffers(keyList);
}
}
此過程的流程圖如下所示:
這裏阻塞等待Commit Index的邏輯取決於當前的Chunk Buffer是否是滿的狀態,如果是滿的,則等待最小的Commit Index,即阻塞等待較早提交的PutBlock請求執行結束,否則等待最新(近)的PutBlock調用(最大Commit Index)結束。
private void watchForCommit(boolean bufferFull) throws IOException {
checkOpen();
try {
// 根據buffer是否是滿的狀態,來進行對應Commit Index的阻塞等待
XceiverClientReply reply = bufferFull ?
commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
if (reply != null) {
//...
}
} catch (IOException ioe) {
setIoException(ioe);
throw getIoException();
}
}
總結
這裏我們簡單總結幾點以上Key數據寫出的過程裏,Ozone在寫數據原子性上做了哪些特別的處理:
- 多副本Pipeline裏的Datanode的數據一致性通過Ratis(Raft)協議實現來保證
- 確保每次Block寫出的數據是有效的,成功的
- 單個Block內部每次寫出的Chunk數據只有被成功寫出得到Datanode ack回覆後,纔會更新Datanode上對應的Container db文件,然後這部分數據纔會被外部視爲可見的數據。倘若中間block數據發生異常,中間的chunk信息也不會在Container db中存在。
因此我們可以看到,Container db的更新在這裏是起到一個很關鍵的作用,確保了數據正確地存在於Datanode Container之上。