diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java index 549735438a02..d1992ac931e5 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java @@ -247,6 +247,23 @@ public enum ChecksumCombineMode { tags = ConfigTag.CLIENT) private String fsDefaultBucketLayout = "FILE_SYSTEM_OPTIMIZED"; + @Config(key = "incremental.chunk.list", + defaultValue = "false", + type = ConfigType.BOOLEAN, + description = "Client PutBlock request can choose incremental chunk " + + "list rather than full chunk list to optimize performance. " + + "Critical to HBase.", + tags = ConfigTag.CLIENT) + private boolean incrementalChunkList = true; + + @Config(key = "stream.putblock.piggybacking", + defaultValue = "false", + type = ConfigType.BOOLEAN, + description = "Allow PutBlock to be piggybacked in WriteChunk " + + "requests if the chunk is small.", + tags = ConfigTag.CLIENT) + private boolean enablePutblockPiggybacking = false; + @PostConstruct public void validate() { Preconditions.checkState(streamBufferSize > 0); @@ -445,6 +462,14 @@ public String getFsDefaultBucketLayout() { return fsDefaultBucketLayout; } + public void setEnablePutblockPiggybacking(boolean enablePutblockPiggybacking) { + this.enablePutblockPiggybacking = enablePutblockPiggybacking; + } + + public boolean getEnablePutblockPiggybacking() { + return enablePutblockPiggybacking; + } + public boolean isDatastreamPipelineMode() { return datastreamPipelineMode; } @@ -452,4 +477,12 @@ public boolean isDatastreamPipelineMode() { public void setDatastreamPipelineMode(boolean datastreamPipelineMode) { this.datastreamPipelineMode = datastreamPipelineMode; } + + public void setIncrementalChunkList(boolean enable) { + this.incrementalChunkList = enable; + } + + public boolean getIncrementalChunkList() { + return this.incrementalChunkList; + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 2b5854ca2086..cb2b85ef1e29 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -449,7 +449,7 @@ private XceiverClientReply sendCommandWithRetry( LOG.debug(message + " on the pipeline {}.", processForDebug(request), pipeline); } else { - LOG.error(message + " on the pipeline {}.", + LOG.warn(message + " on the pipeline {}.", request.getCmdType(), pipeline); } throw ioException; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index b62415395df4..374e90a24c76 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -21,6 +21,7 @@ import java.io.EOFException; import java.io.IOException; import java.io.InputStream; +import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -32,6 +33,7 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetBlockResponseProto; @@ -63,11 +65,12 @@ */ public class BlockInputStream extends BlockExtendedInputStream { - private static final Logger LOG = + public static final Logger LOG = LoggerFactory.getLogger(BlockInputStream.class); private final BlockID blockID; - private final long length; + private long length; + private final BlockLocationInfo blockInfo; private final AtomicReference pipelineRef = new AtomicReference<>(); private final AtomicReference> tokenRef = @@ -112,13 +115,16 @@ public class BlockInputStream extends BlockExtendedInputStream { private final Function refreshFunction; - public BlockInputStream(BlockID blockId, long blockLen, Pipeline pipeline, + public BlockInputStream( + BlockLocationInfo blockInfo, + Pipeline pipeline, Token token, XceiverClientFactory xceiverClientFactory, Function refreshFunction, OzoneClientConfig config) { - this.blockID = blockId; - this.length = blockLen; + this.blockInfo = blockInfo; + this.blockID = blockInfo.getBlockID(); + this.length = blockInfo.getLength(); setPipeline(pipeline); tokenRef.set(token); this.verifyChecksum = config.isChecksumVerify(); @@ -129,14 +135,16 @@ public BlockInputStream(BlockID blockId, long blockLen, Pipeline pipeline, TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); } + // only for unit tests public BlockInputStream(BlockID blockId, long blockLen, Pipeline pipeline, Token token, XceiverClientFactory xceiverClientFactory, OzoneClientConfig config ) { - this(blockId, blockLen, pipeline, token, - xceiverClientFactory, null, config); + this(new BlockLocationInfo(new BlockLocationInfo.Builder().setBlockID(blockId).setLength(blockLen)), + pipeline, token, xceiverClientFactory, null, config); } + /** * Initialize the BlockInputStream. Get the BlockData (list of chunks) from * the Container and create the ChunkInputStreams for each Chunk in the Block. @@ -148,11 +156,17 @@ public synchronized void initialize() throws IOException { return; } + BlockData blockData = null; List chunks = null; IOException catchEx = null; do { try { - chunks = getChunkInfoList(); + blockData = getBlockData(); + chunks = blockData.getChunksList(); + if (blockInfo != null && blockInfo.isUnderConstruction()) { + // use the block length from DN if block is under construction. + length = blockData.getSize(); + } break; // If we get a StorageContainerException or an IOException due to // datanodes are not reachable, refresh to get the latest pipeline @@ -211,18 +225,25 @@ private boolean isConnectivityIssue(IOException ex) { } private void refreshBlockInfo(IOException cause) throws IOException { - LOG.info("Unable to read information for block {} from pipeline {}: {}", + LOG.info("Attempting to update pipeline and block token for block {} from pipeline {}: {}", blockID, pipelineRef.get().getId(), cause.getMessage()); if (refreshFunction != null) { LOG.debug("Re-fetching pipeline and block token for block {}", blockID); BlockLocationInfo blockLocationInfo = refreshFunction.apply(blockID); if (blockLocationInfo == null) { - LOG.debug("No new block location info for block {}", blockID); + LOG.warn("No new block location info for block {}", blockID); } else { - LOG.debug("New pipeline for block {}: {}", blockID, - blockLocationInfo.getPipeline()); setPipeline(blockLocationInfo.getPipeline()); + LOG.info("New pipeline for block {}: {}", blockID, + blockLocationInfo.getPipeline()); + tokenRef.set(blockLocationInfo.getToken()); + if (blockLocationInfo.getToken() != null) { + OzoneBlockTokenIdentifier tokenId = new OzoneBlockTokenIdentifier(); + tokenId.readFromByteArray(tokenRef.get().getIdentifier()); + LOG.info("A new token is added for block {}. Expiry: {}", + blockID, Instant.ofEpochMilli(tokenId.getExpiryDate())); + } } } else { throw cause; @@ -231,24 +252,27 @@ private void refreshBlockInfo(IOException cause) throws IOException { /** * Send RPC call to get the block info from the container. - * @return List of chunks in this block. + * @return BlockData. */ - protected List getChunkInfoList() throws IOException { + protected BlockData getBlockData() throws IOException { acquireClient(); try { - return getChunkInfoListUsingClient(); + return getBlockDataUsingClient(); } finally { releaseClient(); } } - @VisibleForTesting - protected List getChunkInfoListUsingClient() throws IOException { + /** + * Send RPC call to get the block info from the container. + * @return BlockData. + */ + protected BlockData getBlockDataUsingClient() throws IOException { final Pipeline pipeline = xceiverClient.getPipeline(); if (LOG.isDebugEnabled()) { - LOG.debug("Initializing BlockInputStream for get key to access {}", - blockID.getContainerID()); + LOG.debug("Initializing BlockInputStream for get key to access block {}", + blockID); } DatanodeBlockID.Builder blkIDBuilder = @@ -263,8 +287,7 @@ protected List getChunkInfoListUsingClient() throws IOException { GetBlockResponseProto response = ContainerProtocolCalls.getBlock( xceiverClient, VALIDATORS, blkIDBuilder.build(), tokenRef.get()); - - return response.getBlockData().getChunksList(); + return response.getBlockData(); } private void setPipeline(Pipeline pipeline) { @@ -566,7 +589,20 @@ private boolean shouldRetryRead(IOException cause) throws IOException { } catch (Exception e) { throw new IOException(e); } - return retryAction.action == RetryPolicy.RetryAction.RetryDecision.RETRY; + if (retryAction.action == RetryPolicy.RetryAction.RetryDecision.RETRY) { + if (retryAction.delayMillis > 0) { + try { + LOG.debug("Retry read after {}ms", retryAction.delayMillis); + Thread.sleep(retryAction.delayMillis); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + String msg = "Interrupted: action=" + retryAction.action + ", retry policy=" + retryPolicy; + throw new IOException(msg, e); + } + } + return true; + } + return false; } private void handleReadError(IOException cause) throws IOException { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java index 5c0516d7bd4f..f29bf490382f 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hdds.scm.storage; import java.io.IOException; import java.io.OutputStream; +import java.nio.BufferOverflowException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -53,6 +55,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; + +import static org.apache.hadoop.hdds.DatanodeVersion.COMBINED_PUTBLOCK_WRITECHUNK_RPC; import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync; import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.writeChunkAsync; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; @@ -80,6 +84,12 @@ public class BlockOutputStream extends OutputStream { LoggerFactory.getLogger(BlockOutputStream.class); public static final String EXCEPTION_MSG = "Unexpected Storage Container Exception: "; + public static final String INCREMENTAL_CHUNK_LIST = "incremental"; + public static final KeyValue INCREMENTAL_CHUNK_LIST_KV = + KeyValue.newBuilder().setKey(INCREMENTAL_CHUNK_LIST).build(); + public static final String FULL_CHUNK = "full"; + public static final KeyValue FULL_CHUNK_KV = + KeyValue.newBuilder().setKey(FULL_CHUNK).build(); private AtomicReference blockID; private final AtomicReference previousChunkInfo @@ -123,11 +133,16 @@ public class BlockOutputStream extends OutputStream { private int currentBufferRemaining; //current buffer allocated to write private ChunkBuffer currentBuffer; + // last chunk holds the buffer after the last complete chunk, which may be + // different from currentBuffer. We need this to calculate checksum. + private ByteBuffer lastChunkBuffer; + private long lastChunkOffset; private final Token token; private final String tokenString; private int replicationIndex; private Pipeline pipeline; private final ContainerClientMetrics clientMetrics; + private boolean allowPutBlockPiggybacking; /** * Creates a new BlockOutputStream. @@ -165,6 +180,13 @@ public BlockOutputStream( } this.containerBlockData = BlockData.newBuilder().setBlockID( blkIDBuilder.build()).addMetadata(keyValue); + // tell DataNode I will send incremental chunk list + if (config.getIncrementalChunkList()) { + this.containerBlockData.addMetadata(INCREMENTAL_CHUNK_LIST_KV); + this.lastChunkBuffer = + ByteBuffer.allocate(config.getStreamBufferSize()); + this.lastChunkOffset = 0; + } this.xceiverClient = xceiverClientManager.acquireClient(pipeline); this.bufferPool = bufferPool; this.token = token; @@ -192,6 +214,20 @@ public BlockOutputStream( this.clientMetrics = clientMetrics; this.pipeline = pipeline; this.streamBufferArgs = streamBufferArgs; + this.allowPutBlockPiggybacking = config.getEnablePutblockPiggybacking() && + allDataNodesSupportPiggybacking(); + } + + private boolean allDataNodesSupportPiggybacking() { + // return true only if all DataNodes in the pipeline are on a version + // that supports PutBlock piggybacking. + for (DatanodeDetails dn : pipeline.getNodes()) { + if (dn.getCurrentVersion() < + COMBINED_PUTBLOCK_WRITECHUNK_RPC.toProtoValue()) { + return false; + } + } + return true; } void refreshCurrentBuffer() { @@ -460,6 +496,14 @@ ContainerCommandResponseProto> executePutBlock(boolean close, ContainerCommandResponseProto> flushFuture = null; try { BlockData blockData = containerBlockData.build(); + LOG.debug("sending PutBlock {}", blockData); + + if (config.getIncrementalChunkList()) { + // remove any chunks in the containerBlockData list. + // since they are sent. + containerBlockData.clearChunks(); + } + XceiverClientReply asyncReply = putBlockAsync(xceiverClient, blockData, close, tokenString); CompletableFuture future = @@ -472,22 +516,8 @@ ContainerCommandResponseProto> executePutBlock(boolean close, } // if the ioException is not set, putBlock is successful if (getIoException() == null && !force) { - BlockID responseBlockID = BlockID.getFromProtobuf( - e.getPutBlock().getCommittedBlockLength().getBlockID()); - Preconditions.checkState(blockID.get().getContainerBlockID() - .equals(responseBlockID.getContainerBlockID())); - // updates the bcsId of the block - blockID.set(responseBlockID); - if (LOG.isDebugEnabled()) { - LOG.debug( - "Adding index " + asyncReply.getLogIndex() + " flushLength " - + flushPos + " numBuffers " + byteBufferList.size() - + " blockID " + blockID + " bufferPool size" + bufferPool - .getSize() + " currentBufferIndex " + bufferPool - .getCurrentBufferIndex()); - } - // for standalone protocol, logIndex will always be 0. - updateCommitInfo(asyncReply, byteBufferList); + handleSuccessfulPutBlock(e.getPutBlock().getCommittedBlockLength(), + asyncReply, flushPos, byteBufferList); } return e; }, responseExecutor).exceptionally(e -> { @@ -524,7 +554,7 @@ public void flush() throws IOException { } } - private void writeChunk(ChunkBuffer buffer) + private void writeChunkCommon(ChunkBuffer buffer) throws IOException { // This data in the buffer will be pushed to datanode and a reference will // be added to the bufferList. Once putBlock gets executed, this list will @@ -535,7 +565,18 @@ private void writeChunk(ChunkBuffer buffer) bufferList = new ArrayList<>(); } bufferList.add(buffer); - writeChunkToContainer(buffer.duplicate(0, buffer.position())); + } + + private void writeChunk(ChunkBuffer buffer) + throws IOException { + writeChunkCommon(buffer); + writeChunkToContainer(buffer.duplicate(0, buffer.position()), false); + } + + private void writeChunkAndPutBlock(ChunkBuffer buffer) + throws IOException { + writeChunkCommon(buffer); + writeChunkToContainer(buffer.duplicate(0, buffer.position()), true); } /** @@ -567,14 +608,23 @@ private void handleFlushInternal(boolean close) if (totalDataFlushedLength < writtenDataLength) { refreshCurrentBuffer(); Preconditions.checkArgument(currentBuffer.position() > 0); - if (currentBuffer.hasRemaining()) { - writeChunk(currentBuffer); - } + // This can be a partially filled chunk. Since we are flushing the buffer // here, we just limit this buffer to the current position. So that next // write will happen in new buffer - updateFlushLength(); - executePutBlock(close, false); + if (currentBuffer.hasRemaining()) { + if (allowPutBlockPiggybacking) { + updateFlushLength(); + writeChunkAndPutBlock(currentBuffer); + } else { + writeChunk(currentBuffer); + updateFlushLength(); + executePutBlock(close, false); + } + } else { + updateFlushLength(); + executePutBlock(close, false); + } } else if (close) { // forcing an "empty" putBlock if stream is being closed without new // data since latest flush - we need to send the "EOF" flag @@ -686,7 +736,7 @@ public boolean isClosed() { * @return */ CompletableFuture writeChunkToContainer( - ChunkBuffer chunk) throws IOException { + ChunkBuffer chunk, boolean putBlockPiggybacking) throws IOException { int effectiveChunkSize = chunk.remaining(); final long offset = chunkOffset.getAndAdd(effectiveChunkSize); final ByteString data = chunk.toByteString( @@ -699,6 +749,8 @@ CompletableFuture writeChunkToContainer( .setChecksumData(checksumData.getProtoBufMessage()) .build(); + long flushPos = totalDataFlushedLength; + if (LOG.isDebugEnabled()) { LOG.debug("Writing chunk {} length {} at offset {}", chunkInfo.getChunkName(), effectiveChunkSize, offset); @@ -716,37 +768,252 @@ CompletableFuture writeChunkToContainer( + ", previous = " + previous); } + final List byteBufferList; + CompletableFuture + validateFuture = null; try { + BlockData blockData = null; + + if (config.getIncrementalChunkList()) { + updateBlockDataForWriteChunk(chunk); + } else { + containerBlockData.addChunks(chunkInfo); + } + if (putBlockPiggybacking) { + Preconditions.checkNotNull(bufferList); + byteBufferList = bufferList; + bufferList = null; + Preconditions.checkNotNull(byteBufferList); + + blockData = containerBlockData.build(); + LOG.debug("piggyback chunk list {}", blockData); + + if (config.getIncrementalChunkList()) { + // remove any chunks in the containerBlockData list. + // since they are sent. + containerBlockData.clearChunks(); + } + } else { + byteBufferList = null; + } + XceiverClientReply asyncReply = writeChunkAsync(xceiverClient, chunkInfo, - blockID.get(), data, tokenString, replicationIndex); + blockID.get(), data, tokenString, replicationIndex, blockData); CompletableFuture respFuture = asyncReply.getResponse(); - CompletableFuture - validateFuture = respFuture.thenApplyAsync(e -> { - try { - validateResponse(e); - } catch (IOException sce) { - respFuture.completeExceptionally(sce); - } - return e; - }, responseExecutor).exceptionally(e -> { - String msg = "Failed to write chunk " + chunkInfo.getChunkName() + - " into block " + blockID; - LOG.debug("{}, exception: {}", msg, e.getLocalizedMessage()); - CompletionException ce = new CompletionException(msg, e); - setIoException(ce); - throw ce; - }); - containerBlockData.addChunks(chunkInfo); + validateFuture = respFuture.thenApplyAsync(e -> { + try { + validateResponse(e); + } catch (IOException sce) { + respFuture.completeExceptionally(sce); + } + // if the ioException is not set, putBlock is successful + if (getIoException() == null && putBlockPiggybacking) { + handleSuccessfulPutBlock(e.getWriteChunk().getCommittedBlockLength(), + asyncReply, flushPos, byteBufferList); + } + return e; + }, responseExecutor).exceptionally(e -> { + String msg = "Failed to write chunk " + chunkInfo.getChunkName() + + " into block " + blockID; + LOG.debug("{}, exception: {}", msg, e.getLocalizedMessage()); + CompletionException ce = new CompletionException(msg, e); + setIoException(ce); + throw ce; + }); clientMetrics.recordWriteChunk(pipeline, chunkInfo.getLen()); - return validateFuture; + } catch (IOException | ExecutionException e) { throw new IOException(EXCEPTION_MSG + e.toString(), e); } catch (InterruptedException ex) { Thread.currentThread().interrupt(); handleInterruptedException(ex, false); } - return null; + if (putBlockPiggybacking) { + putFlushFuture(flushPos, validateFuture); + } + return validateFuture; + } + + private void handleSuccessfulPutBlock( + ContainerProtos.GetCommittedBlockLengthResponseProto e, + XceiverClientReply asyncReply, long flushPos, + List byteBufferList) { + BlockID responseBlockID = BlockID.getFromProtobuf( + e.getBlockID()); + Preconditions.checkState(blockID.get().getContainerBlockID() + .equals(responseBlockID.getContainerBlockID())); + // updates the bcsId of the block + blockID.set(responseBlockID); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Adding index " + asyncReply.getLogIndex() + " flushLength " + + flushPos + " numBuffers " + byteBufferList.size() + + " blockID " + blockID + " bufferPool size" + bufferPool + .getSize() + " currentBufferIndex " + bufferPool + .getCurrentBufferIndex()); + } + // for standalone protocol, logIndex will always be 0. + updateCommitInfo(asyncReply, byteBufferList); + } + + /** + * Update container block data, which is later sent to DataNodes via PutBlock, + * using the new chunks sent out via WriteChunk. + * + * This method is only used when incremental chunk list is enabled. + * @param chunk the chunk buffer to be sent out by WriteChunk. + * @throws OzoneChecksumException + */ + private void updateBlockDataForWriteChunk(ChunkBuffer chunk) + throws OzoneChecksumException { + // Update lastChunkBuffer using the new chunk data. + // This is used to calculate checksum for the last partial chunk in + // containerBlockData which will used by PutBlock. + + // the last partial chunk in containerBlockData will be replaced. + // So remove it. + removeLastPartialChunk(); + chunk.rewind(); + LOG.debug("Adding chunk pos {} limit {} remaining {}." + + "lastChunkBuffer pos {} limit {} remaining {} lastChunkOffset = {}", + chunk.position(), chunk.limit(), chunk.remaining(), + lastChunkBuffer.position(), lastChunkBuffer.limit(), + lastChunkBuffer.remaining(), lastChunkOffset); + + // Append the chunk to the last chunk buffer. + // if the resulting size exceeds limit (4MB), + // drop the full chunk and keep the rest. + if (lastChunkBuffer.position() + chunk.remaining() <= + lastChunkBuffer.capacity()) { + appendLastChunkBuffer(chunk, 0, chunk.remaining()); + } else { + int remainingBufferSize = + lastChunkBuffer.capacity() - lastChunkBuffer.position(); + appendLastChunkBuffer(chunk, 0, remainingBufferSize); + updateBlockDataWithLastChunkBuffer(); + appendLastChunkBuffer(chunk, remainingBufferSize, + chunk.remaining() - remainingBufferSize); + } + LOG.debug("after append, lastChunkBuffer={} lastChunkOffset={}", + lastChunkBuffer, lastChunkOffset); + + updateBlockDataWithLastChunkBuffer(); + } + + private void updateBlockDataWithLastChunkBuffer() + throws OzoneChecksumException { + // create chunk info for lastChunkBuffer + ChunkInfo lastChunkInfo = createChunkInfo(lastChunkOffset); + LOG.debug("lastChunkInfo = {}", lastChunkInfo); + long lastChunkSize = lastChunkInfo.getLen(); + addToBlockData(lastChunkInfo); + + lastChunkBuffer.clear(); + if (lastChunkSize == config.getStreamBufferSize()) { + lastChunkOffset += config.getStreamBufferSize(); + } else { + lastChunkBuffer.position((int) lastChunkSize); + } + } + + private void appendLastChunkBuffer(ChunkBuffer chunkBuffer, int offset, + int length) { + LOG.debug("copying to last chunk buffer offset={} length={}", + offset, length); + int pos = 0; + int uncopied = length; + for (ByteBuffer bb : chunkBuffer.asByteBufferList()) { + if (pos + bb.remaining() >= offset) { + int copyStart = offset < pos ? 0 : offset - pos; + int copyLen = Math.min(uncopied, bb.remaining()); + try { + LOG.debug("put into last chunk buffer start = {} len = {}", + copyStart, copyLen); + int origPos = bb.position(); + int origLimit = bb.limit(); + bb.position(copyStart).limit(copyStart + copyLen); + lastChunkBuffer.put(bb); + bb.position(origPos).limit(origLimit); + } catch (BufferOverflowException e) { + LOG.error("appending from " + copyStart + " for len=" + copyLen + + ". lastChunkBuffer remaining=" + lastChunkBuffer.remaining() + + " pos=" + lastChunkBuffer.position() + + " limit=" + lastChunkBuffer.limit() + + " capacity=" + lastChunkBuffer.capacity()); + throw e; + } + + uncopied -= copyLen; + } + + pos += bb.remaining(); + if (pos >= offset + length) { + return; + } + if (uncopied == 0) { + return; + } + } + } + + private void removeLastPartialChunk() { + // remove the last chunk if it's partial. + if (containerBlockData.getChunksList().isEmpty()) { + return; + } + int lastChunkIndex = containerBlockData.getChunksCount() - 1; + ChunkInfo lastChunkInBlockData = containerBlockData.getChunks( + lastChunkIndex); + if (!isFullChunk(lastChunkInBlockData)) { + containerBlockData.removeChunks(lastChunkIndex); + } + } + + private ChunkInfo createChunkInfo(long lastPartialChunkOffset) + throws OzoneChecksumException { + lastChunkBuffer.flip(); + int revisedChunkSize = lastChunkBuffer.remaining(); + // create the chunk info to be sent in PutBlock. + ChecksumData revisedChecksumData = + checksum.computeChecksum(lastChunkBuffer); + + long chunkID = lastPartialChunkOffset / config.getStreamBufferSize(); + ChunkInfo.Builder revisedChunkInfo = ChunkInfo.newBuilder() + .setChunkName(blockID.get().getLocalID() + "_chunk_" + chunkID) + .setOffset(lastPartialChunkOffset) + .setLen(revisedChunkSize) + .setChecksumData(revisedChecksumData.getProtoBufMessage()); + // if full chunk + if (revisedChunkSize == config.getStreamBufferSize()) { + revisedChunkInfo.addMetadata(FULL_CHUNK_KV); + } + return revisedChunkInfo.build(); + } + + private boolean isFullChunk(ChunkInfo chunkInfo) { + Preconditions.checkState( + chunkInfo.getLen() <= config.getStreamBufferSize()); + return chunkInfo.getLen() == config.getStreamBufferSize(); + } + + private void addToBlockData(ChunkInfo revisedChunkInfo) { + LOG.debug("containerBlockData chunk: {}", containerBlockData); + if (containerBlockData.getChunksCount() > 0) { + ChunkInfo lastChunk = containerBlockData.getChunks( + containerBlockData.getChunksCount() - 1); + LOG.debug("revisedChunkInfo chunk: {}", revisedChunkInfo); + Preconditions.checkState(lastChunk.getOffset() + lastChunk.getLen() == + revisedChunkInfo.getOffset(), + "lastChunk.getOffset() + lastChunk.getLen() " + + "!= revisedChunkInfo.getOffset()"); + } + containerBlockData.addChunks(revisedChunkInfo); + } + + @VisibleForTesting + public void setXceiverClient(XceiverClientSpi xceiverClient) { + this.xceiverClient = xceiverClient; } /** diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java index adecc3e4c1e2..c8bfaf3e1bce 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java @@ -89,13 +89,14 @@ public ECBlockOutputStream( @Override public void write(byte[] b, int off, int len) throws IOException { this.currentChunkRspFuture = - writeChunkToContainer(ChunkBuffer.wrap(ByteBuffer.wrap(b, off, len))); + writeChunkToContainer( + ChunkBuffer.wrap(ByteBuffer.wrap(b, off, len)), false); updateWrittenDataLength(len); } public CompletableFuture write( ByteBuffer buff) throws IOException { - return writeChunkToContainer(ChunkBuffer.wrap(buff)); + return writeChunkToContainer(ChunkBuffer.wrap(buff), false); } public CompletableFuture chunkList, Map chunks, OzoneClientConfig config) { - super(blockId, blockLen, pipeline, token, + super(new BlockLocationInfo(new BlockLocationInfo.Builder().setBlockID(blockId).setLength(blockLen)), + pipeline, token, xceiverClientManager, refreshFunction, config); this.chunkDataMap = chunks; this.chunks = chunkList; @@ -58,8 +60,10 @@ class DummyBlockInputStream extends BlockInputStream { } @Override - protected List getChunkInfoList() throws IOException { - return chunks; + protected ContainerProtos.BlockData getBlockData() throws IOException { + BlockID blockID = getBlockID(); + ContainerProtos.DatanodeBlockID datanodeBlockID = blockID.getDatanodeBlockIDProtobuf(); + return ContainerProtos.BlockData.newBuilder().addAllChunks(chunks).setBlockID(datanodeBlockID).build(); } @Override diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java index 6d12614228f9..172e62887bdd 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; @@ -74,16 +75,16 @@ final class DummyBlockInputStreamWithRetry } @Override - protected List getChunkInfoList() throws IOException { + protected ContainerProtos.BlockData getBlockData() throws IOException { if (getChunkInfoCount == 0) { getChunkInfoCount++; if (ioException != null) { - throw ioException; + throw ioException; } throw new StorageContainerException("Exception encountered", CONTAINER_NOT_FOUND); } else { - return super.getChunkInfoList(); + return super.getBlockData(); } } } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java index 21b088ce85fd..0012d691f92d 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.scm.OzoneClientConfig; @@ -35,6 +36,7 @@ import org.apache.hadoop.ozone.common.Checksum; import org.apache.hadoop.ozone.common.OzoneChecksumException; +import org.apache.ozone.test.GenericTestUtils; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusException; import org.junit.jupiter.api.BeforeEach; @@ -43,6 +45,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.mockito.stubbing.OngoingStubbing; +import org.slf4j.event.Level; import java.io.EOFException; import java.io.IOException; @@ -59,6 +62,7 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; import static org.apache.hadoop.hdds.scm.storage.TestChunkInputStream.generateRandomData; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -262,6 +266,9 @@ public void testSeekAndRead() throws Exception { @Test public void testRefreshPipelineFunction() throws Exception { + GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer + .captureLogs(BlockInputStream.LOG); + GenericTestUtils.setLogLevel(BlockInputStream.LOG, Level.DEBUG); BlockID blockID = new BlockID(new ContainerBlockID(1, 1)); AtomicBoolean isRefreshed = new AtomicBoolean(); createChunkList(5); @@ -277,6 +284,7 @@ public void testRefreshPipelineFunction() throws Exception { seekAndVerify(50); byte[] b = new byte[200]; blockInputStreamWithRetry.read(b, 0, 200); + assertThat(logCapturer.getOutput()).contains("Retry read after"); assertTrue(isRefreshed.get()); } } @@ -413,17 +421,20 @@ public void testRefreshOnReadFailureAfterUnbuffer(IOException ex) OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setChecksumVerify(false); - BlockInputStream subject = new BlockInputStream(blockID, blockSize, + BlockInputStream subject = new BlockInputStream( + new BlockLocationInfo(new BlockLocationInfo.Builder().setBlockID(blockID).setLength(blockSize)), pipeline, null, clientFactory, refreshFunction, clientConfig) { @Override - protected List getChunkInfoListUsingClient() { - return chunks; + protected ChunkInputStream createChunkInputStream(ChunkInfo chunkInfo) { + return stream; } @Override - protected ChunkInputStream createChunkInputStream(ChunkInfo chunkInfo) { - return stream; + protected ContainerProtos.BlockData getBlockDataUsingClient() throws IOException { + BlockID blockID = getBlockID(); + ContainerProtos.DatanodeBlockID datanodeBlockID = blockID.getDatanodeBlockIDProtobuf(); + return ContainerProtos.BlockData.newBuilder().addAllChunks(chunks).setBlockID(datanodeBlockID).build(); } }; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java index 30f9df597b51..e35d20d53e15 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java @@ -31,6 +31,8 @@ public enum DatanodeVersion implements ComponentVersion { DEFAULT_VERSION(0, "Initial version"), SEPARATE_RATIS_PORTS_AVAILABLE(1, "Version with separated Ratis port."), + COMBINED_PUTBLOCK_WRITECHUNK_RPC(2, "WriteChunk can optionally support " + + "a PutBlock request"), FUTURE_VERSION(-1, "Used internally in the client when the server side is " + " newer and an unknown server version has arrived to the client."); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java index f5a00e2dbe46..e1188f1cd1e6 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java @@ -424,6 +424,7 @@ public static boolean isReadOnly( case ListContainer: case ListChunk: case GetCommittedBlockLength: + case Echo: return true; case CloseContainer: case WriteChunk: @@ -437,6 +438,7 @@ public static boolean isReadOnly( case PutSmallFile: case StreamInit: case StreamWrite: + case FinalizeBlock: default: return false; } @@ -475,6 +477,7 @@ public static boolean requireBlockToken( case PutSmallFile: case ReadChunk: case WriteChunk: + case FinalizeBlock: return true; default: return false; @@ -554,6 +557,11 @@ public static BlockID getBlockID(ContainerCommandRequestProtoOrBuilder msg) { blockID = msg.getWriteChunk().getBlockID(); } break; + case FinalizeBlock: + if (msg.hasFinalizeBlock()) { + blockID = msg.getFinalizeBlock().getBlockID(); + } + break; default: break; } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java index b455daba529f..49f690f7b8c1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java @@ -358,6 +358,9 @@ public static DatanodeDetails.Builder newBuilder( builder.setPersistedOpStateExpiry( datanodeDetailsProto.getPersistedOpStateExpiry()); } + if (datanodeDetailsProto.hasCurrentVersion()) { + builder.setCurrentVersion(datanodeDetailsProto.getCurrentVersion()); + } return builder; } @@ -465,6 +468,8 @@ public HddsProtos.DatanodeDetailsProto.Builder toProtoBuilder( } } + builder.setCurrentVersion(currentVersion); + return builder; } @@ -495,6 +500,7 @@ public ExtendedDatanodeDetailsProto getExtendedProtoBufMessage() { } /** + * Note: Datanode initial version is not passed to the client due to no use case. See HDDS-9884 * @return the version this datanode was initially created with */ public int getInitialVersion() { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java index d8fdbc1063a9..b7f1d1e61500 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java @@ -145,6 +145,10 @@ public final class ScmConfigKeys { public static final String OZONE_CHUNK_READ_MAPPED_BUFFER_THRESHOLD_DEFAULT = "32KB"; + public static final String OZONE_CHUNK_LIST_INCREMENTAL = + "ozone.incremental.chunk.list"; + public static final boolean OZONE_CHUNK_LIST_INCREMENTAL_DEFAULT = true; + public static final String OZONE_SCM_CONTAINER_LAYOUT_KEY = "ozone.scm.container.layout"; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index 73369fc9a38c..d3f39c023b73 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -21,6 +21,8 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.function.Function; + +import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; @@ -38,10 +40,12 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ListBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations; import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion; @@ -210,6 +214,28 @@ public static ContainerCommandResponseProto getPutFileResponseSuccess( .build(); } + /** + * Gets a response for the WriteChunk RPC. + * @param msg - ContainerCommandRequestProto + * @return - ContainerCommandResponseProto + */ + public static ContainerCommandResponseProto getWriteChunkResponseSuccess( + ContainerCommandRequestProto msg, BlockData blockData) { + + WriteChunkResponseProto.Builder writeChunk = + WriteChunkResponseProto.newBuilder(); + if (blockData != null) { + writeChunk.setCommittedBlockLength( + getCommittedBlockLengthResponseBuilder( + blockData.getSize(), blockData.getBlockID())); + + } + return getSuccessResponseBuilder(msg) + .setCmdType(Type.WriteChunk) + .setWriteChunk(writeChunk) + .build(); + } + /** * Gets a response to the read small file call. * @param request - Msg @@ -307,6 +333,43 @@ public static ContainerCommandResponseProto getReadChunkResponse( .build(); } + public static ContainerCommandResponseProto getFinalizeBlockResponse( + ContainerCommandRequestProto msg, BlockData data) { + + ContainerProtos.FinalizeBlockResponseProto.Builder blockData = + ContainerProtos.FinalizeBlockResponseProto.newBuilder() + .setBlockData(data); + + return getSuccessResponseBuilder(msg) + .setFinalizeBlock(blockData) + .build(); + } + + public static ContainerCommandResponseProto getEchoResponse( + ContainerCommandRequestProto msg) { + + ContainerProtos.EchoRequestProto echoRequest = msg.getEcho(); + int responsePayload = echoRequest.getPayloadSizeResp(); + + int sleepTimeMs = echoRequest.getSleepTimeMs(); + try { + if (sleepTimeMs > 0) { + Thread.sleep(sleepTimeMs); + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + + ContainerProtos.EchoResponseProto.Builder echo = + ContainerProtos.EchoResponseProto + .newBuilder() + .setPayload(UnsafeByteOperations.unsafeWrap(RandomUtils.nextBytes(responsePayload))); + + return getSuccessResponseBuilder(msg) + .setEcho(echo) + .build(); + } + private ContainerCommandResponseBuilders() { throw new UnsupportedOperationException("no instances"); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java index 019e16c2f13f..a6b291c3f48a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java @@ -40,6 +40,8 @@ public class BlockLocationInfo { // PartNumber is set for Multipart upload Keys. private int partNumber; + // The block is under construction. Apply to hsynced file last block. + private boolean underConstruction; protected BlockLocationInfo(Builder builder) { this.blockID = builder.blockID; @@ -111,6 +113,14 @@ public int getPartNumber() { return partNumber; } + public void setUnderConstruction(boolean uc) { + this.underConstruction = uc; + } + + public boolean isUnderConstruction() { + return this.underConstruction; + } + /** * Builder of BlockLocationInfo. */ diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 72754d1f1cf6..5f94f6d08474 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -59,6 +59,9 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.FinalizeBlockRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.EchoRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.EchoResponseProto; import org.apache.hadoop.hdds.scm.XceiverClientReply; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator; @@ -291,6 +294,36 @@ public static XceiverClientReply putBlockAsync(XceiverClientSpi xceiverClient, return xceiverClient.sendCommandAsync(request); } + /** + * Calls the container protocol to finalize a container block. + * + * @param xceiverClient client to perform call + * @param blockID block ID to identify block + * @param token a token for this block (may be null) + * @return FinalizeBlockResponseProto + * @throws IOException if there is an I/O error while performing the call + */ + public static ContainerProtos.FinalizeBlockResponseProto finalizeBlock( + XceiverClientSpi xceiverClient, DatanodeBlockID blockID, + Token token) + throws IOException { + FinalizeBlockRequestProto.Builder finalizeBlockRequest = + FinalizeBlockRequestProto.newBuilder().setBlockID(blockID); + String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); + ContainerCommandRequestProto.Builder builder = + ContainerCommandRequestProto.newBuilder().setCmdType(Type.FinalizeBlock) + .setContainerID(blockID.getContainerID()) + .setDatanodeUuid(id) + .setFinalizeBlock(finalizeBlockRequest); + if (token != null) { + builder.setEncodedToken(token.encodeToUrlString()); + } + ContainerCommandRequestProto request = builder.build(); + ContainerCommandResponseProto response = + xceiverClient.sendCommand(request, getValidatorList()); + return response.getFinalizeBlock(); + } + public static ContainerCommandRequestProto getPutBlockRequest( Pipeline pipeline, BlockData containerBlockData, boolean eof, String tokenString) throws IOException { @@ -405,8 +438,10 @@ static long getLen(ReadChunkResponseProto response) { */ public static XceiverClientReply writeChunkAsync( XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID, - ByteString data, String tokenString, int replicationIndex) + ByteString data, String tokenString, + int replicationIndex, BlockData blockData) throws IOException, ExecutionException, InterruptedException { + WriteChunkRequestProto.Builder writeChunkRequest = WriteChunkRequestProto.newBuilder() .setBlockID(DatanodeBlockID.newBuilder() @@ -417,6 +452,12 @@ public static XceiverClientReply writeChunkAsync( .build()) .setChunkData(chunk) .setData(data); + if (blockData != null) { + PutBlockRequestProto.Builder createBlockRequest = + PutBlockRequestProto.newBuilder() + .setBlockData(blockData); + writeChunkRequest.setBlock(createBlockRequest); + } String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto.newBuilder() @@ -684,6 +725,41 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client, return response.getGetSmallFile(); } + /** + * Send an echo to DataNode. + * + * @return EchoResponseProto + */ + public static EchoResponseProto echo(XceiverClientSpi client, String encodedContainerID, + long containerID, ByteString payloadReqBytes, int payloadRespSizeKB, int sleepTimeMs) throws IOException { + ContainerProtos.EchoRequestProto getEcho = + EchoRequestProto + .newBuilder() + .setPayload(payloadReqBytes) + .setPayloadSizeResp(payloadRespSizeKB) + .setSleepTimeMs(sleepTimeMs) + .build(); + String id = client.getPipeline().getClosestNode().getUuidString(); + + ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto + .newBuilder() + .setCmdType(Type.Echo) + .setContainerID(containerID) + .setDatanodeUuid(id) + .setEcho(getEcho); + if (!encodedContainerID.isEmpty()) { + builder.setEncodedToken(encodedContainerID); + } + String traceId = TracingUtil.exportCurrentSpan(); + if (traceId != null) { + builder.setTraceID(traceId); + } + ContainerCommandRequestProto request = builder.build(); + ContainerCommandResponseProto response = + client.sendCommand(request, getValidatorList()); + return response.getEcho(); + } + /** * Validates a response from a container protocol call. Any non-successful * return code is mapped to a corresponding exception and thrown. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java index 0e54decb73c2..f4efe1790bf7 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java @@ -41,7 +41,9 @@ public enum HDDSLayoutFeature implements LayoutFeature { WEBUI_PORTS_IN_DATANODEDETAILS(6, "Adding HTTP and HTTPS ports " + "to DatanodeDetails."), HADOOP_PRC_PORTS_IN_DATANODEDETAILS(7, "Adding Hadoop RPC ports " + - "to DatanodeDetails."); + "to DatanodeDetails."), + HBASE_SUPPORT(8, "Datanode RocksDB Schema Version 3 has an extra table " + + "for the last chunk of blocks to support HBase.)"); ////////////////////////////// ////////////////////////////// diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java index c9a3a4085202..070edac15f52 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java @@ -126,7 +126,15 @@ public final class OzoneConfigKeys { public static final String OZONE_FS_HSYNC_ENABLED = "ozone.fs.hsync.enabled"; public static final boolean OZONE_FS_HSYNC_ENABLED_DEFAULT - = false; + = true; + + /** + * hsync lease soft limit. + */ + public static final String OZONE_OM_LEASE_SOFT_LIMIT + = "ozone.om.lease.soft.limit"; + public static final String OZONE_OM_LEASE_SOFT_LIMIT_DEFAULT + = "60s"; /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java index f3c08b252b1f..5d8eb4327413 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java @@ -386,6 +386,8 @@ private OzoneConsts() { /** Metadata stored in OmKeyInfo. */ public static final String HSYNC_CLIENT_ID = "hsyncClientId"; + public static final String LEASE_RECOVERY = "leaseRecovery"; + public static final String FORCE_LEASE_RECOVERY_ENV = "OZONE.CLIENT.RECOVER.LEASE.FORCE"; //GDPR public static final String GDPR_FLAG = "gdprEnabled"; diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 5a4b9a22c887..8687dbf1a52e 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -860,6 +860,17 @@ The default read threshold to use memory mapped buffers. + + ozone.incremental.chunk.list + true + OZONE, CLIENT, DATANODE, PERFORMANCE + + By default, a writer client sends full chunk list of a block when it + sends PutBlock requests. Changing this configuration to true will send + only incremental chunk list which reduces metadata overhead and improves + hsync performance. + + ozone.scm.container.layout FILE_PER_BLOCK @@ -1424,6 +1435,17 @@ + + ozone.om.lease.hard.limit + 7d + OZONE, OM, PERFORMANCE + + Controls how long an open hsync key is considered as active. Specifically, if a hsync key + has been open longer than the value of this config entry, that open hsync key is considered as + expired (e.g. due to client crash). Unit could be defined with postfix (ns,ms,s,m,h,d) + + + ozone.om.open.key.cleanup.limit.per.task 1000 @@ -4130,12 +4152,20 @@ ozone.fs.hsync.enabled - false + true OZONE, CLIENT Enable hsync/hflush. By default they are disabled. + + ozone.om.lease.soft.limit + 60s + OZONE, OM + + Hsync soft limit lease period. + + ozone.recon.scm.snapshot.task.initial.delay diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index 7ade596add7a..92754c9fa694 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -599,6 +599,12 @@ public static ContainerCommandRequestProto getDummyCommandRequestProto( .build()) .build()); break; + case FinalizeBlock: + builder + .setFinalizeBlock(ContainerProtos + .FinalizeBlockRequestProto.newBuilder() + .setBlockID(fakeBlockId).build()); + break; default: fail("Unhandled request type " + cmdType + " in unit test"); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java index f59622cb0faf..bbaf58d36b4f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java @@ -233,7 +233,6 @@ public void start() { datanodeDetails.setRevision( HddsVersionInfo.HDDS_VERSION_INFO.getRevision()); datanodeDetails.setBuildDate(HddsVersionInfo.HDDS_VERSION_INFO.getDate()); - datanodeDetails.setCurrentVersion(DatanodeVersion.CURRENT_VERSION); TracingUtil.initTracing( "HddsDatanodeService." + datanodeDetails.getUuidString() .substring(0, 8), conf); @@ -424,17 +423,19 @@ private DatanodeDetails initializeDatanodeDetails() String idFilePath = HddsServerUtil.getDatanodeIdFilePath(conf); Preconditions.checkNotNull(idFilePath); File idFile = new File(idFilePath); + DatanodeDetails details; if (idFile.exists()) { - return ContainerUtils.readDatanodeDetailsFrom(idFile); + details = ContainerUtils.readDatanodeDetailsFrom(idFile); + // Current version is always overridden to the latest + details.setCurrentVersion(getDefaultCurrentVersion()); } else { // There is no datanode.id file, this might be the first time datanode // is started. - DatanodeDetails details = DatanodeDetails.newBuilder() - .setUuid(UUID.randomUUID()).build(); - details.setInitialVersion(DatanodeVersion.CURRENT_VERSION); - details.setCurrentVersion(DatanodeVersion.CURRENT_VERSION); - return details; + details = DatanodeDetails.newBuilder().setUuid(UUID.randomUUID()).build(); + details.setInitialVersion(getDefaultInitialVersion()); + details.setCurrentVersion(getDefaultCurrentVersion()); } + return details; } /** @@ -678,4 +679,20 @@ private String reconfigReplicationStreamsLimit(String value) { .setPoolSize(Integer.parseInt(value)); return value; } + + /** + * Returns the initial version of the datanode. + */ + @VisibleForTesting + public static int getDefaultInitialVersion() { + return DatanodeVersion.CURRENT_VERSION; + } + + /** + * Returns the current version of the datanode. + */ + @VisibleForTesting + public static int getDefaultCurrentVersion() { + return DatanodeVersion.CURRENT_VERSION; + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java index 73aff9ac830c..f7a38e3dec8b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java @@ -39,7 +39,9 @@ public enum DNAction implements AuditAction { GET_SMALL_FILE, CLOSE_CONTAINER, GET_COMMITTED_BLOCK_LENGTH, - STREAM_INIT; + STREAM_INIT, + FINALIZE_BLOCK, + ECHO; @Override public String getAction() { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java index f8acbc7e2d69..58b1c1d1d5e0 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java @@ -56,7 +56,7 @@ private DatanodeIdYaml() { } /** - * Creates a yaml file using DatnodeDetails. This method expects the path + * Creates a yaml file using DatanodeDetails. This method expects the path * validation to be performed by the caller. * * @param datanodeDetails {@link DatanodeDetails} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java index 12910b40209c..904ec21b5f62 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java @@ -815,6 +815,8 @@ private static DNAction getAuditAction(Type cmdType) { case CloseContainer : return DNAction.CLOSE_CONTAINER; case GetCommittedBlockLength : return DNAction.GET_COMMITTED_BLOCK_LENGTH; case StreamInit : return DNAction.STREAM_INIT; + case FinalizeBlock : return DNAction.FINALIZE_BLOCK; + case Echo : return DNAction.ECHO; default : LOG.debug("Invalid command type - {}", cmdType); return null; @@ -947,6 +949,12 @@ private static Map getAuditParams( .toString()); return auditParams; + case FinalizeBlock: + auditParams.put("blockData", + BlockID.getFromProtobuf(msg.getFinalizeBlock().getBlockID()) + .toString()); + return auditParams; + default : LOG.debug("Invalid command type - {}", cmdType); return null; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java index 2ffb9d30d1f4..bfdff69be46f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java @@ -213,6 +213,10 @@ public abstract void deleteBlock(Container container, BlockData blockData) public abstract void deleteUnreferenced(Container container, long localID) throws IOException; + public abstract void addFinalizedBlock(Container container, long localID); + + public abstract boolean isFinalizedBlockExist(Container container, long localID); + public void setClusterID(String clusterID) { this.clusterId = clusterID; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java index fdbe8c981cb9..9eb5b909ccea 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -422,32 +422,48 @@ public TransactionContext startTransaction(RaftClientRequest request) } return builder.build().setException(ioe); } - if (proto.getCmdType() == Type.WriteChunk) { + + boolean blockAlreadyFinalized = false; + if (proto.getCmdType() == Type.PutBlock) { + blockAlreadyFinalized = shouldRejectRequest(proto.getPutBlock().getBlockData().getBlockID()); + } else if (proto.getCmdType() == Type.WriteChunk) { final WriteChunkRequestProto write = proto.getWriteChunk(); - // create the log entry proto - final WriteChunkRequestProto commitWriteChunkProto = - WriteChunkRequestProto.newBuilder() - .setBlockID(write.getBlockID()) - .setChunkData(write.getChunkData()) - // skipping the data field as it is - // already set in statemachine data proto - .build(); - ContainerCommandRequestProto commitContainerCommandProto = - ContainerCommandRequestProto - .newBuilder(proto) - .setPipelineID(gid.getUuid().toString()) - .setWriteChunk(commitWriteChunkProto) - .setTraceID(proto.getTraceID()) - .build(); - Preconditions.checkArgument(write.hasData()); - Preconditions.checkArgument(!write.getData().isEmpty()); - - final Context context = new Context(proto, commitContainerCommandProto); - return builder - .setStateMachineContext(context) - .setStateMachineData(write.getData()) - .setLogData(commitContainerCommandProto.toByteString()) - .build(); + blockAlreadyFinalized = shouldRejectRequest(write.getBlockID()); + if (!blockAlreadyFinalized) { + // create the log entry proto + final WriteChunkRequestProto commitWriteChunkProto = + WriteChunkRequestProto.newBuilder(write) + // skipping the data field as it is + // already set in statemachine data proto + .clearData() + .build(); + ContainerCommandRequestProto commitContainerCommandProto = + ContainerCommandRequestProto + .newBuilder(proto) + .setPipelineID(gid.getUuid().toString()) + .setWriteChunk(commitWriteChunkProto) + .setTraceID(proto.getTraceID()) + .build(); + Preconditions.checkArgument(write.hasData()); + Preconditions.checkArgument(!write.getData().isEmpty()); + + final Context context = new Context(proto, commitContainerCommandProto); + return builder + .setStateMachineContext(context) + .setStateMachineData(write.getData()) + .setLogData(commitContainerCommandProto.toByteString()) + .build(); + } + } else if (proto.getCmdType() == Type.FinalizeBlock) { + containerController.addFinalizedBlock(proto.getContainerID(), + proto.getFinalizeBlock().getBlockID().getLocalID()); + } + + if (blockAlreadyFinalized) { + TransactionContext transactionContext = builder.build(); + transactionContext.setException(new StorageContainerException("Block already finalized", + ContainerProtos.Result.BLOCK_ALREADY_FINALIZED)); + return transactionContext; } else { final Context context = new Context(proto, proto); return builder @@ -455,7 +471,10 @@ public TransactionContext startTransaction(RaftClientRequest request) .setLogData(proto.toByteString()) .build(); } + } + private boolean shouldRejectRequest(ContainerProtos.DatanodeBlockID blockID) { + return containerController.isFinalizedBlockExist(blockID.getContainerID(), blockID.getLocalID()); } private static ContainerCommandRequestProto getContainerCommandRequestProto( diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java index 838818266757..98d81c15d0ad 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java @@ -76,6 +76,7 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.ERROR_IN_COMPACT_DB; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.ERROR_IN_DB_SYNC; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.INVALID_CONTAINER_STATE; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.IO_EXCEPTION; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST; import static org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil.onFailure; @@ -433,6 +434,12 @@ public void quasiClose() throws StorageContainerException { @Override public void close() throws StorageContainerException { + try (DBHandle db = BlockUtils.getDB(containerData, config)) { + containerData.clearFinalizedBlock(db); + } catch (IOException ex) { + LOG.error("Error in deleting entry from Finalize Block table", ex); + throw new StorageContainerException(ex, IO_EXCEPTION); + } closeAndFlushIfNeeded(containerData::closeContainer); LOG.info("Container {} is closed with bcsId {}.", containerData.getContainerID(), diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java index 7fce70f8e18e..47d4f3f9e70a 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java @@ -41,6 +41,8 @@ import java.io.File; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; import static java.lang.Math.max; @@ -92,6 +94,8 @@ public class KeyValueContainerData extends ContainerData { private long blockCommitSequenceId; + private final Set finalizedBlockSet; + static { // Initialize YAML fields KV_YAML_FIELDS = Lists.newArrayList(); @@ -114,6 +118,7 @@ public KeyValueContainerData(long id, ContainerLayoutVersion layoutVersion, size, originPipelineId, originNodeId); this.numPendingDeletionBlocks = new AtomicLong(0); this.deleteTransactionId = 0; + finalizedBlockSet = ConcurrentHashMap.newKeySet(); } public KeyValueContainerData(KeyValueContainerData source) { @@ -123,6 +128,7 @@ public KeyValueContainerData(KeyValueContainerData source) { this.numPendingDeletionBlocks = new AtomicLong(0); this.deleteTransactionId = 0; this.schemaVersion = source.getSchemaVersion(); + finalizedBlockSet = ConcurrentHashMap.newKeySet(); } /** @@ -275,6 +281,34 @@ public long getDeleteTransactionId() { return deleteTransactionId; } + /** + * Add the given localID of a block to the finalizedBlockSet. + */ + public void addToFinalizedBlockSet(long localID) { + finalizedBlockSet.add(localID); + } + + public Set getFinalizedBlockSet() { + return finalizedBlockSet; + } + + public boolean isFinalizedBlockExist(long localID) { + return finalizedBlockSet.contains(localID); + } + + public void clearFinalizedBlock(DBHandle db) throws IOException { + if (!finalizedBlockSet.isEmpty()) { + // delete from db and clear memory + // Should never fail. + Preconditions.checkNotNull(db, "DB cannot be null here"); + try (BatchOperation batch = db.getStore().getBatchHandler().initBatchOperation()) { + db.getStore().getFinalizeBlocksTable().deleteBatchWithPrefix(batch, containerPrefix()); + db.getStore().getBatchHandler().commitBatchOperation(batch); + } + finalizedBlockSet.clear(); + } + } + /** * Returns a ProtoBuf Message from ContainerData. * diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index 01bd4db8a115..ed13ebc93b99 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -53,6 +53,7 @@ import org.apache.hadoop.hdds.scm.ByteStringConversion; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.hdds.utils.FaultInjector; import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.common.Checksum; @@ -102,6 +103,8 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.PUT_SMALL_FILE_ERROR; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getBlockDataResponse; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getBlockLengthResponse; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getEchoResponse; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getFinalizeBlockResponse; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getGetSmallFileResponseSuccess; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getListBlockResponse; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getPutFileResponseSuccess; @@ -109,6 +112,7 @@ import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getReadContainerResponse; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getSuccessResponse; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getSuccessResponseBuilder; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getWriteChunkResponseSuccess; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.malformedRequest; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.putBlockResponseSuccess; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; @@ -138,6 +142,7 @@ public class KeyValueHandler extends Handler { private final boolean validateChunkChecksumData; // A striped lock that is held during container creation. private final Striped containerCreationLocks; + private static FaultInjector injector; public KeyValueHandler(ConfigurationSource config, String datanodeId, @@ -274,6 +279,10 @@ static ContainerCommandResponseProto dispatchRequest(KeyValueHandler handler, return handler.handleGetSmallFile(request, kvContainer); case GetCommittedBlockLength: return handler.handleGetCommittedBlockLength(request, kvContainer); + case FinalizeBlock: + return handler.handleFinalizeBlock(request, kvContainer); + case Echo: + return handler.handleEcho(request, kvContainer); default: return null; } @@ -562,6 +571,55 @@ ContainerCommandResponseProto handlePutBlock( return putBlockResponseSuccess(request, blockDataProto); } + ContainerCommandResponseProto handleFinalizeBlock( + ContainerCommandRequestProto request, KeyValueContainer kvContainer) { + ContainerCommandResponseProto responseProto = checkFaultInjector(request); + if (responseProto != null) { + return responseProto; + } + + if (!request.hasFinalizeBlock()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Malformed Finalize block request. trace ID: {}", + request.getTraceID()); + } + return malformedRequest(request); + } + ContainerProtos.BlockData responseData; + + try { + checkContainerOpen(kvContainer); + BlockID blockID = BlockID.getFromProtobuf( + request.getFinalizeBlock().getBlockID()); + Preconditions.checkNotNull(blockID); + + LOG.info("Finalized Block request received {} ", blockID); + + responseData = blockManager.getBlock(kvContainer, blockID) + .getProtoBufMessage(); + + chunkManager.finalizeWriteChunk(kvContainer, blockID); + blockManager.finalizeBlock(kvContainer, blockID); + kvContainer.getContainerData() + .addToFinalizedBlockSet(blockID.getLocalID()); + + LOG.info("Block has been finalized {} ", blockID); + + } catch (StorageContainerException ex) { + return ContainerUtils.logAndReturnError(LOG, ex, request); + } catch (IOException ex) { + return ContainerUtils.logAndReturnError(LOG, + new StorageContainerException( + "Finalize Block failed", ex, IO_EXCEPTION), request); + } + return getFinalizeBlockResponse(request, responseData); + } + + ContainerCommandResponseProto handleEcho( + ContainerCommandRequestProto request, KeyValueContainer kvContainer) { + return getEchoResponse(request); + } + /** * Handle Get Block operation. Calls BlockManager to process the request. */ @@ -602,6 +660,12 @@ ContainerCommandResponseProto handleGetBlock( */ ContainerCommandResponseProto handleGetCommittedBlockLength( ContainerCommandRequestProto request, KeyValueContainer kvContainer) { + + ContainerCommandResponseProto responseProto = checkFaultInjector(request); + if (responseProto != null) { + return responseProto; + } + if (!request.hasGetCommittedBlockLength()) { if (LOG.isDebugEnabled()) { LOG.debug("Malformed Get Key request. trace ID: {}", @@ -717,6 +781,7 @@ ContainerCommandResponseProto handleReadChunk( data = chunkManager.readChunk(kvContainer, blockID, chunkInfo, dispatcherContext); + LOG.debug("read chunk from block {} chunk {}", blockID, chunkInfo); // Validate data only if the read chunk is issued by Ratis for its // internal logic. // For client reads, the client is expected to validate. @@ -778,6 +843,7 @@ ContainerCommandResponseProto handleWriteChunk( return malformedRequest(request); } + ContainerProtos.BlockData blockDataProto = null; try { checkContainerOpen(kvContainer); @@ -801,6 +867,28 @@ ContainerCommandResponseProto handleWriteChunk( chunkManager .writeChunk(kvContainer, blockID, chunkInfo, data, dispatcherContext); + final boolean isCommit = dispatcherContext.getStage().isCommit(); + if (isCommit && writeChunk.hasBlock()) { + metrics.incContainerOpsMetrics(Type.PutBlock); + BlockData blockData = BlockData.getFromProtoBuf( + writeChunk.getBlock().getBlockData()); + // optimization for hsync when WriteChunk is in commit phase: + // + // block metadata is piggybacked in the same message. + // there will not be an additional PutBlock request. + // + // End of block will always be sent as a standalone PutBlock. + // the PutBlock piggybacked in WriteChunk is never end of block. + // + // do not do this in WRITE_DATA phase otherwise PutBlock will be out + // of order. + blockData.setBlockCommitSequenceId(dispatcherContext.getLogIndex()); + blockManager.putBlock(kvContainer, blockData, false); + blockDataProto = blockData.getProtoBufMessage(); + final long numBytes = blockDataProto.getSerializedSize(); + metrics.incContainerBytesStats(Type.PutBlock, numBytes); + } + // We should increment stats after writeChunk if (isWrite) { metrics.incContainerBytesStats(Type.WriteChunk, writeChunk @@ -814,7 +902,7 @@ ContainerCommandResponseProto handleWriteChunk( request); } - return getSuccessResponse(request); + return getWriteChunkResponseSuccess(request, blockDataProto); } /** @@ -1212,6 +1300,16 @@ public void deleteUnreferenced(Container container, long localID) } } + public void addFinalizedBlock(Container container, long localID) { + KeyValueContainer keyValueContainer = (KeyValueContainer)container; + keyValueContainer.getContainerData().addToFinalizedBlockSet(localID); + } + + public boolean isFinalizedBlockExist(Container container, long localID) { + KeyValueContainer keyValueContainer = (KeyValueContainer)container; + return keyValueContainer.getContainerData().isFinalizedBlockExist(localID); + } + private String[] getFilesWithPrefix(String prefix, File chunkDir) { FilenameFilter filter = (dir, name) -> name.startsWith(prefix); return chunkDir.list(filter); @@ -1370,8 +1468,34 @@ private void triggerVolumeScanAndThrowException(Container container, throw new StorageContainerException(msg, result); } + private ContainerCommandResponseProto checkFaultInjector(ContainerCommandRequestProto request) { + if (injector != null) { + Throwable ex = injector.getException(); + if (ex != null) { + // reset injector + injector = null; + return ContainerUtils.logAndReturnError(LOG, (StorageContainerException) ex, request); + } + try { + injector.pause(); + } catch (IOException e) { + // do nothing + } + } + return null; + } + public static Logger getLogger() { return LOG; } + @VisibleForTesting + public static FaultInjector getInjector() { + return injector; + } + + @VisibleForTesting + public static void setInjector(FaultInjector instance) { + injector = instance; + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java index 90ee356ab59d..b7d5b5fa59eb 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java @@ -356,6 +356,29 @@ private static void populateContainerMetadata( // startup. If this method is called but not as a part of startup, // The inspectors will be unloaded and this will be a no-op. ContainerInspectorUtil.process(kvContainerData, store); + + // Load finalizeBlockLocalIds for container in memory. + populateContainerFinalizeBlock(kvContainerData, store); + } + + /** + * Loads finalizeBlockLocalIds for container in memory. + * @param kvContainerData - KeyValueContainerData + * @param store - DatanodeStore + * @throws IOException + */ + private static void populateContainerFinalizeBlock( + KeyValueContainerData kvContainerData, DatanodeStore store) + throws IOException { + if (store.getFinalizeBlocksTable() != null) { + try (BlockIterator iter = + store.getFinalizeBlockIterator(kvContainerData.getContainerID(), + kvContainerData.getUnprefixedKeyFilter())) { + while (iter.hasNext()) { + kvContainerData.addToFinalizedBlockSet(iter.nextBlock()); + } + } + } } /** diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java index 62896561f254..413f36a7616b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature; import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.ozone.container.common.helpers.BlockData; @@ -35,11 +36,15 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; +import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures; import com.google.common.base.Preconditions; + import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.BCSID_MISMATCH; -import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNKNOWN_BCSID; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_CHUNK_LIST_INCREMENTAL; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_CHUNK_LIST_INCREMENTAL_DEFAULT; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,17 +54,19 @@ */ public class BlockManagerImpl implements BlockManager { - static final Logger LOG = LoggerFactory.getLogger(BlockManagerImpl.class); + public static final Logger LOG = + LoggerFactory.getLogger(BlockManagerImpl.class); private ConfigurationSource config; private static final String DB_NULL_ERR_MSG = "DB cannot be null here"; - private static final String NO_SUCH_BLOCK_ERR_MSG = - "Unable to find the block."; + public static final String INCREMENTAL_CHUNK_LIST = "incremental"; + public static final String FULL_CHUNK = "full"; // Default Read Buffer capacity when Checksum is not present private final int defaultReadBufferCapacity; private final int readMappedBufferThreshold; + private boolean incrementalEnabled; /** * Constructs a Block Manager. @@ -75,6 +82,15 @@ public BlockManagerImpl(ConfigurationSource conf) { this.readMappedBufferThreshold = config.getBufferSize( ScmConfigKeys.OZONE_CHUNK_READ_MAPPED_BUFFER_THRESHOLD_KEY, ScmConfigKeys.OZONE_CHUNK_READ_MAPPED_BUFFER_THRESHOLD_DEFAULT); + incrementalEnabled = + config.getBoolean(OZONE_CHUNK_LIST_INCREMENTAL, + OZONE_CHUNK_LIST_INCREMENTAL_DEFAULT); + if (incrementalEnabled && !VersionedDatanodeFeatures.isFinalized( + HDDSLayoutFeature.HBASE_SUPPORT)) { + LOG.warn("DataNode has not finalized upgrading to a version that " + + "supports incremental chunk list. Fallback to full chunk list"); + incrementalEnabled = false; + } } @Override @@ -87,13 +103,11 @@ public long putBlock(Container container, BlockData data, boolean endOfBlock) throws IOException { return persistPutBlock( (KeyValueContainer) container, - data, - config, - endOfBlock); + data, endOfBlock); } - public static long persistPutBlock(KeyValueContainer container, - BlockData data, ConfigurationSource config, boolean endOfBlock) + public long persistPutBlock(KeyValueContainer container, + BlockData data, boolean endOfBlock) throws IOException { Preconditions.checkNotNull(data, "BlockData cannot be null for put " + "operation."); @@ -137,7 +151,6 @@ public static long persistPutBlock(KeyValueContainer container, // update the blockData as well as BlockCommitSequenceId here try (BatchOperation batch = db.getStore().getBatchHandler() .initBatchOperation()) { - // If the block does not exist in the pendingPutBlockCache of the // container, then check the DB to ascertain if it exists or not. // If block exists in cache, blockCount should not be incremented. @@ -150,8 +163,8 @@ public static long persistPutBlock(KeyValueContainer container, } } - db.getStore().getBlockDataTable().putWithBatch( - batch, containerData.getBlockKey(localID), data); + db.getStore().putBlockByID(batch, incrementalEnabled, localID, data, + containerData, endOfBlock); if (bcsId != 0) { db.getStore().getMetadataTable().putWithBatch( batch, containerData.getBcsIdKey(), bcsId); @@ -207,6 +220,34 @@ public static long persistPutBlock(KeyValueContainer container, } } + @Override + public void finalizeBlock(Container container, BlockID blockId) + throws IOException { + Preconditions.checkNotNull(blockId, "blockId cannot " + + "be null for finalizeBlock operation."); + Preconditions.checkState(blockId.getContainerID() >= 0, + "Container Id cannot be negative"); + + KeyValueContainer kvContainer = (KeyValueContainer)container; + long localID = blockId.getLocalID(); + + kvContainer.removeFromPendingPutBlockCache(localID); + + try (DBHandle db = BlockUtils.getDB(kvContainer.getContainerData(), + config)) { + // Should never fail. + Preconditions.checkNotNull(db, DB_NULL_ERR_MSG); + + // persist finalizeBlock + try (BatchOperation batch = db.getStore().getBatchHandler() + .initBatchOperation()) { + db.getStore().getFinalizeBlocksTable().putWithBatch(batch, + kvContainer.getContainerData().getBlockKey(localID), localID); + db.getStore().getBatchHandler().commitBatchOperation(batch); + } + } + } + @Override public BlockData getBlock(Container container, BlockID blockID) throws IOException { @@ -323,13 +364,6 @@ public void shutdown() { private BlockData getBlockByID(DBHandle db, BlockID blockID, KeyValueContainerData containerData) throws IOException { String blockKey = containerData.getBlockKey(blockID.getLocalID()); - - BlockData blockData = db.getStore().getBlockDataTable().get(blockKey); - if (blockData == null) { - throw new StorageContainerException(NO_SUCH_BLOCK_ERR_MSG + - " BlockID : " + blockID, NO_SUCH_BLOCK); - } - - return blockData; + return db.getStore().getBlockByID(blockID, blockKey); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java index 9faf4ed91c10..6a1d5533cf2c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java @@ -97,6 +97,12 @@ public void finishWriteChunks(KeyValueContainer kvContainer, .finishWriteChunks(kvContainer, blockData); } + @Override + public void finalizeWriteChunk(KeyValueContainer kvContainer, + BlockID blockId) throws IOException { + selectHandler(kvContainer).finalizeWriteChunk(kvContainer, blockId); + } + @Override public ChunkBuffer readChunk(Container container, BlockID blockID, ChunkInfo info, DispatcherContext dispatcherContext) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java index 9a607ceebed9..a87b184ccecf 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java @@ -223,6 +223,23 @@ public void finishWriteChunks(KeyValueContainer container, } } + @Override + public void finalizeWriteChunk(KeyValueContainer container, + BlockID blockId) throws IOException { + synchronized (container) { + File chunkFile = getChunkFile(container, blockId); + try { + if (files.isOpen(chunkFile)) { + files.close(chunkFile); + } + verifyChunkFileExists(chunkFile); + } catch (IOException e) { + onFailure(container.getContainerData().getVolume()); + throw e; + } + } + } + private void deleteChunk(Container container, BlockID blockID, ChunkInfo info, boolean verifyLength) throws StorageContainerException { @@ -304,6 +321,11 @@ public void close(File file) { } } + public boolean isOpen(File file) { + return file != null && + files.getIfPresent(file.getPath()) != null; + } + private static void close(String filename, OpenFile openFile) { if (openFile != null) { if (LOG.isDebugEnabled()) { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java index 02b7e93d50f4..d1b648a993a7 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java @@ -91,6 +91,9 @@ List listBlock(Container container, long startLocalID, int count) long getCommittedBlockLength(Container container, BlockID blockID) throws IOException; + void finalizeBlock(Container container, BlockID blockId) + throws IOException; + int getDefaultReadBufferCapacity(); /** @return the threshold to read using memory mapped buffers. */ diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java index 151c15f35676..ed3142c8570d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java @@ -106,6 +106,11 @@ default void finishWriteChunks(KeyValueContainer kvContainer, // no-op } + default void finalizeWriteChunk(KeyValueContainer container, + BlockID blockId) throws IOException { + // no-op + } + default String streamInit(Container container, BlockID blockID) throws StorageContainerException { return null; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeDBDefinition.java index c174108ba239..cc83f453ebdf 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeDBDefinition.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeDBDefinition.java @@ -69,4 +69,11 @@ public ConfigurationSource getConfig() { public abstract DBColumnFamilyDefinition getMetadataColumnFamily(); + + public DBColumnFamilyDefinition getFinalizeBlocksColumnFamily() { + return null; + } + + public abstract DBColumnFamilyDefinition + getLastChunkInfoColumnFamily(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java index faa3b195f14b..26719d7f035a 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/AbstractDatanodeStore.java @@ -58,9 +58,15 @@ public abstract class AbstractDatanodeStore implements DatanodeStore { private Table blockDataTable; + private Table lastChunkInfoTable; + private Table blockDataTableWithIterator; - static final Logger LOG = + private Table finalizeBlocksTable; + + private Table finalizeBlocksTableWithIterator; + + public static final Logger LOG = LoggerFactory.getLogger(AbstractDatanodeStore.class); private volatile DBStore store; private final AbstractDatanodeDBDefinition dbDef; @@ -152,6 +158,21 @@ public void start(ConfigurationSource config) blockDataTable = new DatanodeTable<>(blockDataTableWithIterator); checkTableStatus(blockDataTable, blockDataTable.getName()); + + if (dbDef.getFinalizeBlocksColumnFamily() != null) { + finalizeBlocksTableWithIterator = + dbDef.getFinalizeBlocksColumnFamily().getTable(this.store); + + finalizeBlocksTable = new DatanodeTable<>( + finalizeBlocksTableWithIterator); + checkTableStatus(finalizeBlocksTable, finalizeBlocksTable.getName()); + } + + if (dbDef.getLastChunkInfoColumnFamily() != null) { + lastChunkInfoTable = new DatanodeTable<>( + dbDef.getLastChunkInfoColumnFamily().getTable(this.store)); + checkTableStatus(lastChunkInfoTable, lastChunkInfoTable.getName()); + } } } @@ -183,23 +204,40 @@ public Table getBlockDataTable() { return blockDataTable; } + @Override + public Table getLastChunkInfoTable() { + return lastChunkInfoTable; + } + @Override public Table getDeletedBlocksTable() { throw new UnsupportedOperationException("DeletedBlocksTable is only supported in Container Schema One"); } + @Override + public Table getFinalizeBlocksTable() { + return finalizeBlocksTable; + } + @Override public BlockIterator getBlockIterator(long containerID) throws IOException { return new KeyValueBlockIterator(containerID, - blockDataTableWithIterator.iterator()); + blockDataTableWithIterator.iterator()); } @Override public BlockIterator getBlockIterator(long containerID, KeyPrefixFilter filter) throws IOException { return new KeyValueBlockIterator(containerID, - blockDataTableWithIterator.iterator(), filter); + blockDataTableWithIterator.iterator(), filter); + } + + @Override + public BlockIterator getFinalizeBlockIterator(long containerID, + KeyPrefixFilter filter) throws IOException { + return new KeyValueBlockLocalIdIterator(containerID, + finalizeBlocksTableWithIterator.iterator(), filter); } @Override @@ -244,6 +282,10 @@ protected Table getBlockDataTableWithIterator() { return this.blockDataTableWithIterator; } + protected Table getFinalizeBlocksTableWithIterator() { + return this.finalizeBlocksTableWithIterator; + } + protected static void checkTableStatus(Table table, String name) throws IOException { String logMessage = "Unable to get a reference to %s table. Cannot " + @@ -359,4 +401,94 @@ public void close() throws IOException { blockIterator.close(); } } + + /** + * Block localId Iterator for KeyValue Container. + * This Block localId iterator returns localIds + * which match with the {@link MetadataKeyFilters.KeyPrefixFilter}. If no + * filter is specified, then default filter used is + * {@link MetadataKeyFilters#getUnprefixedKeyFilter()} + */ + @InterfaceAudience.Public + public static class KeyValueBlockLocalIdIterator implements + BlockIterator, Closeable { + + private static final Logger LOG = LoggerFactory.getLogger( + KeyValueBlockLocalIdIterator.class); + + private final TableIterator> blockLocalIdIterator; + private final KeyPrefixFilter localIdFilter; + private Long nextLocalId; + private final long containerID; + + /** + * KeyValueBlockLocalIdIterator to iterate block localIds in a container. + * @param iterator - The iterator to apply the blockLocalId filter to. + * @param filter - BlockLocalId filter to be applied for block localIds. + */ + KeyValueBlockLocalIdIterator(long containerID, + TableIterator> + iterator, KeyPrefixFilter filter) { + this.containerID = containerID; + this.blockLocalIdIterator = iterator; + this.localIdFilter = filter; + } + + /** + * This method returns blocks matching with the filter. + * @return next block local Id or null if no more block localIds + * @throws IOException + */ + @Override + public Long nextBlock() throws IOException, NoSuchElementException { + if (nextLocalId != null) { + Long currentLocalId = nextLocalId; + nextLocalId = null; + return currentLocalId; + } + if (hasNext()) { + return nextBlock(); + } + throw new NoSuchElementException("Block Local ID Iterator " + + "reached end for ContainerID " + containerID); + } + + @Override + public boolean hasNext() throws IOException { + if (nextLocalId != null) { + return true; + } + while (blockLocalIdIterator.hasNext()) { + Table.KeyValue keyValue = blockLocalIdIterator.next(); + byte[] keyBytes = StringUtils.string2Bytes(keyValue.getKey()); + if (localIdFilter.filterKey(null, keyBytes, null)) { + nextLocalId = keyValue.getValue(); + if (LOG.isTraceEnabled()) { + LOG.trace("Block matching with filter found: LocalID is : " + + "{} for containerID {}", nextLocalId, containerID); + } + return true; + } + } + return false; + } + + @Override + public void seekToFirst() { + nextLocalId = null; + blockLocalIdIterator.seekToFirst(); + } + + @Override + public void seekToLast() { + nextLocalId = null; + blockLocalIdIterator.seekToLast(); + } + + @Override + public void close() throws IOException { + blockLocalIdIterator.close(); + } + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java index f0bab5e5d026..4f54e85da2b1 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaOneDBDefinition.java @@ -101,6 +101,12 @@ public DBColumnFamilyDefinition getMetadataColumnFamily() { return DELETED_BLOCKS; } + @Override + public DBColumnFamilyDefinition + getLastChunkInfoColumnFamily() { + return null; + } + @Override public List> getColumnFamilies(String name) { return COLUMN_FAMILIES.get(name); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java index 3be229a45ac7..d47446d49b0f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaThreeDBDefinition.java @@ -82,13 +82,33 @@ public class DatanodeSchemaThreeDBDefinition DeletedBlocksTransaction.class, Proto2Codec.get(DeletedBlocksTransaction.getDefaultInstance())); + public static final DBColumnFamilyDefinition + FINALIZE_BLOCKS = + new DBColumnFamilyDefinition<>( + "finalize_blocks", + String.class, + FixedLengthStringCodec.get(), + Long.class, + LongCodec.get()); + + public static final DBColumnFamilyDefinition + LAST_CHUNK_INFO = + new DBColumnFamilyDefinition<>( + "last_chunk_info", + String.class, + FixedLengthStringCodec.get(), + BlockData.class, + BlockData.getCodec()); + private static String separator = ""; private static final Map> COLUMN_FAMILIES = DBColumnFamilyDefinition.newUnmodifiableMap( BLOCK_DATA, METADATA, - DELETE_TRANSACTION); + DELETE_TRANSACTION, + FINALIZE_BLOCKS, + LAST_CHUNK_INFO); public DatanodeSchemaThreeDBDefinition(String dbPath, ConfigurationSource config) { @@ -110,6 +130,8 @@ public DatanodeSchemaThreeDBDefinition(String dbPath, BLOCK_DATA.setCfOptions(cfOptions); METADATA.setCfOptions(cfOptions); DELETE_TRANSACTION.setCfOptions(cfOptions); + FINALIZE_BLOCKS.setCfOptions(cfOptions); + LAST_CHUNK_INFO.setCfOptions(cfOptions); } @Override @@ -128,11 +150,23 @@ public DBColumnFamilyDefinition getMetadataColumnFamily() { return METADATA; } + @Override + public DBColumnFamilyDefinition + getLastChunkInfoColumnFamily() { + return LAST_CHUNK_INFO; + } + public DBColumnFamilyDefinition getDeleteTransactionsColumnFamily() { return DELETE_TRANSACTION; } + @Override + public DBColumnFamilyDefinition + getFinalizeBlocksColumnFamily() { + return FINALIZE_BLOCKS; + } + public static int getContainerKeyPrefixLength() { return FixedLengthStringCodec.string2Bytes( getContainerKeyPrefix(0L)).length; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java index cc6c3dc83725..b9e7ec7bd5bf 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeSchemaTwoDBDefinition.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.utils.db.DBColumnFamilyDefinition; import org.apache.hadoop.hdds.utils.db.DBDefinition; +import org.apache.hadoop.hdds.utils.db.FixedLengthStringCodec; import org.apache.hadoop.hdds.utils.db.LongCodec; import org.apache.hadoop.hdds.utils.db.Proto2Codec; import org.apache.hadoop.hdds.utils.db.StringCodec; @@ -66,6 +67,24 @@ public class DatanodeSchemaTwoDBDefinition StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction.class, Proto2Codec.get(DeletedBlocksTransaction.getDefaultInstance())); + public static final DBColumnFamilyDefinition + FINALIZE_BLOCKS = + new DBColumnFamilyDefinition<>( + "finalize_blocks", + String.class, + FixedLengthStringCodec.get(), + Long.class, + LongCodec.get()); + + public static final DBColumnFamilyDefinition + LAST_CHUNK_INFO = + new DBColumnFamilyDefinition<>( + "last_chunk_info", + String.class, + FixedLengthStringCodec.get(), + BlockData.class, + BlockData.getCodec()); + public DatanodeSchemaTwoDBDefinition(String dbPath, ConfigurationSource config) { super(dbPath, config); @@ -75,7 +94,9 @@ public DatanodeSchemaTwoDBDefinition(String dbPath, COLUMN_FAMILIES = DBColumnFamilyDefinition.newUnmodifiableMap( BLOCK_DATA, METADATA, - DELETE_TRANSACTION); + DELETE_TRANSACTION, + FINALIZE_BLOCKS, + LAST_CHUNK_INFO); @Override public Map> getMap() { @@ -93,8 +114,18 @@ public DBColumnFamilyDefinition getMetadataColumnFamily() { return METADATA; } + @Override + public DBColumnFamilyDefinition + getLastChunkInfoColumnFamily() { + return LAST_CHUNK_INFO; + } + public DBColumnFamilyDefinition getDeleteTransactionsColumnFamily() { return DELETE_TRANSACTION; } + + public DBColumnFamilyDefinition getFinalizeBlocksColumnFamily() { + return FINALIZE_BLOCKS; + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java index dd2aa5234b92..35f8bf832227 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStore.java @@ -18,22 +18,31 @@ package org.apache.hadoop.ozone.container.metadata; import com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.utils.MetadataKeyFilters.KeyPrefixFilter; +import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.apache.hadoop.hdds.utils.db.BatchOperationHandler; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfoList; import org.apache.hadoop.ozone.container.common.interfaces.BlockIterator; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import java.io.Closeable; import java.io.IOException; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; + /** * Interface for interacting with datanode databases. */ public interface DatanodeStore extends Closeable { + String NO_SUCH_BLOCK_ERR_MSG = + "Unable to find the block."; /** * Start datanode manager. @@ -77,6 +86,20 @@ public interface DatanodeStore extends Closeable { */ Table getDeletedBlocksTable(); + /** + * A Table that keeps finalize blocks requested from client. + * + * @return Table + */ + Table getFinalizeBlocksTable(); + + /** + * A Table that keeps the metadata of the last chunk of blocks. + * + * @return Table + */ + Table getLastChunkInfoTable(); + /** * Helper to create and write batch transactions. */ @@ -94,6 +117,9 @@ BlockIterator getBlockIterator(long containerID) BlockIterator getBlockIterator(long containerID, KeyPrefixFilter filter) throws IOException; + BlockIterator getFinalizeBlockIterator(long containerID, + KeyPrefixFilter filter) throws IOException; + /** * Returns if the underlying DB is closed. This call is thread safe. * @return true if the DB is closed. @@ -102,4 +128,27 @@ BlockIterator getBlockIterator(long containerID, default void compactionIfNeeded() throws Exception { } + + default BlockData getBlockByID(BlockID blockID, + String blockKey) throws IOException { + + // check block data table + BlockData blockData = getBlockDataTable().get(blockKey); + + if (blockData == null) { + throw new StorageContainerException( + NO_SUCH_BLOCK_ERR_MSG + " BlockID : " + blockID, NO_SUCH_BLOCK); + } + + return blockData; + } + + default void putBlockByID(BatchOperation batch, boolean incremental, + long localID, BlockData data, KeyValueContainerData containerData, + boolean endOfBlock) + throws IOException { + // old client: override chunk list. + getBlockDataTable().putWithBatch( + batch, containerData.getBlockKey(localID), data); + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java index c16d478b166d..7bdc7f1dbd5e 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaThreeImpl.java @@ -53,7 +53,7 @@ * - All keys have containerID as prefix. * - The table 3 has String as key instead of Long since we want to use prefix. */ -public class DatanodeStoreSchemaThreeImpl extends AbstractDatanodeStore +public class DatanodeStoreSchemaThreeImpl extends DatanodeStoreWithIncrementalChunkList implements DeleteTransactionStore { public static final String DUMP_FILE_SUFFIX = ".data"; @@ -94,6 +94,13 @@ public BlockIterator getBlockIterator(long containerID, .iterator(getContainerKeyPrefix(containerID)), filter); } + @Override + public BlockIterator getFinalizeBlockIterator(long containerID, + MetadataKeyFilters.KeyPrefixFilter filter) throws IOException { + return new KeyValueBlockLocalIdIterator(containerID, + getFinalizeBlocksTableWithIterator().iterator(getContainerKeyPrefix(containerID)), filter); + } + public void removeKVContainerData(long containerID) throws IOException { String prefix = getContainerKeyPrefix(containerID); try (BatchOperation batch = getBatchHandler().initBatchOperation()) { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaTwoImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaTwoImpl.java index f09d30e45a77..c9ea52b47c7d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaTwoImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreSchemaTwoImpl.java @@ -31,7 +31,7 @@ * 2. A metadata table. * 3. A Delete Transaction Table. */ -public class DatanodeStoreSchemaTwoImpl extends AbstractDatanodeStore +public class DatanodeStoreSchemaTwoImpl extends DatanodeStoreWithIncrementalChunkList implements DeleteTransactionStore { private final Table diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreWithIncrementalChunkList.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreWithIncrementalChunkList.java new file mode 100644 index 000000000000..84000ba2fb96 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeStoreWithIncrementalChunkList.java @@ -0,0 +1,237 @@ +/* + * 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.ozone.container.metadata; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.hdds.utils.db.BatchOperation; +import org.apache.hadoop.ozone.container.common.helpers.BlockData; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; +import static org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl.FULL_CHUNK; +import static org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl.INCREMENTAL_CHUNK_LIST; + +/** + * Constructs a datanode store in accordance with schema version 2, which uses + * three column families/tables: + * 1. A block data table. + * 2. A metadata table. + * 3. A Delete Transaction Table. + */ +public class DatanodeStoreWithIncrementalChunkList extends AbstractDatanodeStore { + /** + * Constructs the metadata store and starts the DB services. + * + * @param config - Ozone Configuration. + * @throws IOException - on Failure. + */ + public DatanodeStoreWithIncrementalChunkList(ConfigurationSource config, + AbstractDatanodeDBDefinition dbDef, boolean openReadOnly) throws IOException { + super(config, dbDef, openReadOnly); + } + + + @Override + public BlockData getBlockByID(BlockID blockID, + String blockKey) throws IOException { + BlockData lastChunk = null; + // check block data table + BlockData blockData = getBlockDataTable().get(blockKey); + if (blockData == null || isPartialChunkList(blockData)) { + // check last chunk table + lastChunk = getLastChunkInfoTable().get(blockKey); + } + + if (blockData == null) { + if (lastChunk == null) { + throw new StorageContainerException( + NO_SUCH_BLOCK_ERR_MSG + " BlockID : " + blockID, NO_SUCH_BLOCK); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("blockData=(null), lastChunk={}", lastChunk.getChunks()); + } + return lastChunk; + } + } else { + if (lastChunk != null) { + reconcilePartialChunks(lastChunk, blockData); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("blockData={}, lastChunk=(null)", blockData.getChunks()); + } + } + } + + return blockData; + } + + private void reconcilePartialChunks( + BlockData lastChunk, BlockData blockData) { + LOG.debug("blockData={}, lastChunk={}", + blockData.getChunks(), lastChunk.getChunks()); + Preconditions.checkState(lastChunk.getChunks().size() == 1); + if (!blockData.getChunks().isEmpty()) { + ContainerProtos.ChunkInfo lastChunkInBlockData = + blockData.getChunks().get(blockData.getChunks().size() - 1); + if (lastChunkInBlockData != null) { + Preconditions.checkState( + lastChunkInBlockData.getOffset() + lastChunkInBlockData.getLen() + == lastChunk.getChunks().get(0).getOffset(), + "chunk offset does not match"); + } + } + + // append last partial chunk to the block data + List chunkInfos = + new ArrayList<>(blockData.getChunks()); + chunkInfos.add(lastChunk.getChunks().get(0)); + blockData.setChunks(chunkInfos); + + blockData.setBlockCommitSequenceId( + lastChunk.getBlockCommitSequenceId()); + } + + private static boolean isPartialChunkList(BlockData data) { + return data.getMetadata().containsKey(INCREMENTAL_CHUNK_LIST); + } + + private static boolean isFullChunk(ContainerProtos.ChunkInfo chunkInfo) { + for (ContainerProtos.KeyValue kv: chunkInfo.getMetadataList()) { + if (kv.getKey().equals(FULL_CHUNK)) { + return true; + } + } + return false; + } + + // if eob or if the last chunk is full, + private static boolean shouldAppendLastChunk(boolean endOfBlock, + BlockData data) { + if (endOfBlock || data.getChunks().isEmpty()) { + return true; + } + return isFullChunk(data.getChunks().get(data.getChunks().size() - 1)); + } + + public void putBlockByID(BatchOperation batch, boolean incremental, + long localID, BlockData data, KeyValueContainerData containerData, + boolean endOfBlock) throws IOException { + if (!incremental || !isPartialChunkList(data)) { + // Case (1) old client: override chunk list. + getBlockDataTable().putWithBatch( + batch, containerData.getBlockKey(localID), data); + } else if (shouldAppendLastChunk(endOfBlock, data)) { + moveLastChunkToBlockData(batch, localID, data, containerData); + } else { + // incremental chunk list, + // not end of block, has partial chunks + putBlockWithPartialChunks(batch, localID, data, containerData); + } + } + + private void moveLastChunkToBlockData(BatchOperation batch, long localID, + BlockData data, KeyValueContainerData containerData) throws IOException { + // if data has no chunks, fetch the last chunk info from lastChunkInfoTable + if (data.getChunks().isEmpty()) { + BlockData lastChunk = getLastChunkInfoTable().get(containerData.getBlockKey(localID)); + if (lastChunk != null) { + reconcilePartialChunks(lastChunk, data); + } + } + // if eob or if the last chunk is full, + // the 'data' is full so append it to the block table's chunk info + // and then remove from lastChunkInfo + BlockData blockData = getBlockDataTable().get( + containerData.getBlockKey(localID)); + if (blockData == null) { + // Case 2.1 if the block did not have full chunks before + // the block's chunk is what received from client this time, plus the chunks in lastChunkInfoTable + blockData = data; + } else { + // case 2.2 the block already has some full chunks + List chunkInfoList = blockData.getChunks(); + blockData.setChunks(new ArrayList<>(chunkInfoList)); + for (ContainerProtos.ChunkInfo chunk : data.getChunks()) { + blockData.addChunk(chunk); + } + blockData.setBlockCommitSequenceId(data.getBlockCommitSequenceId()); + } + // delete the entry from last chunk info table + getLastChunkInfoTable().deleteWithBatch( + batch, containerData.getBlockKey(localID)); + // update block data table + getBlockDataTable().putWithBatch(batch, + containerData.getBlockKey(localID), blockData); + } + + private void putBlockWithPartialChunks(BatchOperation batch, long localID, + BlockData data, KeyValueContainerData containerData) throws IOException { + if (data.getChunks().size() == 1) { + // Case (3.1) replace/update the last chunk info table + getLastChunkInfoTable().putWithBatch( + batch, containerData.getBlockKey(localID), data); + } else { + int lastChunkIndex = data.getChunks().size() - 1; + // received more than one chunk this time + List lastChunkInfo = + Collections.singletonList( + data.getChunks().get(lastChunkIndex)); + BlockData blockData = getBlockDataTable().get( + containerData.getBlockKey(localID)); + if (blockData == null) { + // Case 3.2: if the block does not exist in the block data table + List chunkInfos = + new ArrayList<>(data.getChunks()); + chunkInfos.remove(lastChunkIndex); + data.setChunks(chunkInfos); + blockData = data; + LOG.debug("block {} does not have full chunks yet. Adding the " + + "chunks to it {}", localID, blockData); + } else { + // Case 3.3: if the block exists in the block data table, + // append chunks till except the last one (supposedly partial) + List chunkInfos = + new ArrayList<>(blockData.getChunks()); + + LOG.debug("blockData.getChunks()={}", chunkInfos); + LOG.debug("data.getChunks()={}", data.getChunks()); + + for (int i = 0; i < lastChunkIndex; i++) { + chunkInfos.add(data.getChunks().get(i)); + } + blockData.setChunks(chunkInfos); + blockData.setBlockCommitSequenceId(data.getBlockCommitSequenceId()); + } + getBlockDataTable().putWithBatch(batch, + containerData.getBlockKey(localID), blockData); + // update the last partial chunk + data.setChunks(lastChunkInfo); + getLastChunkInfoTable().putWithBatch( + batch, containerData.getBlockKey(localID), data); + } + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java index feb580538747..ecc4e80b4bff 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java @@ -159,6 +159,29 @@ public void closeContainer(final long containerId) throws IOException { getHandler(container).closeContainer(container); } + /** + * Returns the Container given a container id. + * + * @param containerId ID of the container + * @return Container + */ + public void addFinalizedBlock(final long containerId, + final long localId) { + Container container = containerSet.getContainer(containerId); + if (container != null) { + getHandler(container).addFinalizedBlock(container, localId); + } + } + + public boolean isFinalizedBlockExist(final long containerId, + final long localId) { + Container container = containerSet.getContainer(containerId); + if (container != null) { + return getHandler(container).isFinalizedBlockExist(container, localId); + } + return false; + } + public Container importContainer( final ContainerData containerData, final InputStream rawContainerStream, diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index b9c8feae16ce..2637f1922c68 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -245,6 +245,14 @@ public void testHandlerCommandHandling() throws Exception { .dispatchRequest(handler, getSmallFileRequest, container, null); verify(handler, times(1)).handleGetSmallFile( any(ContainerCommandRequestProto.class), any()); + + // Test Finalize Block Request handling + ContainerCommandRequestProto finalizeBlock = + getDummyCommandRequestProto(ContainerProtos.Type.FinalizeBlock); + KeyValueHandler + .dispatchRequest(handler, finalizeBlock, container, null); + verify(handler, times(1)).handleFinalizeBlock( + any(ContainerCommandRequestProto.class), any()); } @Test diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java index d9b85a7ce806..565c3c94408e 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java @@ -41,6 +41,7 @@ import java.util.UUID; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_INTERNAL_ERROR; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_UNHEALTHY; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.SUCCESS; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNKNOWN_BCSID; import static org.apache.hadoop.ozone.container.ContainerTestHelper.DATANODE_UUID; @@ -121,6 +122,19 @@ public void testReadChunk() { assertEquals(UNKNOWN_BCSID, response.getResult()); } + @Test + public void testFinalizeBlock() { + KeyValueContainer container = getMockUnhealthyContainer(); + KeyValueHandler handler = getDummyHandler(); + + ContainerProtos.ContainerCommandResponseProto response = + handler.handleFinalizeBlock( + getDummyCommandRequestProto( + ContainerProtos.Type.FinalizeBlock), + container); + assertEquals(CONTAINER_UNHEALTHY, response.getResult()); + } + @Test public void testGetSmallFile() { KeyValueContainer container = getMockUnhealthyContainer(); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java index dabd471be4f9..26d959e88600 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestBlockManagerImpl.java @@ -36,17 +36,23 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.io.TempDir; +import java.io.IOException; import java.nio.file.Path; import java.util.ArrayList; import java.util.List; import java.util.UUID; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_CHUNK_LIST_INCREMENTAL; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.mockito.Mockito.anyList; -import static org.mockito.Mockito.anyLong; +import static org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil.isSameSchemaVersion; +import static org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl.FULL_CHUNK; +import static org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl.INCREMENTAL_CHUNK_LIST; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -78,6 +84,7 @@ private void initTest(ContainerTestVersionInfo versionInfo) this.schemaVersion = versionInfo.getSchemaVersion(); this.config = new OzoneConfiguration(); ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, config); + config.setBoolean(OZONE_CHUNK_LIST_INCREMENTAL, true); initilaze(); } @@ -222,4 +229,181 @@ public void testListBlock(ContainerTestVersionInfo versionInfo) assertNotNull(listBlockData); assertEquals(10, listBlockData.size()); } + + private BlockData createBlockData(long containerID, long blockNo, + int chunkID, long offset, long len, long bcsID) + throws IOException { + blockID1 = new BlockID(containerID, blockNo); + blockData = new BlockData(blockID1); + List chunkList1 = new ArrayList<>(); + ChunkInfo info1 = new ChunkInfo(String.format("%d_chunk_%d", blockID1 + .getLocalID(), chunkID), offset, len); + chunkList1.add(info1.getProtoBufMessage()); + blockData.setChunks(chunkList1); + blockData.setBlockCommitSequenceId(bcsID); + blockData.addMetadata(INCREMENTAL_CHUNK_LIST, ""); + + return blockData; + } + + private BlockData createBlockDataWithOneFullChunk(long containerID, + long blockNo, int chunkID, long offset, long len, long bcsID) + throws IOException { + blockID1 = new BlockID(containerID, blockNo); + blockData = new BlockData(blockID1); + List chunkList1 = new ArrayList<>(); + ChunkInfo info1 = new ChunkInfo(String.format("%d_chunk_%d", blockID1 + .getLocalID(), 1), 0, 4 * 1024 * 1024); + info1.addMetadata(FULL_CHUNK, ""); + + ChunkInfo info2 = new ChunkInfo(String.format("%d_chunk_%d", blockID1 + .getLocalID(), chunkID), offset, len); + chunkList1.add(info1.getProtoBufMessage()); + chunkList1.add(info2.getProtoBufMessage()); + blockData.setChunks(chunkList1); + blockData.setBlockCommitSequenceId(bcsID); + blockData.addMetadata(INCREMENTAL_CHUNK_LIST, ""); + + return blockData; + } + + private BlockData createBlockDataWithThreeFullChunks(long containerID, + long blockNo, long bcsID) throws IOException { + blockID1 = new BlockID(containerID, blockNo); + blockData = new BlockData(blockID1); + List chunkList1 = new ArrayList<>(); + long chunkLimit = 4 * 1024 * 1024; + for (int i = 1; i < 4; i++) { + ChunkInfo info1 = new ChunkInfo( + String.format("%d_chunk_%d", blockID1.getLocalID(), i), + chunkLimit * i, chunkLimit); + info1.addMetadata(FULL_CHUNK, ""); + chunkList1.add(info1.getProtoBufMessage()); + } + blockData.setChunks(chunkList1); + blockData.setBlockCommitSequenceId(bcsID); + blockData.addMetadata(INCREMENTAL_CHUNK_LIST, ""); + + return blockData; + } + + @ContainerTestVersionInfo.ContainerTest + public void testFlush1(ContainerTestVersionInfo versionInfo) + throws Exception { + initTest(versionInfo); + Assumptions.assumeFalse( + isSameSchemaVersion(schemaVersion, OzoneConsts.SCHEMA_V1)); + // simulates writing 1024 bytes, hsync, + // write another 1024 bytes, hsync + // write another 1024 bytes, hsync + long containerID = 1; + long blockNo = 2; + // put 1st chunk + blockData1 = createBlockData(containerID, blockNo, 1, 0, 1024, + 1); + blockManager.putBlock(keyValueContainer, blockData1, false); + // put 2nd chunk + BlockData blockData2 = createBlockData(containerID, blockNo, 1, 0, 2048, + 2); + blockManager.putBlock(keyValueContainer, blockData2, false); + assertEquals(1, keyValueContainer.getContainerData().getBlockCount()); + + BlockData getBlockData = blockManager.getBlock(keyValueContainer, + new BlockID(containerID, blockNo)); + assertEquals(2048, getBlockData.getSize()); + assertEquals(2, getBlockData.getBlockCommitSequenceId()); + List chunkInfos = getBlockData.getChunks(); + assertEquals(1, chunkInfos.size()); + assertEquals(2048, chunkInfos.get(0).getLen()); + assertEquals(0, chunkInfos.get(0).getOffset()); + + // put 3rd chunk, end-of-block + BlockData blockData3 = createBlockData(containerID, blockNo, 1, 0, 3072, + 3); + blockManager.putBlock(keyValueContainer, blockData3, true); + assertEquals(1, keyValueContainer.getContainerData().getBlockCount()); + + getBlockData = blockManager.getBlock(keyValueContainer, + new BlockID(containerID, blockNo)); + assertEquals(3072, getBlockData.getSize()); + assertEquals(3, getBlockData.getBlockCommitSequenceId()); + chunkInfos = getBlockData.getChunks(); + assertEquals(1, chunkInfos.size()); + assertEquals(3072, chunkInfos.get(0).getLen()); + assertEquals(0, chunkInfos.get(0).getOffset()); + } + + @ContainerTestVersionInfo.ContainerTest + public void testFlush2(ContainerTestVersionInfo versionInfo) + throws Exception { + initTest(versionInfo); + Assumptions.assumeFalse( + isSameSchemaVersion(schemaVersion, OzoneConsts.SCHEMA_V1)); + // simulates writing a full chunk + 1024 bytes, hsync, + // write another 1024 bytes, hsync + // write another 1024 bytes, hsync + long containerID = 1; + long blockNo = 2; + long chunkLimit = 4 * 1024 * 1024; + // first hsync (a full chunk + 1024 bytes) + blockData1 = createBlockDataWithOneFullChunk(containerID, + blockNo, 2, chunkLimit, 1024, 1); + blockManager.putBlock(keyValueContainer, blockData1, false); + // second hsync (1024 bytes) + BlockData blockData2 = createBlockData(containerID, blockNo, 2, + chunkLimit, 2048, 2); + blockManager.putBlock(keyValueContainer, blockData2, false); + assertEquals(1, keyValueContainer.getContainerData().getBlockCount()); + // third hsync (1024 bytes) + BlockData blockData3 = createBlockData(containerID, blockNo, 2, + chunkLimit, 3072, 3); + blockManager.putBlock(keyValueContainer, blockData3, false); + assertEquals(1, keyValueContainer.getContainerData().getBlockCount()); + + // verify that first chunk is full, second chunk is 3072 bytes + BlockData getBlockData = blockManager.getBlock(keyValueContainer, + new BlockID(containerID, blockNo)); + assertEquals(3072 + chunkLimit, getBlockData.getSize()); + assertEquals(3, getBlockData.getBlockCommitSequenceId()); + List chunkInfos = getBlockData.getChunks(); + assertEquals(2, chunkInfos.size()); + assertEquals(chunkLimit, chunkInfos.get(0).getLen()); + assertEquals(0, chunkInfos.get(0).getOffset()); + assertEquals(3072, chunkInfos.get(1).getLen()); + assertEquals(chunkLimit, chunkInfos.get(1).getOffset()); + } + + @ContainerTestVersionInfo.ContainerTest + public void testFlush3(ContainerTestVersionInfo versionInfo) + throws Exception { + initTest(versionInfo); + Assumptions.assumeFalse( + isSameSchemaVersion(schemaVersion, OzoneConsts.SCHEMA_V1)); + // simulates writing 1024 bytes, hsync, + // and then write till 4 chunks are full + long containerID = 1; + long blockNo = 2; + long chunkLimit = 4 * 1024 * 1024; + // first hsync (1024 bytes) + blockData1 = createBlockDataWithOneFullChunk(containerID, blockNo, 2, + chunkLimit, 1024, 1); + blockManager.putBlock(keyValueContainer, blockData1, false); + // full flush (4 chunks) + BlockData blockData2 = createBlockDataWithThreeFullChunks( + containerID, blockNo, 2); + blockManager.putBlock(keyValueContainer, blockData2, false); + assertEquals(1, keyValueContainer.getContainerData().getBlockCount()); + + // verify that the four chunks are full + BlockData getBlockData = blockManager.getBlock(keyValueContainer, + new BlockID(containerID, blockNo)); + assertEquals(chunkLimit * 4, getBlockData.getSize()); + assertEquals(2, getBlockData.getBlockCommitSequenceId()); + List chunkInfos = getBlockData.getChunks(); + assertEquals(4, chunkInfos.size()); + for (int i = 0; i < 4; i++) { + assertEquals(chunkLimit, chunkInfos.get(i).getLen()); + assertEquals(chunkLimit * i, chunkInfos.get(i).getOffset()); + } + } } diff --git a/hadoop-hdds/docs/content/tools/Admin.md b/hadoop-hdds/docs/content/tools/Admin.md index a05065e8ebe7..e89331230fbd 100644 --- a/hadoop-hdds/docs/content/tools/Admin.md +++ b/hadoop-hdds/docs/content/tools/Admin.md @@ -32,4 +32,143 @@ And quick overview about the available functionalities: * `ozone admin replicationmanager`: Can be used to check the status of the replications (and start / stop replication in case of emergency). * `ozone admin om`: Ozone Manager HA related tool to get information about the current cluster. -For more detailed usage see the output of the `--help`. +For more detailed usage see the output of `--help`. + +```bash +$ ozone admin --help +Usage: ozone admin [-hV] [--verbose] [-conf=] + [-D=]... [COMMAND] +Developer tools for Ozone Admin operations + -conf= + + -D, --set= + + -h, --help Show this help message and exit. + -V, --version Print version information and exit. + --verbose More verbose output. Show the stack trace of the errors. +Commands: + containerbalancer ContainerBalancer specific operations + replicationmanager ReplicationManager specific operations + safemode Safe mode specific operations + printTopology Print a tree of the network topology as reported by SCM + cert Certificate related operations + container Container specific operations + datanode Datanode specific operations + pipeline Pipeline specific operations + namespace Namespace Summary specific admin operations + om Ozone Manager specific admin operations + reconfig Dynamically reconfigure server without restarting it + scm Ozone Storage Container Manager specific admin operations +``` + +Some of those subcommand usages has been detailed in their dedicated feature documentation pages. For instance, [Decommissioning]({{}}), [Non-Rolling Upgrades and Downgrades]({{}}). + + +## List open files + +List open files admin command lists open keys in Ozone Manager's `OpenKeyTable`. +Works for all bucket types. +Argument `--prefix` could be root (`/`), path to a bucket (`/vol1/buck`) or a key prefix (for FSO buckets the key prefix could contain parent object ID). But it can't be a volume. + +```bash +$ ozone admin om lof --help +Usage: ozone admin om list-open-files [-hV] [--json] [-l=] + [-p=] [-s=] + [--service-host=] + [--service-id=] +Lists open files (keys) in Ozone Manager. + -h, --help Show this help message and exit. + --json Format output as JSON + -l, --length= Maximum number of items to list + -p, --prefix= Filter results by the specified path on the server + side. + -s, --start= The item to start the listing from. + i.e. continuation token. This will be excluded from + the result. + --service-host= + Ozone Manager Host. If OM HA is enabled, use + --service-id instead. If you must use + --service-host with OM HA, this must point + directly to the leader OM. This option is + required when --service-id is not provided or + when HA is not enabled. + --service-id, --om-service-id= + Ozone Manager Service ID + -V, --version Print version information and exit. +``` + +### Example usages + +- In human-readable format, list open files (keys) under bucket `/volumelof/buck1` with a batch size of 3: + +```bash +$ ozone admin om lof --service-id=om-service-test1 --length=3 --prefix=/volumelof/buck1 +``` + +```bash +5 total open files (est.). Showing 3 open files (limit 3) under path prefix: + /volume-lof/buck1 + +Client ID Creation time Hsync'ed Open File Path +111726338148007937 1704808626523 No /volume-lof/buck1/-9223372036854774527/key0 +111726338151415810 1704808626578 No /volume-lof/buck1/-9223372036854774527/key1 +111726338152071171 1704808626588 No /volume-lof/buck1/-9223372036854774527/key2 + +To get the next batch of open keys, run: + ozone admin om lof -id=om-service-test1 --length=3 --prefix=/volume-lof/buck1 --start=/-9223372036854775552/-9223372036854775040/-9223372036854774527/key2/111726338152071171 +``` + +- In JSON, list open files (keys) under bucket `/volumelof/buck1` with a batch size of 3: + +```bash +$ ozone admin om lof --service-id=om-service-test1 --length=3 --prefix=/volumelof/buck1 --json +``` + +```json +{ + "openKeys" : [ { + "keyInfo" : { + "metadata" : { }, + "objectID" : -9223372036854774015, + "updateID" : 7, + "parentObjectID" : -9223372036854774527, + "volumeName" : "volume-lof", + "bucketName" : "buck1", + "keyName" : "key0", + "dataSize" : 4194304, + "keyLocationVersions" : [ ... ], + "creationTime" : 1704808722487, + "modificationTime" : 1704808722487, + "replicationConfig" : { + "replicationFactor" : "THREE", + "requiredNodes" : 3, + "replicationType" : "RATIS" + }, + "fileName" : "key0", + "acls" : [ ... ], + "path" : "-9223372036854774527/key0", + "file" : true, + "replicatedSize" : 12582912, + "objectInfo" : "OMKeyInfo{volume='volume-lof', bucket='buck1', key='key0', dataSize='4194304', creationTime='1704808722487', objectID='-9223372036854774015', parentID='-9223372036854774527', replication='RATIS/THREE', fileChecksum='null}", + "hsync" : false, + "latestVersionLocations" : { ... }, + "updateIDset" : true + }, + "openVersion" : 0, + "clientId" : 111726344437039105 + }, { + "keyInfo" : { ... }, + "openVersion" : 0, + "clientId" : 111726344440578050 + }, { + "keyInfo" : { ... }, + "openVersion" : 0, + "clientId" : 111726344441233411 + } ], + "totalOpenKeyCount" : 5, + "hasMore" : true, + "contToken" : "/-9223372036854775552/-9223372036854775040/-9223372036854774527/key2/111726344441233411" +} +``` + +Note in JSON output mode, field `contToken` won't show up at all in the result if there are no more entries after the batch (i.e. when `hasMore` is `false`). diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/FaultInjector.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/FaultInjector.java index 27957d162a55..32076abb3fb3 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/FaultInjector.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/FaultInjector.java @@ -40,4 +40,13 @@ public void resume() throws IOException { @VisibleForTesting public void reset() throws IOException { } + + @VisibleForTesting + public void setException(Throwable e) { + } + + @VisibleForTesting + public Throwable getException() { + return null; + } } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 718e2a108c77..f5cac299238d 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -75,6 +75,8 @@ package hadoop.hdds.datanode; * 17. CloseContainer - Closes an open container and makes it immutable. * * 18. CopyContainer - Copies a container from a remote machine. + * + * 19. FinalizeBlock - Finalize block request from client. */ enum Type { @@ -103,6 +105,9 @@ enum Type { StreamInit = 19; StreamWrite = 20; + + FinalizeBlock = 21; + Echo = 22; } @@ -152,6 +157,7 @@ enum Result { DELETE_ON_NON_EMPTY_CONTAINER = 44; EXPORT_CONTAINER_METADATA_FAILED = 45; IMPORT_CONTAINER_METADATA_FAILED = 46; + BLOCK_ALREADY_FINALIZED = 47; } /** @@ -208,6 +214,9 @@ message ContainerCommandRequestProto { optional string encodedToken = 23; optional uint32 version = 24; + + optional FinalizeBlockRequestProto finalizeBlock = 25; + optional EchoRequestProto echo = 26; } message ContainerCommandResponseProto { @@ -237,7 +246,10 @@ message ContainerCommandResponseProto { optional PutSmallFileResponseProto putSmallFile = 19; optional GetSmallFileResponseProto getSmallFile = 20; - optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21; + optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21; + + optional FinalizeBlockResponseProto finalizeBlock = 22; + optional EchoResponseProto echo = 23; } message ContainerDataProto { @@ -338,6 +350,14 @@ message PutBlockResponseProto { required GetCommittedBlockLengthResponseProto committedBlockLength = 1; } +message FinalizeBlockRequestProto { + required DatanodeBlockID blockID = 1; +} + +message FinalizeBlockResponseProto { + required BlockData blockData = 1; +} + message GetBlockRequestProto { required DatanodeBlockID blockID = 1; } @@ -373,6 +393,16 @@ message ListBlockResponseProto { repeated BlockData blockData = 1; } +message EchoRequestProto { + optional bytes payload = 1; + optional int32 payloadSizeResp = 2; + optional int32 sleepTimeMs = 3; +} + +message EchoResponseProto { + optional bytes payload = 1; +} + // Chunk Operations message ChunkInfo { @@ -406,9 +436,11 @@ message WriteChunkRequestProto { required DatanodeBlockID blockID = 1; optional ChunkInfo chunkData = 2; optional bytes data = 3; + optional PutBlockRequestProto block = 4; } message WriteChunkResponseProto { + optional GetCommittedBlockLengthResponseProto committedBlockLength = 1; } enum ReadChunkVersion { diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto index 405845312357..4555d1cf4a39 100644 --- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto +++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto @@ -45,6 +45,7 @@ message DatanodeDetailsProto { optional string networkLocation = 7; // Network topology location optional NodeOperationalState persistedOpState = 8; // The Operational state persisted in the datanode.id file optional int64 persistedOpStateExpiry = 9; // The seconds after the epoch when the OpState should expire + optional int32 currentVersion = 10; // Current datanode wire version // TODO(runzhiwang): when uuid is gone, specify 1 as the index of uuid128 and mark as required optional UUID uuid128 = 100; // UUID with 128 bits assigned to the Datanode. optional uint32 level = 101; diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java index d51479d44bbb..f334f1a03e90 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java @@ -184,7 +184,7 @@ public void createContainer(XceiverClientSpi client, } } - private String getEncodedContainerToken(long containerId) throws IOException { + public String getEncodedContainerToken(long containerId) throws IOException { if (!containerTokenEnabled) { return ""; } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java index 51383e8717ab..1b7918a45a71 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java @@ -28,6 +28,7 @@ import java.util.concurrent.ExecutorService; import java.util.function.Supplier; +import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.scm.ByteStringConversion; import org.apache.hadoop.hdds.scm.ContainerClientMetrics; import org.apache.hadoop.hdds.scm.OzoneClientConfig; @@ -86,6 +87,8 @@ public class BlockOutputStreamEntryPool implements KeyMetadataAware { private final ContainerClientMetrics clientMetrics; private final StreamBufferArgs streamBufferArgs; private final Supplier executorServiceSupplier; + // update blocks on OM + private ContainerBlockID lastUpdatedBlockId = new ContainerBlockID(-1, -1); public BlockOutputStreamEntryPool(KeyOutputStream.Builder b) { this.config = b.getClientConfig(); @@ -346,7 +349,16 @@ void hsyncKey(long offset) throws IOException { if (keyArgs.getIsMultipartKey()) { throw new IOException("Hsync is unsupported for multipart keys."); } else { - omClient.hsyncKey(keyArgs, openID); + if (keyArgs.getLocationInfoList().size() == 0) { + omClient.hsyncKey(keyArgs, openID); + } else { + ContainerBlockID lastBLockId = keyArgs.getLocationInfoList().get(keyArgs.getLocationInfoList().size() - 1) + .getBlockID().getContainerBlockID(); + if (!lastUpdatedBlockId.equals(lastBLockId)) { + omClient.hsyncKey(keyArgs, openID); + lastUpdatedBlockId = lastBLockId; + } + } } } else { LOG.warn("Closing KeyOutputStream, but key args is null"); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java index 2d40841ee499..15e84cf37037 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy; import org.apache.hadoop.hdds.scm.storage.MultipartInputStream; import org.apache.hadoop.hdds.scm.storage.PartInputStream; +import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; @@ -62,8 +63,10 @@ private static List createStreams( Function retryFunction, BlockInputStreamFactory blockStreamFactory, OzoneClientConfig config) { + boolean isHsyncFile = keyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID); List partStreams = new ArrayList<>(); - for (OmKeyLocationInfo omKeyLocationInfo : blockInfos) { + for (int i = 0; i < blockInfos.size(); i++) { + OmKeyLocationInfo omKeyLocationInfo = blockInfos.get(i); if (LOG.isDebugEnabled()) { LOG.debug("Adding stream for accessing {}. The stream will be " + "initialized later.", omKeyLocationInfo); @@ -86,6 +89,11 @@ private static List createStreams( retry = null; } + if (i == (blockInfos.size() - 1) && isHsyncFile) { + // block is under construction + omKeyLocationInfo.setUnderConstruction(true); + } + BlockExtendedInputStream stream = blockStreamFactory.create( keyInfo.getReplicationConfig(), diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java index ef2f1fa118e1..caa3996a09ff 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java @@ -19,6 +19,7 @@ import org.apache.commons.collections.map.HashedMap; import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; @@ -30,18 +31,25 @@ import java.util.List; import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * State represents persisted data of one specific datanode. */ public class MockDatanodeStorage { - - private final Map blocks = new HashedMap(); + public static final Logger LOG = + LoggerFactory.getLogger(MockDatanodeStorage.class); + public static final String INCREMENTAL_CHUNK_LIST = "incremental"; + public static final String FULL_CHUNK = "full"; + public static final ContainerProtos.KeyValue FULL_CHUNK_KV = + ContainerProtos.KeyValue.newBuilder().setKey(FULL_CHUNK).build(); + + private final Map blocks = new HashedMap(); private final Map> containerBlocks = new HashedMap(); private final Map fullBlockData = new HashMap<>(); - private final Map chunks = new HashMap<>(); - private final Map data = new HashMap<>(); private IOException exception = null; @@ -50,8 +58,70 @@ public void setStorageFailed(IOException reason) { this.exception = reason; } + private boolean isIncrementalChunkList(BlockData blockData) { + for (ContainerProtos.KeyValue kv : blockData.getMetadataList()) { + if (kv.getKey().equals(INCREMENTAL_CHUNK_LIST)) { + return true; + } + } + return false; + } + + private BlockID toBlockID(DatanodeBlockID datanodeBlockID) { + return new BlockID(datanodeBlockID.getContainerID(), + datanodeBlockID.getLocalID()); + } + public void putBlock(DatanodeBlockID blockID, BlockData blockData) { - blocks.put(blockID, blockData); + if (isIncrementalChunkList(blockData)) { + LOG.debug("incremental chunk list"); + putBlockIncremental(blockID, blockData); + } else { + LOG.debug("full chunk list"); + putBlockFull(blockID, blockData); + } + } + + private boolean isFullChunk(ChunkInfo chunkInfo) { + return (chunkInfo.getMetadataList().contains(FULL_CHUNK_KV)); + } + + public void putBlockIncremental( + DatanodeBlockID blockID, BlockData blockData) { + BlockID id = toBlockID(blockID); + if (blocks.containsKey(id)) { + // block already exists. let's append the chunk list to it. + BlockData existing = blocks.get(id); + if (existing.getChunksCount() == 0) { + // empty chunk list. override it. + putBlockFull(blockID, blockData); + } else { + BlockData.Builder blockDataBuilder = pruneLastPartialChunks(existing); + blockDataBuilder.addAllChunks(blockData.getChunksList()); + blocks.put(id, blockDataBuilder.build()); + } + // TODO: verify the chunk list beginning/offset/len is sane + } else { + // the block does not exist yet, simply add it + putBlockFull(blockID, blockData); + } + } + + private BlockData.Builder pruneLastPartialChunks(BlockData existing) { + BlockData.Builder blockDataBuilder = BlockData.newBuilder(existing); + int lastChunkIndex = existing.getChunksCount() - 1; + // if the last chunk in the existing block is full, append after it. + ChunkInfo chunkInfo = existing.getChunks(lastChunkIndex); + if (!isFullChunk(chunkInfo)) { + // otherwise, remove it and append + blockDataBuilder.removeChunks(lastChunkIndex); + } + return blockDataBuilder; + } + + public void putBlockFull(DatanodeBlockID blockID, BlockData blockData) { + BlockID id = toBlockID(blockID); + blocks.put(id, blockData); List dnBlocks = containerBlocks .getOrDefault(blockID.getContainerID(), new ArrayList<>()); dnBlocks.add(blockID); @@ -59,14 +129,24 @@ public void putBlock(DatanodeBlockID blockID, BlockData blockData) { } public BlockData getBlock(DatanodeBlockID blockID) { - return blocks.get(blockID); + BlockID id = toBlockID(blockID); + //assert blocks.containsKey(blockID); + if (!blocks.containsKey(id)) { + StringBuilder sb = new StringBuilder(); + for (BlockID bid : blocks.keySet()) { + sb.append(bid).append("\n"); + } + throw new AssertionError("blockID " + id + + " not found in blocks. Available block ID: \n" + sb); + } + return blocks.get(id); } public List listBlock(long containerID) { List datanodeBlockIDS = containerBlocks.get(containerID); List listBlocksData = new ArrayList<>(); for (DatanodeBlockID dBlock : datanodeBlockIDS) { - listBlocksData.add(blocks.get(dBlock)); + listBlocksData.add(blocks.get(toBlockID(dBlock))); } return listBlocksData; } @@ -77,31 +157,39 @@ public void writeChunk( if (exception != null) { throw exception; } - data.put(createKey(blockID, chunkInfo), - ByteString.copyFrom(bytes.toByteArray())); - chunks.put(createKey(blockID, chunkInfo), chunkInfo); + String blockKey = createKey(blockID); + ByteString block; + if (data.containsKey(blockKey)) { + block = data.get(blockKey); + assert block.size() == chunkInfo.getOffset(); + data.put(blockKey, block.concat(ByteString.copyFrom(bytes.asReadOnlyByteBuffer()))); + } else { + assert chunkInfo.getOffset() == 0; + data.put(blockKey, ByteString.copyFrom(bytes.asReadOnlyByteBuffer())); + } + fullBlockData .put(new BlockID(blockID.getContainerID(), blockID.getLocalID()), - fullBlockData.getOrDefault(blockID, "") + fullBlockData.getOrDefault(toBlockID(blockID), "") .concat(bytes.toStringUtf8())); } - public ChunkInfo readChunkInfo( - DatanodeBlockID blockID, - ChunkInfo chunkInfo) { - return chunks.get(createKey(blockID, chunkInfo)); - } - public ByteString readChunkData( DatanodeBlockID blockID, ChunkInfo chunkInfo) { - return data.get(createKey(blockID, chunkInfo)); - + if (LOG.isDebugEnabled()) { + LOG.debug( + "readChunkData: blockID={}, offset={}, len={}", + createKey(blockID), chunkInfo.getOffset(), chunkInfo.getLen()); + } + ByteString str = data.get(createKey(blockID)).substring( + (int)chunkInfo.getOffset(), + (int)chunkInfo.getOffset() + (int)chunkInfo.getLen()); + return str; } - private String createKey(DatanodeBlockID blockId, ChunkInfo chunkInfo) { - return blockId.getContainerID() + "_" + blockId.getLocalID() + "_" - + chunkInfo.getChunkName() + "_" + chunkInfo.getOffset(); + private String createKey(DatanodeBlockID blockId) { + return blockId.getContainerID() + "_" + blockId.getLocalID(); } public Map getAllBlockData() { diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java index e91a71a856bb..31f5e20bc882 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java @@ -45,6 +45,8 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocationList; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListOpenFilesRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListOpenFilesResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LookupKeyRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LookupKeyResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; @@ -54,6 +56,8 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.HashMap; @@ -65,6 +69,8 @@ * OM transport for testing with in-memory state. */ public class MockOmTransport implements OmTransport { + private static final Logger LOG = + LoggerFactory.getLogger(MockOmTransport.class); private final MockBlockAllocator blockAllocator; //volumename -> volumeinfo @@ -125,6 +131,10 @@ public OMResponse submitRequest(OMRequest payload) throws IOException { return response(payload, r -> r.setServiceListResponse( serviceList(payload.getServiceListRequest()))); + case ListOpenFiles: + return response(payload, + r -> r.setListOpenFilesResponse( + listOpenFiles(payload.getListOpenFilesRequest()))); case AllocateBlock: return response(payload, r -> r.setAllocateBlockResponse( allocateBlock(payload.getAllocateBlockRequest()))); @@ -179,11 +189,44 @@ private GetKeyInfoResponse getKeyInfo(GetKeyInfoRequest request) { .build(); } + private boolean isHSync(CommitKeyRequest commitKeyRequest) { + return commitKeyRequest.hasHsync() && commitKeyRequest.getHsync(); + } + + private boolean isRecovery(CommitKeyRequest commitKeyRequest) { + return commitKeyRequest.hasRecovery() && commitKeyRequest.getRecovery(); + } + + private String toOperationString(CommitKeyRequest commitKeyRequest) { + boolean hsync = isHSync(commitKeyRequest); + boolean recovery = isRecovery(commitKeyRequest); + if (hsync) { + return "hsync"; + } + if (recovery) { + return "recover"; + } + return "commit"; + } + + private CommitKeyResponse commitKey(CommitKeyRequest commitKeyRequest) { final KeyArgs keyArgs = commitKeyRequest.getKeyArgs(); final KeyInfo openKey = openKeys.get(keyArgs.getVolumeName()).get(keyArgs.getBucketName()) - .remove(keyArgs.getKeyName()); + .get(keyArgs.getKeyName()); + LOG.debug("{} open key vol: {} bucket: {} key: {}", + toOperationString(commitKeyRequest), + keyArgs.getVolumeName(), + keyArgs.getBucketName(), + keyArgs.getKeyName()); + boolean hsync = isHSync(commitKeyRequest); + if (!hsync) { + KeyInfo deleteKey = openKeys.get(keyArgs.getVolumeName()) + .get(keyArgs.getBucketName()) + .remove(keyArgs.getKeyName()); + assert deleteKey != null; + } final KeyInfo.Builder committedKeyInfoWithLocations = KeyInfo.newBuilder().setVolumeName(keyArgs.getVolumeName()) .setBucketName(keyArgs.getBucketName()) @@ -324,6 +367,11 @@ private ServiceListResponse serviceList( .build(); } + private ListOpenFilesResponse listOpenFiles( + ListOpenFilesRequest listOpenFilesRequest) { + return ListOpenFilesResponse.newBuilder().build(); + } + private OMResponse response(OMRequest payload, Function function) { Builder builder = OMResponse.newBuilder(); diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java index 59eb49e5557c..0d82f0f8bbb2 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java @@ -110,8 +110,7 @@ public XceiverClientReply sendCommandAsync( private ReadChunkResponseProto readChunk(ReadChunkRequestProto readChunk) { return ReadChunkResponseProto.newBuilder() - .setChunkData(datanodeStorage - .readChunkInfo(readChunk.getBlockID(), readChunk.getChunkData())) + .setChunkData(readChunk.getChunkData()) .setData(datanodeStorage .readChunkData(readChunk.getBlockID(), readChunk.getChunkData())) .setBlockID(readChunk.getBlockID()) @@ -130,21 +129,26 @@ private ContainerProtos.ListBlockResponseProto listBlock(long containerID) { } private PutBlockResponseProto putBlock(PutBlockRequestProto putBlock) { + return PutBlockResponseProto.newBuilder() + .setCommittedBlockLength( + doPutBlock(putBlock.getBlockData())) + .build(); + } + + private GetCommittedBlockLengthResponseProto doPutBlock( + ContainerProtos.BlockData blockData) { long length = 0; - for (ChunkInfo chunk : putBlock.getBlockData().getChunksList()) { + for (ChunkInfo chunk : blockData.getChunksList()) { length += chunk.getLen(); } - datanodeStorage.putBlock(putBlock.getBlockData().getBlockID(), - putBlock.getBlockData()); + datanodeStorage.putBlock(blockData.getBlockID(), + blockData); - return PutBlockResponseProto.newBuilder() - .setCommittedBlockLength( - GetCommittedBlockLengthResponseProto.newBuilder() - .setBlockID(putBlock.getBlockData().getBlockID()) + return GetCommittedBlockLengthResponseProto.newBuilder() + .setBlockID(blockData.getBlockID()) .setBlockLength(length) - .build()) - .build(); + .build(); } private XceiverClientReply result( @@ -167,8 +171,15 @@ private WriteChunkResponseProto writeChunk( datanodeStorage .writeChunk(writeChunk.getBlockID(), writeChunk.getChunkData(), writeChunk.getData()); - return WriteChunkResponseProto.newBuilder() - .build(); + + WriteChunkResponseProto.Builder builder = + WriteChunkResponseProto.newBuilder(); + if (writeChunk.hasBlock()) { + ContainerProtos.BlockData + blockData = writeChunk.getBlock().getBlockData(); + builder.setCommittedBlockLength(doPutBlock(blockData)); + } + return builder.build(); } @Override diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestBlockOutputStreamIncrementalPutBlock.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestBlockOutputStreamIncrementalPutBlock.java new file mode 100644 index 000000000000..1014b943a2a2 --- /dev/null +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestBlockOutputStreamIncrementalPutBlock.java @@ -0,0 +1,163 @@ +/* + * 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.ozone.client; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.HashMap; +import java.util.UUID; + +import jakarta.annotation.Nonnull; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.om.helpers.ServiceInfoEx; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.conf.InMemoryConfiguration; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.ozone.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.client.rpc.RpcClient; +import org.apache.hadoop.ozone.om.protocolPB.OmTransport; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_CHUNK_LIST_INCREMENTAL; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; + + +/** + * Verify BlockOutputStream with incremental PutBlock feature. + * (ozone.client.incremental.chunk.list = true) + */ +public class TestBlockOutputStreamIncrementalPutBlock { + private OzoneClient client; + private final String keyName = UUID.randomUUID().toString(); + private final String volumeName = UUID.randomUUID().toString(); + private final String bucketName = UUID.randomUUID().toString(); + private OzoneBucket bucket; + private final ConfigurationSource config = new InMemoryConfiguration(); + + public static Iterable parameters() { + return Arrays.asList(true, false); + } + + private void init(boolean incrementalChunkList) throws IOException { + OzoneClientConfig clientConfig = config.getObject(OzoneClientConfig.class); + + clientConfig.setIncrementalChunkList(incrementalChunkList); + clientConfig.setChecksumType(ContainerProtos.ChecksumType.CRC32C); + + ((InMemoryConfiguration)config).setFromObject(clientConfig); + + ((InMemoryConfiguration) config).setBoolean( + OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); + ((InMemoryConfiguration) config).setBoolean( + OZONE_CHUNK_LIST_INCREMENTAL, incrementalChunkList); + + RpcClient rpcClient = new RpcClient(config, null) { + + @Override + protected OmTransport createOmTransport( + String omServiceId) + throws IOException { + return new MockOmTransport(); + } + + @Nonnull + @Override + protected XceiverClientFactory createXceiverClientFactory( + ServiceInfoEx serviceInfo) throws IOException { + return new MockXceiverClientFactory(); + } + }; + + client = new OzoneClient(config, rpcClient); + ObjectStore store = client.getObjectStore(); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + bucket = volume.getBucket(bucketName); + } + + @AfterEach + public void close() throws IOException { + client.close(); + } + + @ParameterizedTest + @MethodSource("parameters") + public void writeSmallChunk(boolean incrementalChunkList) + throws IOException { + init(incrementalChunkList); + + int size = 1024; + String s = RandomStringUtils.randomAlphabetic(1024); + ByteBuffer byteBuffer = ByteBuffer.wrap(s.getBytes(StandardCharsets.UTF_8)); + + try (OzoneOutputStream out = bucket.createKey(keyName, size, + ReplicationConfig.getDefault(config), new HashMap<>())) { + for (int i = 0; i < 4097; i++) { + out.write(byteBuffer); + out.hsync(); + } + } + + try (OzoneInputStream is = bucket.readKey(keyName)) { + ByteBuffer readBuffer = ByteBuffer.allocate(size); + for (int i = 0; i < 4097; i++) { + is.read(readBuffer); + assertArrayEquals(readBuffer.array(), byteBuffer.array()); + } + } + } + + @ParameterizedTest + @MethodSource("parameters") + public void writeLargeChunk(boolean incrementalChunkList) + throws IOException { + init(incrementalChunkList); + + int size = 1024 * 1024 + 1; + ByteBuffer byteBuffer = ByteBuffer.allocate(size); + + try (OzoneOutputStream out = bucket.createKey(keyName, size, + ReplicationConfig.getDefault(config), new HashMap<>())) { + for (int i = 0; i < 4; i++) { + out.write(byteBuffer); + out.hsync(); + } + } + + try (OzoneInputStream is = bucket.readKey(keyName)) { + ByteBuffer readBuffer = ByteBuffer.allocate(size); + for (int i = 0; i < 4; i++) { + is.read(readBuffer); + assertArrayEquals(readBuffer.array(), byteBuffer.array()); + } + } + } +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java index b4777c7a016b..d2f68f1e4d81 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java @@ -244,6 +244,7 @@ public static boolean isReadOnly( case ListKeysLight: case ListTrash: case ServiceList: + case ListOpenFiles: case ListMultiPartUploadParts: case GetFileStatus: case LookupFile: diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java index ff3a7beee509..b1f9f5bd6890 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java @@ -129,6 +129,11 @@ private OMConfigKeys() { public static final String OZONE_OM_OPEN_KEY_EXPIRE_THRESHOLD_DEFAULT = "7d"; + public static final String OZONE_OM_LEASE_HARD_LIMIT = + "ozone.om.lease.hard.limit"; + public static final String OZONE_OM_LEASE_HARD_LIMIT_DEFAULT = + "7d"; + public static final String OZONE_OM_OPEN_KEY_CLEANUP_LIMIT_PER_TASK = "ozone.om.open.key.cleanup.limit.per.task"; public static final int OZONE_OM_OPEN_KEY_CLEANUP_LIMIT_PER_TASK_DEFAULT = diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java index 2e5a19685204..3897a0228346 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java @@ -268,7 +268,9 @@ public enum ResultCodes { S3_SECRET_ALREADY_EXISTS, INVALID_PATH, - - TOO_MANY_BUCKETS + TOO_MANY_BUCKETS, + KEY_UNDER_LEASE_RECOVERY, + KEY_ALREADY_CLOSED, + KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD } } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/LeaseKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/LeaseKeyInfo.java new file mode 100644 index 000000000000..a97ca6816828 --- /dev/null +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/LeaseKeyInfo.java @@ -0,0 +1,43 @@ +/* + * 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.ozone.om.helpers; + +/** + * This class represents LeaseKeyInfo. + */ +public class LeaseKeyInfo { + private final OmKeyInfo keyInfo; + /** + * isKeyInfo = true indicates keyInfo is from keyTable. + * isKeyInfo = false indicates keyInfo is from openKeyTable. + */ + private boolean isKeyInfo; + + public LeaseKeyInfo(OmKeyInfo info, boolean isKeyInfo) { + this.keyInfo = info; + this.isKeyInfo = isKeyInfo; + } + + public boolean getIsKeyInfo() { + return this.isKeyInfo; + } + + public OmKeyInfo getKeyInfo() { + return keyInfo; + } +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/ListOpenFilesResult.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/ListOpenFilesResult.java new file mode 100644 index 000000000000..67ef5dfec255 --- /dev/null +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/ListOpenFilesResult.java @@ -0,0 +1,115 @@ +/** + * 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.ozone.om.helpers; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Encapsulates the result of listOpenFiles. It has a list of + * {@link OpenKeySession} and a boolean flag indicating if there + * are more entries that are not fetched after the current batch of result. + */ +public class ListOpenFilesResult { + /** + * Number of total open files globally. + */ + @JsonProperty("totalOpenKeyCount") + private final long totalOpenKeyCount; + /** + * True if there are more entries after this batch under the given path. + */ + @JsonProperty("hasMore") + private final boolean hasMore; + /** + * True if there are more entries after this batch under the given path. + */ + @JsonProperty("contToken") + private final String continuationToken; + /** + * List of open files. Each has client ID and OmKeyInfo. + */ + private final List openKeySessionList; + + public ListOpenFilesResult(long totalOpenKeyCount, + boolean hasMore, + String continuationToken, + List openKeySessionList) { + this.openKeySessionList = openKeySessionList; + this.hasMore = hasMore; + this.continuationToken = continuationToken; + this.totalOpenKeyCount = totalOpenKeyCount; + } + + public ListOpenFilesResult(long totalOpenKeyCount, + boolean hasMore, + String continuationToken, + List clientIDsList, + List keyInfosList) + throws IOException { + this.openKeySessionList = getOpenKeySessionListFromPB(clientIDsList, + keyInfosList); + this.hasMore = hasMore; + this.continuationToken = continuationToken; + this.totalOpenKeyCount = totalOpenKeyCount; + } + + /** + * Combines clientIDsList and keyInfosList into OpenKeySessionList for + * transfer to the client. + */ + private List getOpenKeySessionListFromPB( + List clientIDsList, List keyInfosList) + throws IOException { + + Preconditions.checkArgument(clientIDsList.size() == keyInfosList.size(), + "clientIDsList size (" + clientIDsList.size() + ") should be " + + "the same as keyInfosList's (" + keyInfosList.size() + ")"); + + List res = new ArrayList<>(); + + for (int i = 0; i < clientIDsList.size(); i++) { + OmKeyInfo omKeyInfo = OmKeyInfo.getFromProtobuf(keyInfosList.get(i)); + res.add(new OpenKeySession(clientIDsList.get(i), + omKeyInfo, + omKeyInfo.getLatestVersionLocations().getVersion())); + } + return res; + } + + public long getTotalOpenKeyCount() { + return totalOpenKeyCount; + } + + public boolean hasMore() { + return hasMore; + } + + public String getContinuationToken() { + return continuationToken; + } + + public List getOpenKeys() { + return openKeySessionList; + } +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java index 11ee622494dd..d3b32815d822 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java @@ -17,12 +17,15 @@ */ package org.apache.hadoop.ozone.om.helpers; +import com.fasterxml.jackson.annotation.JsonProperty; + /** * This class represents a open key "session". A session here means a key is * opened by a specific client, the client sends the handler to server, such * that servers can recognize this client, and thus know how to close the key. */ public class OpenKeySession { + @JsonProperty("clientId") private final long id; private final OmKeyInfo keyInfo; // the version of the key when it is being opened in this session. diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java index ab3f576d4492..b1f572358362 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java @@ -33,6 +33,8 @@ import org.apache.hadoop.ozone.om.helpers.DBUpdates; import org.apache.hadoop.ozone.om.helpers.DeleteTenantState; import org.apache.hadoop.ozone.om.helpers.KeyInfoWithVolumeContext; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmDeleteKeys; @@ -269,6 +271,19 @@ default void hsyncKey(OmKeyArgs args, long clientID) "this to be implemented, as write requests use a new approach."); } + /** + * Recovery and commit a key. This will make the change from the client visible. The client + * is identified by the clientID. + * + * @param args the key to commit + * @param clientID the client identification + * @throws IOException + */ + default void recoverKey(OmKeyArgs args, long clientID) + throws IOException { + throw new UnsupportedOperationException("OzoneManager does not require " + + "this to be implemented, as write requests use a new approach."); + } /** * Allocate a new block, it is assumed that the client is having an open key @@ -408,6 +423,17 @@ List listBuckets(String volumeName, ServiceInfoEx getServiceInfo() throws IOException; + /** + * List open files in OM. + * @param path One of: root "/", path to a bucket, key path, or key prefix + * @param maxKeys Limit the number of keys that can be returned in this batch. + * @param contToken Continuation token. + * @return ListOpenFilesResult + * @throws IOException + */ + ListOpenFilesResult listOpenFiles(String path, int maxKeys, String contToken) + throws IOException; + /** * Transfer the raft leadership. * @@ -1101,11 +1127,11 @@ EchoRPCResponse echoRPCReq(byte[] payloadReq, int payloadSizeResp, * @param volumeName - The volume name. * @param bucketName - The bucket name. * @param keyName - The key user want to recover. - * @return true if the file is already closed + * @param force - force recover the file. + * @return LeaseKeyInfo KeyInfo of file under recovery * @throws IOException if an error occurs */ - boolean recoverLease(String volumeName, String bucketName, - String keyName) throws IOException; + LeaseKeyInfo recoverLease(String volumeName, String bucketName, String keyName, boolean force) throws IOException; /** * Update modification time and access time of a file. diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java index 08fa029833e7..0a17f4527fc3 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java @@ -47,6 +47,8 @@ import org.apache.hadoop.ozone.om.helpers.DeleteTenantState; import org.apache.hadoop.ozone.om.helpers.KeyInfoWithVolumeContext; import org.apache.hadoop.ozone.om.helpers.KeyValueUtil; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmDeleteKeys; @@ -137,6 +139,8 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListKeysResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListMultipartUploadsRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListMultipartUploadsResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListOpenFilesRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListOpenFilesResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListStatusLightResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListStatusRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListStatusResponse; @@ -792,13 +796,19 @@ public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientId, @Override public void hsyncKey(OmKeyArgs args, long clientId) throws IOException { - updateKey(args, clientId, true); + updateKey(args, clientId, true, false); } @Override public void commitKey(OmKeyArgs args, long clientId) throws IOException { - updateKey(args, clientId, false); + updateKey(args, clientId, false, false); + } + + @Override + public void recoverKey(OmKeyArgs args, long clientId) + throws IOException { + updateKey(args, clientId, false, true); } public static void setReplicationConfig(ReplicationConfig replication, @@ -814,7 +824,7 @@ public static void setReplicationConfig(ReplicationConfig replication, b.setType(replication.getReplicationType()); } - private void updateKey(OmKeyArgs args, long clientId, boolean hsync) + private void updateKey(OmKeyArgs args, long clientId, boolean hsync, boolean recovery) throws IOException { CommitKeyRequest.Builder req = CommitKeyRequest.newBuilder(); List locationInfoList = args.getLocationInfoList(); @@ -835,15 +845,13 @@ private void updateKey(OmKeyArgs args, long clientId, boolean hsync) req.setKeyArgs(keyArgsBuilder.build()); req.setClientID(clientId); req.setHsync(hsync); - + req.setRecovery(recovery); OMRequest omRequest = createOMRequest(Type.CommitKey) .setCommitKeyRequest(req) .build(); handleError(submitRequest(omRequest)); - - } @Override @@ -1825,6 +1833,32 @@ public ServiceInfoEx getServiceInfo() throws IOException { resp.getCaCertificate(), resp.getCaCertsList()); } + @Override + public ListOpenFilesResult listOpenFiles(String path, + int maxKeys, + String contToken) + throws IOException { + ListOpenFilesRequest req = ListOpenFilesRequest.newBuilder() + .setPath(path) + .setCount(maxKeys) + .setToken(contToken) + .build(); + + OMRequest omRequest = createOMRequest(Type.ListOpenFiles) + .setListOpenFilesRequest(req) + .build(); + + final ListOpenFilesResponse resp = handleError(submitRequest(omRequest)) + .getListOpenFilesResponse(); + + return new ListOpenFilesResult( + resp.getTotalOpenKeyCount(), + resp.getHasMore(), + resp.hasContinuationToken() ? resp.getContinuationToken() : null, + resp.getClientIDList(), + resp.getKeyInfoList()); + } + @Override public void transferLeadership(String newLeaderId) throws IOException { @@ -2509,21 +2543,24 @@ public EchoRPCResponse echoRPCReq(byte[] payloadReq, int payloadSizeResp, } @Override - public boolean recoverLease(String volumeName, String bucketName, - String keyName) throws IOException { + public LeaseKeyInfo recoverLease(String volumeName, String bucketName, String keyName, boolean force) + throws IOException { RecoverLeaseRequest recoverLeaseRequest = - RecoverLeaseRequest.newBuilder() - .setVolumeName(volumeName) - .setBucketName(bucketName) - .setKeyName(keyName) - .build(); + RecoverLeaseRequest.newBuilder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .setForce(force) + .build(); OMRequest omRequest = createOMRequest(Type.RecoverLease) - .setRecoverLeaseRequest(recoverLeaseRequest).build(); + .setRecoverLeaseRequest(recoverLeaseRequest).build(); RecoverLeaseResponse recoverLeaseResponse = - handleError(submitRequest(omRequest)).getRecoverLeaseResponse(); - return recoverLeaseResponse.getResponse(); + handleError(submitRequest(omRequest)).getRecoverLeaseResponse(); + + return new LeaseKeyInfo(OmKeyInfo.getFromProtobuf(recoverLeaseResponse.getKeyInfo()), + recoverLeaseResponse.getIsKeyInfo()); } @Override diff --git a/hadoop-ozone/dist/src/main/compose/ozone/docker-config b/hadoop-ozone/dist/src/main/compose/ozone/docker-config index 83057f9d394e..30fa84406485 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone/docker-config +++ b/hadoop-ozone/dist/src/main/compose/ozone/docker-config @@ -54,6 +54,8 @@ OZONE-SITE.XML_hdds.scm.wait.time.after.safemode.exit=30s OZONE-SITE.XML_hdds.container.ratis.datastream.enabled=true +OZONE-SITE.XML_ozone.fs.hsync.enabled=true + OZONE_CONF_DIR=/etc/hadoop OZONE_LOG_DIR=/var/log/hadoop diff --git a/hadoop-ozone/dist/src/main/smoketest/freon/echoRPCLoad.robot b/hadoop-ozone/dist/src/main/smoketest/freon/echoRPCLoad.robot index 32456af48869..c6ea4e63468e 100644 --- a/hadoop-ozone/dist/src/main/smoketest/freon/echoRPCLoad.robot +++ b/hadoop-ozone/dist/src/main/smoketest/freon/echoRPCLoad.robot @@ -23,6 +23,27 @@ ${PREFIX} ${EMPTY} ${n} 1 *** Test Cases *** +Get Container ID + ${result} = Execute ozone admin container create + ${containerID} = Execute ozone admin container list --count 1 --state=OPEN | grep -o '"containerID" *: *[^,}]*' | awk -F'[:,]' '{print $2}' | tr -d '" ' + Set Suite Variable ${containerID} + +[Read] Ozone DataNode Echo RPC Load Generator with request payload and response payload + ${result} = Execute ozone freon dne -t=1 -n=${n} --payload-req=1 --payload-resp=1 --container-id=${containerID} + Should contain ${result} Successful executions: ${n} + +[Read] Ozone DataNode Echo RPC Load Generator with request payload and empty response payload + ${result} = Execute ozone freon dne -t=1 -n=${n} --payload-req=1 --container-id=${containerID} + Should contain ${result} Successful executions: ${n} + +[Read] Ozone DataNode Echo RPC Load Generator with empty request payload and response payload + ${result} = Execute ozone freon dne -t=1 -n=${n} --payload-resp=1 --container-id=${containerID} + Should contain ${result} Successful executions: ${n} + +[Read] Ozone DataNode Echo RPC Load Generator with empty request payload and empty response payload no sleep time one xceiver client + ${result} = Execute ozone freon dne -t=1 -n=${n} --sleep-time-ms=0 --clients=1 --container-id=${containerID} + Should contain ${result} Successful executions: ${n} + [Read] Ozone Echo RPC Load Generator with request payload and response payload ${result} = Execute ozone freon ome -t=1 -n=${n} --payload-req=1 --payload-resp=1 Should contain ${result} Successful executions: ${n} diff --git a/hadoop-ozone/fault-injection-test/pom.xml b/hadoop-ozone/fault-injection-test/pom.xml index 7de9bcc297da..432faab48777 100644 --- a/hadoop-ozone/fault-injection-test/pom.xml +++ b/hadoop-ozone/fault-injection-test/pom.xml @@ -33,4 +33,12 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> mini-chaos-tests + + + org.mockito + mockito-inline + test + + + diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java index 1cc7ee21a6ba..430ec4e03fd2 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java @@ -183,6 +183,7 @@ void init() throws Exception { conf.setBoolean(OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY, omRatisEnabled); conf.setBoolean(OZONE_ACL_ENABLED, true); conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); + conf.set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "0s"); if (!bucketLayout.equals(FILE_SYSTEM_OPTIMIZED)) { conf.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS, enabledFileSystemPaths); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractRootedOzoneFileSystemTest.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractRootedOzoneFileSystemTest.java index ff136b3caead..a092890ae2a4 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractRootedOzoneFileSystemTest.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractRootedOzoneFileSystemTest.java @@ -238,6 +238,7 @@ void initClusterAndEnv() throws IOException, InterruptedException, TimeoutExcept conf.setFloat(FS_TRASH_CHECKPOINT_INTERVAL_KEY, TRASH_INTERVAL / 2); conf.setBoolean(OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY, omRatisEnabled); conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); + conf.set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "0s"); if (bucketLayout == BucketLayout.FILE_SYSTEM_OPTIMIZED) { conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT, bucketLayout.name()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java index 31ac58ad432d..daa433f68f8a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java @@ -21,31 +21,45 @@ import java.io.Closeable; import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.security.GeneralSecurityException; import java.security.PrivilegedExceptionAction; +import java.util.HashMap; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Stream; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.crypto.CipherSuite; import org.apache.hadoop.crypto.CryptoCodec; import org.apache.hadoop.crypto.CryptoOutputStream; import org.apache.hadoop.crypto.Encryptor; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileAlreadyExistsException; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.scm.storage.BlockOutputStream; +import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.client.DefaultReplicationConfig; import org.apache.hadoop.hdds.client.ECReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.StorageType; -import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StreamCapabilities; + import org.apache.hadoop.ozone.ClientConfigForTesting; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConfigKeys; @@ -54,31 +68,44 @@ import org.apache.hadoop.ozone.client.BucketArgs; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.OzoneKeyDetails; import org.apache.hadoop.ozone.client.io.ECKeyOutputStream; import org.apache.hadoop.ozone.client.io.KeyOutputStream; +import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; +import org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl; +import org.apache.hadoop.ozone.container.metadata.AbstractDatanodeStore; import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OMMetrics; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; -import org.apache.hadoop.ozone.om.request.key.OMKeyCommitRequest; -import org.apache.hadoop.ozone.om.request.key.OMKeyCommitRequestWithFSO; -import org.apache.hadoop.ozone.om.request.key.OMKeyRequest; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Time; import org.apache.ozone.test.GenericTestUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.MethodOrderer.OrderAnnotation; +import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_CHUNK_LIST_INCREMENTAL; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_RATIS_PIPELINE_LIMIT; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_ROOT; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER; @@ -86,11 +113,14 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.params.provider.Arguments.arguments; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -98,6 +128,7 @@ * Test HSync. */ @Timeout(value = 300) +@TestMethodOrder(OrderAnnotation.class) public class TestHSync { private static final Logger LOG = LoggerFactory.getLogger(TestHSync.class); @@ -107,27 +138,34 @@ public class TestHSync { private static final OzoneConfiguration CONF = new OzoneConfiguration(); private static OzoneClient client; + private static final BucketLayout BUCKET_LAYOUT = BucketLayout.FILE_SYSTEM_OPTIMIZED; + + private static final int CHUNK_SIZE = 4 << 12; + private static final int FLUSH_SIZE = 2 * CHUNK_SIZE; + private static final int MAX_FLUSH_SIZE = 2 * FLUSH_SIZE; + private static final int BLOCK_SIZE = 2 * MAX_FLUSH_SIZE; @BeforeAll public static void init() throws Exception { - final int chunkSize = 16 << 10; - final int flushSize = 2 * chunkSize; - final int maxFlushSize = 2 * flushSize; - final int blockSize = 2 * maxFlushSize; - final BucketLayout layout = BucketLayout.FILE_SYSTEM_OPTIMIZED; + final BucketLayout layout = BUCKET_LAYOUT; CONF.setBoolean(OZONE_OM_RATIS_ENABLE_KEY, false); CONF.set(OZONE_DEFAULT_BUCKET_LAYOUT, layout.name()); CONF.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); CONF.setInt(OZONE_SCM_RATIS_PIPELINE_LIMIT, 10); + // Reduce KeyDeletingService interval + CONF.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 100, TimeUnit.MILLISECONDS); + CONF.setBoolean("ozone.client.incremental.chunk.list", true); + CONF.setBoolean("ozone.client.stream.putblock.piggybacking", true); + CONF.setBoolean(OZONE_CHUNK_LIST_INCREMENTAL, true); ClientConfigForTesting.newBuilder(StorageUnit.BYTES) - .setBlockSize(blockSize) - .setChunkSize(chunkSize) - .setStreamBufferFlushSize(flushSize) - .setStreamBufferMaxSize(maxFlushSize) - .setDataStreamBufferFlushSize(maxFlushSize) - .setDataStreamMinPacketSize(chunkSize) - .setDataStreamWindowSize(5 * chunkSize) + .setBlockSize(BLOCK_SIZE) + .setChunkSize(CHUNK_SIZE) + .setStreamBufferFlushSize(FLUSH_SIZE) + .setStreamBufferMaxSize(MAX_FLUSH_SIZE) + .setDataStreamBufferFlushSize(MAX_FLUSH_SIZE) + .setDataStreamMinPacketSize(CHUNK_SIZE) + .setDataStreamWindowSize(5 * CHUNK_SIZE) .applyTo(CONF); cluster = MiniOzoneCluster.newBuilder(CONF) @@ -140,9 +178,11 @@ public static void init() throws Exception { bucket = TestDataUtil.createVolumeAndBucket(client, layout); // Enable DEBUG level logging for relevant classes - GenericTestUtils.setLogLevel(OMKeyRequest.LOG, Level.DEBUG); - GenericTestUtils.setLogLevel(OMKeyCommitRequest.LOG, Level.DEBUG); - GenericTestUtils.setLogLevel(OMKeyCommitRequestWithFSO.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(BlockManagerImpl.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(AbstractDatanodeStore.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(BlockOutputStream.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(BlockInputStream.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(KeyValueHandler.LOG, Level.DEBUG); } @AfterAll @@ -153,6 +193,83 @@ public static void teardown() { } } + @Test + // Making this the first test to be run to avoid db key composition headaches + @Order(1) + public void testKeyMetadata() throws Exception { + // Tests key metadata behavior upon create(), hsync() and close(): + // 1. When a key is create()'d, neither OpenKeyTable nor KeyTable entry shall have hsync metadata. + // 2. When the key is hsync()'ed, both OpenKeyTable and KeyTable shall have hsync metadata. + // 3. When the key is hsync()'ed again, both OpenKeyTable and KeyTable shall have hsync metadata. + // 4. When the key is close()'d, KeyTable entry shall not have hsync metadata. Key shall not exist in OpenKeyTable. + + final String rootPath = String.format("%s://%s/", + OZONE_OFS_URI_SCHEME, CONF.get(OZONE_OM_ADDRESS_KEY)); + CONF.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + + final String dir = OZONE_ROOT + bucket.getVolumeName() + + OZONE_URI_DELIMITER + bucket.getName(); + final String keyName = "file-test-key-metadata"; + final Path file = new Path(dir, keyName); + + OMMetadataManager omMetadataManager = + cluster.getOzoneManager().getMetadataManager(); + + // Expect empty OpenKeyTable and KeyTable before key creation + Table openKeyTable = omMetadataManager.getOpenKeyTable(BUCKET_LAYOUT); + assertTrue(openKeyTable.isEmpty()); + Table keyTable = omMetadataManager.getKeyTable(BUCKET_LAYOUT); + assertTrue(keyTable.isEmpty()); + + try (FileSystem fs = FileSystem.get(CONF)) { + try (FSDataOutputStream os = fs.create(file, true)) { + // Wait for double buffer flush to avoid flakiness because RDB iterator bypasses table cache + cluster.getOzoneManager().awaitDoubleBufferFlush(); + // OpenKeyTable key should NOT have HSYNC_CLIENT_ID + OmKeyInfo keyInfo = getFirstKeyInTable(keyName, openKeyTable); + assertFalse(keyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID)); + // KeyTable should still be empty + assertTrue(keyTable.isEmpty()); + + os.hsync(); + cluster.getOzoneManager().awaitDoubleBufferFlush(); + // OpenKeyTable key should have HSYNC_CLIENT_ID now + keyInfo = getFirstKeyInTable(keyName, openKeyTable); + assertTrue(keyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID)); + // KeyTable key should be there and have HSYNC_CLIENT_ID + keyInfo = getFirstKeyInTable(keyName, keyTable); + assertTrue(keyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID)); + + // hsync again, metadata should not change + os.hsync(); + cluster.getOzoneManager().awaitDoubleBufferFlush(); + keyInfo = getFirstKeyInTable(keyName, openKeyTable); + assertTrue(keyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID)); + keyInfo = getFirstKeyInTable(keyName, keyTable); + assertTrue(keyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID)); + } + // key is closed, OpenKeyTable should be empty + cluster.getOzoneManager().awaitDoubleBufferFlush(); + assertTrue(openKeyTable.isEmpty()); + // KeyTable should have the key. But the key shouldn't have metadata HSYNC_CLIENT_ID anymore + OmKeyInfo keyInfo = getFirstKeyInTable(keyName, keyTable); + assertFalse(keyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID)); + + // Clean up + assertTrue(fs.delete(file, false)); + // Wait for KeyDeletingService to finish to avoid interfering other tests + Table deletedTable = omMetadataManager.getDeletedTable(); + GenericTestUtils.waitFor( + () -> { + try { + return deletedTable.isEmpty(); + } catch (IOException e) { + return false; + } + }, 250, 10000); + } + } + @Test public void testKeyHSyncThenClose() throws Exception { // Check that deletedTable should not have keys with the same block as in @@ -198,13 +315,15 @@ public void testKeyHSyncThenClose() throws Exception { } } - @Test - public void testO3fsHSync() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testO3fsHSync(boolean incrementalChunkList) throws Exception { // Set the fs.defaultFS final String rootPath = String.format("%s://%s.%s/", OZONE_URI_SCHEME, bucket.getName(), bucket.getVolumeName()); CONF.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + initClientConfig(incrementalChunkList); try (FileSystem fs = FileSystem.get(CONF)) { for (int i = 0; i < 10; i++) { final Path file = new Path("/file" + i); @@ -213,8 +332,10 @@ public void testO3fsHSync() throws Exception { } } - @Test - public void testOfsHSync() throws Exception { + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testOfsHSync(boolean incrementalChunkList) throws Exception { // Set the fs.defaultFS final String rootPath = String.format("%s://%s/", OZONE_OFS_URI_SCHEME, CONF.get(OZONE_OM_ADDRESS_KEY)); @@ -223,6 +344,7 @@ public void testOfsHSync() throws Exception { final String dir = OZONE_ROOT + bucket.getVolumeName() + OZONE_URI_DELIMITER + bucket.getName(); + initClientConfig(incrementalChunkList); try (FileSystem fs = FileSystem.get(CONF)) { for (int i = 0; i < 10; i++) { final Path file = new Path(dir, "file" + i); @@ -324,6 +446,50 @@ public void testOverwriteHSyncFile() throws Exception { } } + @Test + public void testHsyncKeyCallCount() throws Exception { + // Set the fs.defaultFS + final String rootPath = String.format("%s://%s/", + OZONE_OFS_URI_SCHEME, CONF.get(OZONE_OM_ADDRESS_KEY)); + CONF.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + + final String dir = OZONE_ROOT + bucket.getVolumeName() + + OZONE_URI_DELIMITER + bucket.getName(); + + OMMetrics omMetrics = cluster.getOzoneManager().getMetrics(); + omMetrics.resetNumKeyHSyncs(); + final byte[] data = new byte[128]; + ThreadLocalRandom.current().nextBytes(data); + + final Path file = new Path(dir, "file-hsync-then-close"); + try (FileSystem fs = FileSystem.get(CONF)) { + long fileSize = 0; + try (FSDataOutputStream outputStream = fs.create(file, true)) { + // make sure at least writing 2 blocks data + while (fileSize <= BLOCK_SIZE) { + outputStream.write(data, 0, data.length); + outputStream.hsync(); + fileSize += data.length; + } + } + } + assertEquals(2, omMetrics.getNumKeyHSyncs()); + + // test file with all blocks pre-allocated + omMetrics.resetNumKeyHSyncs(); + long writtenSize = 0; + try (OzoneOutputStream outputStream = bucket.createKey("key-" + RandomStringUtils.randomNumeric(5), + BLOCK_SIZE * 2, ReplicationType.RATIS, ReplicationFactor.THREE, new HashMap<>())) { + // make sure at least writing 2 blocks data + while (writtenSize <= BLOCK_SIZE) { + outputStream.write(data, 0, data.length); + outputStream.hsync(); + writtenSize += data.length; + } + } + assertEquals(2, omMetrics.getNumKeyHSyncs()); + } + static void runTestHSync(FileSystem fs, Path file, int initialDataSize) throws Exception { try (StreamWithLength out = new StreamWithLength( @@ -384,7 +550,8 @@ static void runTestHSync(FileSystem fs, Path file, break; } for (int i = 0; i < n; i++) { - assertEquals(data[offset + i], buffer[i]); + assertEquals(data[offset + i], buffer[i], + "expected at offset " + offset + " i=" + i); } offset += n; } @@ -454,7 +621,7 @@ public void testConcurrentWriteHSync() + OZONE_URI_DELIMITER + bucket.getName(); try (FileSystem fs = FileSystem.get(CONF)) { - for (int i = 0; i < 10; i++) { + for (int i = 0; i < 5; i++) { final Path file = new Path(dir, "file" + i); try (FSDataOutputStream out = fs.create(file, true)) { @@ -550,6 +717,23 @@ public void testDisableHsync() throws Exception { } } + /** + * Helper method to check and get the first key in the OpenKeyTable. + * @param keyName expect key name to contain this string + * @param openKeyTable Table + * @return OmKeyInfo + */ + private OmKeyInfo getFirstKeyInTable(String keyName, Table openKeyTable) throws IOException { + try (TableIterator> it = openKeyTable.iterator()) { + assertTrue(it.hasNext()); + Table.KeyValue kv = it.next(); + String dbOpenKey = kv.getKey(); + assertNotNull(dbOpenKey); + assertTrue(dbOpenKey.contains(keyName)); + return kv.getValue(); + } + } + private void testEncryptedStreamCapabilities(boolean isEC) throws IOException, GeneralSecurityException { KeyOutputStream kos; @@ -581,4 +765,117 @@ private void testEncryptedStreamCapabilities(boolean isEC) throws IOException, assertFalse(cofsos.hasCapability(StreamCapabilities.HFLUSH)); } } + + public void initClientConfig(boolean incrementalChunkList) { + OzoneClientConfig clientConfig = CONF.getObject(OzoneClientConfig.class); + clientConfig.setIncrementalChunkList(incrementalChunkList); + clientConfig.setChecksumType(ContainerProtos.ChecksumType.CRC32C); + CONF.setFromObject(clientConfig); + } + + public static Stream parameters1() { + return Stream.of( + arguments(true, 512), + arguments(true, 511), + arguments(true, 513), + arguments(false, 512), + arguments(false, 511), + arguments(false, 513) + ); + } + + @ParameterizedTest + @MethodSource("parameters1") + public void writeWithSmallBuffer(boolean incrementalChunkList, int bufferSize) + throws IOException { + initClientConfig(incrementalChunkList); + + final String keyName = UUID.randomUUID().toString(); + int fileSize = 16 << 11; + String s = RandomStringUtils.randomAlphabetic(bufferSize); + ByteBuffer byteBuffer = ByteBuffer.wrap(s.getBytes(StandardCharsets.UTF_8)); + + int writtenSize = 0; + try (OzoneOutputStream out = bucket.createKey(keyName, fileSize, + ReplicationConfig.getDefault(CONF), new HashMap<>())) { + while (writtenSize < fileSize) { + int len = Math.min(bufferSize, fileSize - writtenSize); + out.write(byteBuffer, 0, len); + out.hsync(); + writtenSize += bufferSize; + } + } + + OzoneKeyDetails keyInfo = bucket.getKey(keyName); + assertEquals(fileSize, keyInfo.getDataSize()); + + int readSize = 0; + try (OzoneInputStream is = bucket.readKey(keyName)) { + while (readSize < fileSize) { + int len = Math.min(bufferSize, fileSize - readSize); + ByteBuffer readBuffer = ByteBuffer.allocate(len); + int readLen = is.read(readBuffer); + assertEquals(len, readLen); + if (len < bufferSize) { + for (int i = 0; i < len; i++) { + assertEquals(readBuffer.array()[i], byteBuffer.array()[i]); + } + } else { + assertArrayEquals(readBuffer.array(), byteBuffer.array()); + } + readSize += readLen; + } + } + bucket.deleteKey(keyName); + } + + public static Stream parameters2() { + return Stream.of( + arguments(true, 1024 * 1024 + 1), + arguments(true, 1024 * 1024 + 1 + CHUNK_SIZE), + arguments(true, 1024 * 1024 - 1 + CHUNK_SIZE), + arguments(false, 1024 * 1024 + 1), + arguments(false, 1024 * 1024 + 1 + CHUNK_SIZE), + arguments(false, 1024 * 1024 - 1 + CHUNK_SIZE) + ); + } + + @ParameterizedTest + @MethodSource("parameters2") + public void writeWithBigBuffer(boolean incrementalChunkList, int bufferSize) + throws IOException { + initClientConfig(incrementalChunkList); + + final String keyName = UUID.randomUUID().toString(); + int count = 2; + int fileSize = bufferSize * count; + ByteBuffer byteBuffer = ByteBuffer.allocate(bufferSize); + + try (OzoneOutputStream out = bucket.createKey(keyName, fileSize, + ReplicationConfig.getDefault(CONF), new HashMap<>())) { + for (int i = 0; i < count; i++) { + out.write(byteBuffer); + out.hsync(); + } + } + + OzoneKeyDetails keyInfo = bucket.getKey(keyName); + assertEquals(fileSize, keyInfo.getDataSize()); + int totalReadLen = 0; + try (OzoneInputStream is = bucket.readKey(keyName)) { + + for (int i = 0; i < count; i++) { + ByteBuffer readBuffer = ByteBuffer.allocate(bufferSize); + int readLen = is.read(readBuffer); + if (bufferSize != readLen) { + throw new IOException("failed to read " + bufferSize + " from offset " + totalReadLen + + ", actually read " + readLen + ", block " + totalReadLen / + BLOCK_SIZE); + } + assertArrayEquals(byteBuffer.array(), readBuffer.array()); + totalReadLen += readLen; + } + } + bucket.deleteKey(keyName); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java index ffd54cfea869..64029b05180e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java @@ -17,33 +17,54 @@ */ package org.apache.hadoop.fs.ozone; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; 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.Path; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.XceiverClientGrpc; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; +import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.ozone.ClientConfigForTesting; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneTestUtils; import org.apache.hadoop.ozone.TestDataUtil; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.utils.FaultInjectorImpl; +import org.apache.ozone.test.GenericTestUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.slf4j.event.Level; import java.io.IOException; import java.io.OutputStream; +import java.net.ConnectException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeoutException; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_RATIS_PIPELINE_LIMIT; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_WAIT_BETWEEN_RETRIES_MILLIS_KEY; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT; +import static org.apache.hadoop.ozone.OzoneConsts.FORCE_LEASE_RECOVERY_ENV; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_ROOT; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER; @@ -67,17 +88,22 @@ public class TestLeaseRecovery { private OzoneClient client; private final OzoneConfiguration conf = new OzoneConfiguration(); + private String dir; + private Path file; /** * Closing the output stream after lease recovery throws because the key * is no longer open in OM. This is currently expected (see HDDS-9358). */ - public static void closeIgnoringKeyNotFound(OutputStream stream) - throws IOException { + public static void closeIgnoringKeyNotFound(OutputStream stream) { + closeIgnoringOMException(stream, OMException.ResultCodes.KEY_NOT_FOUND); + } + + public static void closeIgnoringOMException(OutputStream stream, OMException.ResultCodes expectedResultCode) { try { stream.close(); - } catch (OMException e) { - assertEquals(OMException.ResultCodes.KEY_NOT_FOUND, e.getResult()); + } catch (IOException e) { + assertEquals(expectedResultCode, ((OMException)e).getResult()); } } @@ -94,6 +120,9 @@ public void init() throws IOException, InterruptedException, conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); conf.set(OZONE_DEFAULT_BUCKET_LAYOUT, layout.name()); conf.setInt(OZONE_SCM_RATIS_PIPELINE_LIMIT, 10); + conf.set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "0s"); + // make sure flush will write data to DN + conf.setBoolean("ozone.client.stream.buffer.flush.delay", false); ClientConfigForTesting.newBuilder(StorageUnit.BYTES) .setBlockSize(blockSize) @@ -106,13 +135,21 @@ public void init() throws IOException, InterruptedException, .applyTo(conf); cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(5) + .setNumDatanodes(3) .build(); cluster.waitForClusterToBeReady(); client = cluster.newClient(); // create a volume and a bucket to be used by OzoneFileSystem bucket = TestDataUtil.createVolumeAndBucket(client, layout); + + GenericTestUtils.setLogLevel(XceiverClientGrpc.getLogger(), Level.DEBUG); + + // Set the fs.defaultFS + final String rootPath = String.format("%s://%s/", OZONE_OFS_URI_SCHEME, conf.get(OZONE_OM_ADDRESS_KEY)); + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + dir = OZONE_ROOT + bucket.getVolumeName() + OZONE_URI_DELIMITER + bucket.getName(); + file = new Path(dir, "file"); } @AfterEach @@ -123,21 +160,48 @@ public void tearDown() { } } - @Test - public void testRecovery() throws Exception { - // Set the fs.defaultFS - final String rootPath = String.format("%s://%s/", - OZONE_OFS_URI_SCHEME, conf.get(OZONE_OM_ADDRESS_KEY)); - conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + @ParameterizedTest + @ValueSource(ints = {1 << 20, (1 << 20) + 1, (1 << 20) - 1}) + public void testRecovery(int dataSize) throws Exception { + RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); - final String dir = OZONE_ROOT + bucket.getVolumeName() - + OZONE_URI_DELIMITER + bucket.getName(); - final Path file = new Path(dir, "file"); + final byte[] data = getData(dataSize); + final FSDataOutputStream stream = fs.create(file, true); + try { + stream.write(data); + stream.hsync(); + assertFalse(fs.isFileClosed(file)); + + // write more data without hsync + stream.write(data); + stream.flush(); + + int count = 0; + while (count++ < 15 && !fs.recoverLease(file)) { + Thread.sleep(1000); + } + // The lease should have been recovered. + assertTrue(fs.isFileClosed(file), "File should be closed"); + + // A second call to recoverLease should succeed too. + assertTrue(fs.recoverLease(file)); + } finally { + closeIgnoringKeyNotFound(stream); + } + + // open it again, make sure the data is correct + verifyData(data, dataSize * 2, file, fs); + } + + @Test + public void testRecoveryWithoutHsyncHflushOnLastBlock() throws Exception { RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); - final byte[] data = new byte[1 << 20]; - ThreadLocalRandom.current().nextBytes(data); + int blockSize = (int) cluster.getOzoneManager().getConfiguration().getStorageSize( + OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES); + + final byte[] data = getData(blockSize / 2 + 1); final FSDataOutputStream stream = fs.create(file, true); try { @@ -145,24 +209,25 @@ public void testRecovery() throws Exception { stream.hsync(); assertFalse(fs.isFileClosed(file)); + // It will write into new block as well + // Don't do hsync/flush + stream.write(data); + int count = 0; while (count++ < 15 && !fs.recoverLease(file)) { Thread.sleep(1000); } // The lease should have been recovered. - assertTrue(fs.recoverLease(file), "File should be closed"); - assertTrue(fs.isFileClosed(file)); + assertTrue(fs.isFileClosed(file), "File should be closed"); + + // A second call to recoverLease should succeed too. + assertTrue(fs.recoverLease(file)); } finally { closeIgnoringKeyNotFound(stream); } // open it again, make sure the data is correct - byte[] readData = new byte[1 << 20]; - try (FSDataInputStream fdis = fs.open(file)) { - int readBytes = fdis.read(readData); - assertEquals(readBytes, 1 << 20); - assertArrayEquals(readData, data); - } + verifyData(data, blockSize / 2 + 1, file, fs); } @Test @@ -174,11 +239,288 @@ public void testOBSRecoveryShouldFail() throws Exception { conf.get(OZONE_OM_ADDRESS_KEY)); conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); - final String dir = OZONE_ROOT + bucket.getVolumeName() + + final String directory = OZONE_ROOT + bucket.getVolumeName() + OZONE_URI_DELIMITER + bucket.getName(); - final Path file = new Path(dir, "file"); + final Path f = new Path(directory, "file"); RootedOzoneFileSystem fs = (RootedOzoneFileSystem) FileSystem.get(conf); - assertThrows(IllegalArgumentException.class, () -> fs.recoverLease(file)); + assertThrows(IllegalArgumentException.class, () -> fs.recoverLease(f)); + } + + @Test + public void testFinalizeBlockFailure() throws Exception { + RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); + int dataSize = 100; + final byte[] data = getData(dataSize); + + final FSDataOutputStream stream = fs.create(file, true); + try { + stream.write(data); + stream.hsync(); + assertFalse(fs.isFileClosed(file)); + + // write more data without hsync + stream.write(data); + stream.flush(); + + FaultInjectorImpl injector = new FaultInjectorImpl(); + KeyValueHandler.setInjector(injector); + StorageContainerException sce = new StorageContainerException( + "Requested operation not allowed as ContainerState is CLOSED", + ContainerProtos.Result.CLOSED_CONTAINER_IO); + injector.setException(sce); + GenericTestUtils.LogCapturer logs = + GenericTestUtils.LogCapturer.captureLogs(BasicRootedOzoneClientAdapterImpl.LOG); + + fs.recoverLease(file); + assertTrue(logs.getOutput().contains("Failed to execute finalizeBlock command")); + assertTrue(logs.getOutput().contains("Requested operation not allowed as ContainerState is CLOSED")); + + // The lease should have been recovered. + assertTrue(fs.isFileClosed(file), "File should be closed"); + FileStatus fileStatus = fs.getFileStatus(file); + assertEquals(dataSize * 2, fileStatus.getLen()); + } finally { + closeIgnoringKeyNotFound(stream); + } + + // open it again, make sure the data is correct + verifyData(data, dataSize * 2, file, fs); + } + + @Test + public void testBlockPipelineClosed() throws Exception { + RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); + int dataSize = 100; + final byte[] data = getData(dataSize); + + final FSDataOutputStream stream = fs.create(file, true); + try { + stream.write(data); + stream.hsync(); + assertFalse(fs.isFileClosed(file)); + + // write more data without hsync + stream.write(data); + stream.flush(); + + // close the pipeline + StorageContainerManager scm = cluster.getStorageContainerManager(); + ContainerInfo container = scm.getContainerManager().getContainers().get(0); + OzoneTestUtils.closeContainer(scm, container); + GenericTestUtils.waitFor(() -> { + try { + return scm.getPipelineManager().getPipeline(container.getPipelineID()).isClosed(); + } catch (PipelineNotFoundException e) { + throw new RuntimeException(e); + } + }, 200, 30000); + + fs.recoverLease(file); + + // The lease should have been recovered. + assertTrue(fs.isFileClosed(file), "File should be closed"); + FileStatus fileStatus = fs.getFileStatus(file); + assertEquals(dataSize * 2, fileStatus.getLen()); + } finally { + closeIgnoringKeyNotFound(stream); + } + + // open it again, make sure the data is correct + verifyData(data, dataSize * 2, file, fs); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testGetCommittedBlockLengthTimeout(boolean forceRecovery) throws Exception { + // reduce read timeout + conf.set(OZONE_CLIENT_READ_TIMEOUT, "2s"); + // set force recovery + System.setProperty(FORCE_LEASE_RECOVERY_ENV, String.valueOf(forceRecovery)); + RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); + int dataSize = 100; + final byte[] data = getData(dataSize); + + final FSDataOutputStream stream = fs.create(file, true); + try { + stream.write(data); + stream.hsync(); + assertFalse(fs.isFileClosed(file)); + + // write more data without hsync + stream.write(data); + stream.flush(); + + // close the pipeline and container + ContainerInfo container = cluster.getStorageContainerManager().getContainerManager().getContainers().get(0); + OzoneTestUtils.closeContainer(cluster.getStorageContainerManager(), container); + // pause getCommittedBlockLength handling on all DNs to make sure all getCommittedBlockLength will time out + FaultInjectorImpl injector = new FaultInjectorImpl(); + KeyValueHandler.setInjector(injector); + GenericTestUtils.LogCapturer logs = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientGrpc.getLogger()); + if (!forceRecovery) { + assertThrows(IOException.class, () -> fs.recoverLease(file)); + return; + } else { + fs.recoverLease(file); + } + assertEquals(3, StringUtils.countMatches(logs.getOutput(), + "Executing command cmdType: GetCommittedBlockLength")); + + // The lease should have been recovered. + assertTrue(fs.isFileClosed(file), "File should be closed"); + FileStatus fileStatus = fs.getFileStatus(file); + // Since all DNs are out, then the length in OM keyInfo will be used as the final file length + assertEquals(dataSize, fileStatus.getLen()); + } finally { + if (!forceRecovery) { + closeIgnoringOMException(stream, OMException.ResultCodes.KEY_UNDER_LEASE_RECOVERY); + } else { + closeIgnoringKeyNotFound(stream); + } + KeyValueHandler.setInjector(null); + } + + // open it again, make sure the data is correct + verifyData(data, dataSize, file, fs); + } + + @Test + public void testGetCommittedBlockLengthWithException() throws Exception { + RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); + int dataSize = 100; + final byte[] data = getData(dataSize); + + final FSDataOutputStream stream = fs.create(file, true); + try { + stream.write(data); + stream.hsync(); + assertFalse(fs.isFileClosed(file)); + + // write more data without hsync + stream.write(data); + stream.flush(); + + // close the pipeline and container + ContainerInfo container = cluster.getStorageContainerManager().getContainerManager().getContainers().get(0); + OzoneTestUtils.closeContainer(cluster.getStorageContainerManager(), container); + // throw exception on first DN getCommittedBlockLength handling + FaultInjectorImpl injector = new FaultInjectorImpl(); + KeyValueHandler.setInjector(injector); + StorageContainerException sce = new StorageContainerException( + "ContainerID " + container.getContainerID() + " does not exist", + ContainerProtos.Result.CONTAINER_NOT_FOUND); + injector.setException(sce); + + GenericTestUtils.LogCapturer logs = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientGrpc.getLogger()); + fs.recoverLease(file); + + assertEquals(2, StringUtils.countMatches(logs.getOutput(), + "Executing command cmdType: GetCommittedBlockLength")); + assertEquals(1, StringUtils.countMatches(logs.getOutput(), + "Failed to execute command cmdType: GetCommittedBlockLength")); + + // The lease should have been recovered. + assertTrue(fs.isFileClosed(file), "File should be closed"); + FileStatus fileStatus = fs.getFileStatus(file); + assertEquals(dataSize * 2, fileStatus.getLen()); + } finally { + closeIgnoringKeyNotFound(stream); + KeyValueHandler.setInjector(null); + } + + // open it again, make sure the data is correct + verifyData(data, dataSize * 2, file, fs); + } + + @Test + public void testOMConnectionFailure() throws Exception { + // reduce hadoop RPC retry max attempts + conf.setInt(OzoneConfigKeys.OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY, 5); + conf.setLong(OZONE_CLIENT_WAIT_BETWEEN_RETRIES_MILLIS_KEY, 100); + RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); + int dataSize = 100; + final byte[] data = getData(dataSize); + + final FSDataOutputStream stream = fs.create(file, true); + try { + stream.write(data); + stream.hsync(); + assertFalse(fs.isFileClosed(file)); + + // close OM + cluster.getOzoneManager().stop(); + assertThrows(ConnectException.class, () -> fs.recoverLease(file)); + } finally { + try { + stream.close(); + } catch (Throwable e) { + } + cluster.getOzoneManager().restart(); + cluster.waitForClusterToBeReady(); + assertTrue(fs.recoverLease(file)); + } + } + + @Test + public void testRecoverWrongFile() throws Exception { + final Path notExistFile = new Path(dir, "file1"); + + RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); + int dataSize = 100; + final byte[] data = getData(dataSize); + + final FSDataOutputStream stream = fs.create(file, true); + try { + stream.write(data); + stream.hsync(); + assertFalse(fs.isFileClosed(file)); + + assertThrows(OMException.class, () -> fs.recoverLease(notExistFile)); + } finally { + closeIgnoringKeyNotFound(stream); + } + } + + @Test + public void testRecoveryWithoutBlocks() throws Exception { + RootedOzoneFileSystem fs = (RootedOzoneFileSystem)FileSystem.get(conf); + + final FSDataOutputStream stream = fs.create(file, true); + try { + stream.hsync(); + assertFalse(fs.isFileClosed(file)); + + int count = 0; + while (count++ < 15 && !fs.recoverLease(file)) { + Thread.sleep(1000); + } + // The lease should have been recovered. + assertTrue(fs.isFileClosed(file), "File should be closed"); + + } finally { + closeIgnoringKeyNotFound(stream); + } + } + + private void verifyData(byte[] data, int dataSize, Path filePath, RootedOzoneFileSystem fs) throws IOException { + try (FSDataInputStream fdis = fs.open(filePath)) { + int bufferSize = dataSize > data.length ? dataSize / 2 : dataSize; + while (dataSize > 0) { + byte[] readData = new byte[bufferSize]; + int readBytes = fdis.read(readData); + assertEquals(readBytes, bufferSize); + assertArrayEquals(readData, data); + dataSize -= bufferSize; + } + } + } + + private byte[] getData(int dataSize) { + final byte[] data = new byte[dataSize]; + ThreadLocalRandom.current().nextBytes(data); + return data; } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java index c6893c57e969..78c4bf4961d9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.ozone; import java.io.BufferedInputStream; +import java.io.EOFException; import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -50,14 +51,18 @@ import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import static org.apache.hadoop.hdds.StringUtils.string2Bytes; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Test OzoneFSInputStream by reading through multiple interfaces. @@ -162,6 +167,124 @@ public void testO3FSSingleByteRead() throws IOException { } } + @Test + public void testByteBufferPositionedRead() throws IOException { + try (FSDataInputStream inputStream = fs.open(filePath)) { + int bufferCapacity = 20; + ByteBuffer buffer = ByteBuffer.allocate(bufferCapacity); + long currentPos = inputStream.getPos(); + // Read positional data from 50th index + int position = 50; + int readBytes = inputStream.read(position, buffer); + + // File position should not be changed after positional read + assertEquals(currentPos, inputStream.getPos()); + // Total read bytes should be equal to bufferCapacity + // As file has more data than bufferCapacity + assertEquals(readBytes, bufferCapacity); + byte[] value1 = new byte[readBytes]; + System.arraycopy(buffer.array(), 0, value1, 0, readBytes); + byte[] value2 = new byte[readBytes]; + System.arraycopy(data, position, value2, 0, readBytes); + // Verify input and positional read data + assertArrayEquals(value1, value2, "value mismatch"); + buffer.clear(); + + // Read positional from 8th index again using same inputStream + position = 8; + readBytes = inputStream.read(position, buffer); + assertEquals(currentPos, inputStream.getPos()); + assertEquals(readBytes, bufferCapacity); + byte[] value3 = new byte[readBytes]; + System.arraycopy(buffer.array(), 0, value3, 0, readBytes); + byte[] value4 = new byte[readBytes]; + System.arraycopy(data, position, value4, 0, readBytes); + // Verify input and positional read data + assertArrayEquals(value3, value4, "value mismatch"); + + // Buffer size more than actual data, still read should succeed + ByteBuffer buffer1 = ByteBuffer.allocate(30 * 1024 * 1024 * 2); + // Read positional from 12th index + position = 12; + readBytes = inputStream.read(position, buffer1); + assertEquals(currentPos, inputStream.getPos()); + // Total read bytes should be (total file bytes - position) as buffer is not filled completely + assertEquals(readBytes, 30 * 1024 * 1024 - position); + + byte[] value5 = new byte[readBytes]; + System.arraycopy(buffer1.array(), 0, value5, 0, readBytes); + byte[] value6 = new byte[readBytes]; + System.arraycopy(data, position, value6, 0, readBytes); + // Verify input and positional read data + assertArrayEquals(value5, value6, "value mismatch"); + } + } + + @ParameterizedTest + @ValueSource(ints = { -1, 30 * 1024 * 1024, 30 * 1024 * 1024 + 1 }) + public void testByteBufferPositionedReadWithInvalidPosition(int position) throws IOException { + try (FSDataInputStream inputStream = fs.open(filePath)) { + long currentPos = inputStream.getPos(); + ByteBuffer buffer = ByteBuffer.allocate(20); + assertEquals(-1, inputStream.read(position, buffer)); + // File position should not be changed + assertEquals(currentPos, inputStream.getPos()); + } + } + + @Test + public void testByteBufferPositionedReadFully() throws IOException { + try (FSDataInputStream inputStream = fs.open(filePath)) { + int bufferCapacity = 20; + long currentPos = inputStream.getPos(); + ByteBuffer buffer = ByteBuffer.allocate(bufferCapacity); + // Read positional data from 50th index + int position = 50; + inputStream.readFully(position, buffer); + // File position should not be changed after positional readFully + assertEquals(currentPos, inputStream.getPos()); + // Make sure buffer is full after readFully + Assertions.assertThat((!buffer.hasRemaining())); + + byte[] value1 = new byte[bufferCapacity]; + System.arraycopy(buffer.array(), 0, value1, 0, bufferCapacity); + byte[] value2 = new byte[bufferCapacity]; + System.arraycopy(data, position, value2, 0, bufferCapacity); + // Verify input and positional read data + assertArrayEquals(value1, value2, "value mismatch"); + buffer.clear(); + + // Read positional from 8th index again using same inputStream + position = 8; + inputStream.readFully(position, buffer); + assertEquals(currentPos, inputStream.getPos()); + Assertions.assertThat((!buffer.hasRemaining())); + byte[] value3 = new byte[bufferCapacity]; + System.arraycopy(buffer.array(), 0, value3, 0, bufferCapacity); + byte[] value4 = new byte[bufferCapacity]; + System.arraycopy(data, position, value4, 0, bufferCapacity); + // Verify input and positional read data + assertArrayEquals(value3, value4, "value mismatch"); + + // Buffer size is more than actual data, readFully should fail in this case + ByteBuffer buffer1 = ByteBuffer.allocate(30 * 1024 * 1024 * 2); + assertThrows(EOFException.class, () -> inputStream.readFully(12, buffer1)); + assertEquals(currentPos, inputStream.getPos()); + } + } + + @ParameterizedTest + @ValueSource(ints = { -1, 30 * 1024 * 1024, 30 * 1024 * 1024 + 1 }) + public void testByteBufferPositionedReadFullyWithInvalidPosition(int position) throws IOException { + try (FSDataInputStream inputStream = fs.open(filePath)) { + long currentPos = inputStream.getPos(); + ByteBuffer buffer = ByteBuffer.allocate(20); + assertThrows(EOFException.class, () -> inputStream.readFully(position, buffer)); + // File position should not be changed + assertEquals(currentPos, inputStream.getPos()); + } + } + @Test public void testO3FSMultiByteRead() throws IOException { try (FSDataInputStream inputStream = fs.open(filePath)) { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerSmallFile.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerSmallFile.java index 30c4e4cd5b4d..5dab271d9edb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerSmallFile.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerSmallFile.java @@ -31,6 +31,8 @@ import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.common.SCMTestUtils; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -188,6 +190,23 @@ public void testReadWriteWithBCSId() throws Exception { assertEquals("data123", readData); xceiverClientManager.releaseClient(client, false); } + + @Test + public void testEcho() throws Exception { + ContainerWithPipeline container = + storageContainerLocationClient.allocateContainer( + SCMTestUtils.getReplicationType(ozoneConfig), + HddsProtos.ReplicationFactor.ONE, OzoneConsts.OZONE); + XceiverClientSpi client = xceiverClientManager + .acquireClient(container.getPipeline()); + ContainerProtocolCalls.createContainer(client, + container.getContainerInfo().getContainerID(), null); + ByteString byteString = UnsafeByteOperations.unsafeWrap(new byte[0]); + ContainerProtos.EchoResponseProto response = + ContainerProtocolCalls.echo(client, "", container.getContainerInfo().getContainerID(), byteString, 1, 0); + assertEquals(1, response.getPayload().size()); + xceiverClientManager.releaseClient(client, false); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java index fcde2bed4293..9c76c0ec0c79 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java @@ -22,6 +22,7 @@ import java.util.UUID; import java.util.concurrent.TimeoutException; +import org.apache.hadoop.hdds.DatanodeVersion; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -270,9 +271,12 @@ abstract class Builder { protected String scmId = UUID.randomUUID().toString(); protected String omId = UUID.randomUUID().toString(); - + protected boolean includeRecon = false; + protected int dnInitialVersion = DatanodeVersion.FUTURE_VERSION.toProtoValue(); + protected int dnCurrentVersion = DatanodeVersion.COMBINED_PUTBLOCK_WRITECHUNK_RPC.toProtoValue(); + protected int numOfDatanodes = 3; protected boolean startDataNodes = true; protected CertificateClient certClient; @@ -344,6 +348,30 @@ public Builder setNumDatanodes(int val) { return this; } + /** + * Set the initialVersion for all datanodes. + * + * @param val initialVersion value to be set for all datanodes. + * + * @return MiniOzoneCluster.Builder + */ + public Builder setDatanodeInitialVersion(int val) { + dnInitialVersion = val; + return this; + } + + /** + * Set the currentVersion for all datanodes. + * + * @param val currentVersion value to be set for all datanodes. + * + * @return MiniOzoneCluster.Builder + */ + public Builder setDatanodeCurrentVersion(int val) { + dnCurrentVersion = val; + return this; + } + public Builder setDatanodeFactory(DatanodeFactory factory) { this.dnFactory = factory; return this; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java index 72d6a29fce64..50013b57f4c3 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java @@ -33,6 +33,7 @@ import java.util.concurrent.TimeoutException; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hdds.DatanodeVersion; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.client.RatisReplicationConfig; @@ -91,6 +92,8 @@ import static org.apache.ozone.test.GenericTestUtils.PortAllocator.localhostWithFreePort; import org.hadoop.ozone.recon.codegen.ReconSqlDbConfig; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,6 +126,7 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster { private CertificateClient caClient; private final Set clients = ConcurrentHashMap.newKeySet(); private SecretKeyClient secretKeyClient; + private static MockedStatic mockDNStatic = Mockito.mockStatic(HddsDatanodeService.class); /** * Creates a new MiniOzoneCluster with Recon. @@ -349,6 +353,16 @@ private void waitForHddsDatanodeToStop(DatanodeDetails dn) }, 1000, waitForClusterToBeReadyTimeout); } + private static void overrideDatanodeVersions(int dnInitialVersion, int dnCurrentVersion) { + // FUTURE_VERSION (-1) is not a valid version for a datanode, using it as a marker when version is not overridden + if (dnInitialVersion != DatanodeVersion.FUTURE_VERSION.toProtoValue()) { + mockDNStatic.when(HddsDatanodeService::getDefaultInitialVersion).thenReturn(dnInitialVersion); + } + if (dnCurrentVersion != DatanodeVersion.FUTURE_VERSION.toProtoValue()) { + mockDNStatic.when(HddsDatanodeService::getDefaultCurrentVersion).thenReturn(dnCurrentVersion); + } + } + @Override public void restartHddsDatanode(int i, boolean waitForDatanode) throws InterruptedException, TimeoutException { @@ -734,6 +748,10 @@ protected ReconServer createRecon() { protected List createHddsDatanodes() throws IOException { List hddsDatanodes = new ArrayList<>(); + + // Override default datanode initial and current version if necessary + overrideDatanodeVersions(dnInitialVersion, dnCurrentVersion); + for (int i = 0; i < numOfDatanodes; i++) { OzoneConfiguration dnConf = dnFactory.apply(conf); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java index adf4d5ab9551..884e435d25ec 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java @@ -156,7 +156,7 @@ public static void closeContainer(StorageContainerManager scm, throws IOException, TimeoutException, InterruptedException { Pipeline pipeline = scm.getPipelineManager() .getPipeline(container.getPipelineID()); - scm.getPipelineManager().closePipeline(pipeline, false); + scm.getPipelineManager().closePipeline(pipeline, true); GenericTestUtils.waitFor(() -> container.getState() == HddsProtos.LifeCycleState.CLOSED, 200, 30000); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockDataStreamOutput.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockDataStreamOutput.java index ea1b16b0483a..90a3f1d68933 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockDataStreamOutput.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockDataStreamOutput.java @@ -17,9 +17,11 @@ package org.apache.hadoop.ozone.client.rpc; +import org.apache.hadoop.hdds.DatanodeVersion; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientManager; @@ -28,6 +30,7 @@ import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.ozone.ClientConfigForTesting; +import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.client.ObjectStore; @@ -45,6 +48,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.List; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -70,6 +74,7 @@ public class TestBlockDataStreamOutput { private static String volumeName; private static String bucketName; private static String keyString; + private static final int DN_OLD_VERSION = DatanodeVersion.SEPARATE_RATIS_PORTS_AVAILABLE.toProtoValue(); /** * Create a MiniDFSCluster for testing. @@ -105,6 +110,7 @@ public static void init() throws Exception { cluster = MiniOzoneCluster.newBuilder(conf) .setNumDatanodes(5) + .setDatanodeCurrentVersion(DN_OLD_VERSION) .build(); cluster.waitForClusterToBeReady(); //the easiest way to create an open container is creating a key @@ -269,4 +275,25 @@ public void testTotalAckDataLength() throws Exception { assertEquals(dataLength, stream.getTotalAckDataLength()); } + @Test + public void testDatanodeVersion() throws Exception { + // Verify all DNs internally have versions set correctly + List dns = cluster.getHddsDatanodes(); + for (HddsDatanodeService dn : dns) { + DatanodeDetails details = dn.getDatanodeDetails(); + assertEquals(DN_OLD_VERSION, details.getCurrentVersion()); + } + + String keyName = getKeyName(); + OzoneDataStreamOutput key = createKey(keyName, ReplicationType.RATIS, 0); + KeyDataStreamOutput keyDataStreamOutput = (KeyDataStreamOutput) key.getByteBufStreamOutput(); + BlockDataStreamOutputEntry stream = keyDataStreamOutput.getStreamEntries().get(0); + + // Now check 3 DNs in a random pipeline returns the correct DN versions + List streamDnDetails = stream.getPipeline().getNodes(); + for (DatanodeDetails details : streamDnDetails) { + assertEquals(DN_OLD_VERSION, details.getCurrentVersion()); + } + } + } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java index e15e1e4d63ba..ce5432739cbd 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java @@ -80,8 +80,13 @@ class TestBlockOutputStream { static MiniOzoneCluster createCluster() throws IOException, InterruptedException, TimeoutException { - OzoneConfiguration conf = new OzoneConfiguration(); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setChecksumType(ChecksumType.NONE); + clientConfig.setStreamBufferFlushDelay(false); + clientConfig.setEnablePutblockPiggybacking(true); + conf.setFromObject(clientConfig); + conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 6, TimeUnit.SECONDS); conf.setQuietMode(false); @@ -397,7 +402,7 @@ void testWriteMoreThanChunkSize(boolean flushDelay) throws Exception { key.flush(); assertEquals(writeChunkCount + 2, metrics.getContainerOpCountMetrics(WriteChunk)); - assertEquals(putBlockCount + 1, + assertEquals(putBlockCount, metrics.getContainerOpCountMetrics(PutBlock)); assertEquals(pendingWriteChunkCount, metrics.getPendingContainerOpCountMetrics(WriteChunk)); @@ -426,9 +431,9 @@ void testWriteMoreThanChunkSize(boolean flushDelay) throws Exception { metrics.getPendingContainerOpCountMetrics(PutBlock)); assertEquals(writeChunkCount + 2, metrics.getContainerOpCountMetrics(WriteChunk)); - assertEquals(putBlockCount + 2, + assertEquals(putBlockCount + 1, metrics.getContainerOpCountMetrics(PutBlock)); - assertEquals(totalOpCount + 4, metrics.getTotalOpCount()); + assertEquals(totalOpCount + 3, metrics.getTotalOpCount()); assertEquals(0, keyOutputStream.getStreamEntries().size()); validateData(keyName, data1, client.getObjectStore(), VOLUME, BUCKET); @@ -493,9 +498,9 @@ void testWriteMoreThanFlushSize(boolean flushDelay) throws Exception { metrics.getPendingContainerOpCountMetrics(PutBlock)); assertEquals(writeChunkCount + 3, metrics.getContainerOpCountMetrics(WriteChunk)); - assertEquals(putBlockCount + 2, + assertEquals(putBlockCount + 1, metrics.getContainerOpCountMetrics(PutBlock)); - assertEquals(totalOpCount + 5, metrics.getTotalOpCount()); + assertEquals(totalOpCount + 4, metrics.getTotalOpCount()); assertEquals(dataLength, blockOutputStream.getTotalAckDataLength()); // make sure the bufferPool is empty assertEquals(0, blockOutputStream.getBufferPool().computeBufferData()); @@ -686,9 +691,9 @@ void testWriteMoreThanMaxFlushSize(boolean flushDelay) throws Exception { assertEquals(writeChunkCount + 5, metrics.getContainerOpCountMetrics(WriteChunk)); // The previous flush did not trigger any action with flushDelay enabled - assertEquals(putBlockCount + (flushDelay ? 3 : 4), + assertEquals(putBlockCount + (flushDelay ? 2 : 3), metrics.getContainerOpCountMetrics(PutBlock)); - assertEquals(totalOpCount + (flushDelay ? 8 : 9), + assertEquals(totalOpCount + (flushDelay ? 7 : 8), metrics.getTotalOpCount()); assertEquals(dataLength, blockOutputStream.getTotalAckDataLength()); assertEquals(0, blockOutputStream.getCommitIndex2flushedDataMap().size()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDatanodeVersion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDatanodeVersion.java new file mode 100644 index 000000000000..5e7d8a4b0525 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDatanodeVersion.java @@ -0,0 +1,143 @@ +/** + * 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.ozone.client.rpc; + +import org.apache.hadoop.hdds.DatanodeVersion; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.ozone.ClientConfigForTesting; +import org.apache.hadoop.ozone.HddsDatanodeService; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.client.ObjectStore; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.OzoneClientFactory; +import org.apache.hadoop.ozone.client.io.BlockDataStreamOutputEntry; +import org.apache.hadoop.ozone.client.io.KeyDataStreamOutput; +import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; +import org.apache.hadoop.ozone.container.TestHelper; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.List; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Tests DatanodeVersion in client stream. + */ +@Timeout(120) +public class TestDatanodeVersion { + private static MiniOzoneCluster cluster; + private static OzoneConfiguration conf = new OzoneConfiguration(); + private static OzoneClient client; + private static ObjectStore objectStore; + private static int chunkSize; + private static int flushSize; + private static int maxFlushSize; + private static int blockSize; + private static String volumeName; + private static String bucketName; + private static final int DN_OLD_VERSION = DatanodeVersion.SEPARATE_RATIS_PORTS_AVAILABLE.toProtoValue(); + + /** + * Create a MiniDFSCluster for testing. + *

+ * Ozone is made active by setting OZONE_ENABLED = true + */ + @BeforeAll + public static void init() throws Exception { + chunkSize = 100; + flushSize = 2 * chunkSize; + maxFlushSize = 2 * flushSize; + blockSize = 2 * maxFlushSize; + + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + conf.setFromObject(clientConfig); + + conf.setQuietMode(false); + conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4, StorageUnit.MB); + + ClientConfigForTesting.newBuilder(StorageUnit.BYTES) + .setBlockSize(blockSize) + .setChunkSize(chunkSize) + .setStreamBufferFlushSize(flushSize) + .setStreamBufferMaxSize(maxFlushSize) + .setDataStreamBufferFlushSize(maxFlushSize) + .setDataStreamMinPacketSize(chunkSize) + .setDataStreamWindowSize(5 * chunkSize) + .applyTo(conf); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(3) + .setDatanodeCurrentVersion(DN_OLD_VERSION) + .build(); + cluster.waitForClusterToBeReady(); + //the easiest way to create an open container is creating a key + client = OzoneClientFactory.getRpcClient(conf); + objectStore = client.getObjectStore(); + volumeName = "testblockoutputstream"; + bucketName = volumeName; + objectStore.createVolume(volumeName); + objectStore.getVolume(volumeName).createBucket(bucketName); + } + + /** + * Shutdown MiniDFSCluster. + */ + @AfterAll + public static void shutdown() { + IOUtils.closeQuietly(client); + if (cluster != null) { + cluster.shutdown(); + } + } + + static OzoneDataStreamOutput createKey(String keyName, ReplicationType type, long size) throws Exception { + return TestHelper.createStreamKey(keyName, type, size, objectStore, volumeName, bucketName); + } + + @Test + public void testStreamDatanodeVersion() throws Exception { + // Verify all DNs internally have versions set correctly + List dns = cluster.getHddsDatanodes(); + for (HddsDatanodeService dn : dns) { + DatanodeDetails details = dn.getDatanodeDetails(); + assertEquals(DN_OLD_VERSION, details.getCurrentVersion()); + } + + String keyName = UUID.randomUUID().toString(); + OzoneDataStreamOutput key = createKey(keyName, ReplicationType.RATIS, 0); + KeyDataStreamOutput keyDataStreamOutput = (KeyDataStreamOutput) key.getByteBufStreamOutput(); + BlockDataStreamOutputEntry stream = keyDataStreamOutput.getStreamEntries().get(0); + + // Now check 3 DNs in a random pipeline returns the correct DN versions + List streamDnDetails = stream.getPipeline().getNodes(); + for (DatanodeDetails details : streamDnDetails) { + assertEquals(DN_OLD_VERSION, details.getCurrentVersion()); + } + } + +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSecureOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSecureOzoneRpcClient.java index 0f69245a6d30..15af5a2d8e06 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSecureOzoneRpcClient.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSecureOzoneRpcClient.java @@ -18,6 +18,11 @@ package org.apache.hadoop.ozone.client.rpc; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.ozone.RootedOzoneFileSystem; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationFactor; @@ -47,10 +52,13 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OMMetrics; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.S3SecretManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.S3SecretValue; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; @@ -67,20 +75,32 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.File; import java.io.IOException; import java.time.Instant; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS; +import static org.apache.hadoop.ozone.OzoneConsts.FORCE_LEASE_RECOVERY_ENV; +import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME; +import static org.apache.hadoop.ozone.OzoneConsts.OZONE_ROOT; +import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.helpers.BucketLayout.FILE_SYSTEM_OPTIMIZED; import static org.apache.ozone.test.GenericTestUtils.getTestStartTime; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; /** @@ -125,6 +145,7 @@ public static void init() throws Exception { // constructed. conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT, OMConfigKeys.OZONE_BUCKET_LAYOUT_OBJECT_STORE); + conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); cluster = MiniOzoneCluster.newBuilder(conf) .setNumDatanodes(14) .setCertificateClient(certificateClientTest) @@ -155,7 +176,7 @@ public static void init() throws Exception { public void testPutKeySuccessWithBlockToken() throws Exception { testPutKeySuccessWithBlockTokenWithBucketLayout(BucketLayout.OBJECT_STORE); testPutKeySuccessWithBlockTokenWithBucketLayout( - BucketLayout.FILE_SYSTEM_OPTIMIZED); + FILE_SYSTEM_OPTIMIZED); } private void testPutKeySuccessWithBlockTokenWithBucketLayout( @@ -226,6 +247,148 @@ private void testPutKeySuccessWithBlockTokenWithBucketLayout( } } + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFileRecovery(boolean forceRecovery) throws Exception { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + + String value = "sample value"; + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName, + new BucketArgs.Builder().setBucketLayout(FILE_SYSTEM_OPTIMIZED).build()); + OzoneBucket bucket = volume.getBucket(bucketName); + + String keyName = UUID.randomUUID().toString(); + final String dir = OZONE_ROOT + bucket.getVolumeName() + OZONE_URI_DELIMITER + bucket.getName(); + final Path file = new Path(dir, keyName); + + // Set the fs.defaultFS + final String rootPath = String.format("%s://%s/", + OZONE_OFS_URI_SCHEME, conf.get(OZONE_OM_ADDRESS_KEY)); + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + // force recovery file + System.setProperty(FORCE_LEASE_RECOVERY_ENV, String.valueOf(forceRecovery)); + conf.setBoolean(String.format("fs.%s.impl.disable.cache", OZONE_OFS_URI_SCHEME), true); + RootedOzoneFileSystem fs = (RootedOzoneFileSystem) FileSystem.get(conf); + OzoneOutputStream out = null; + try { + out = bucket.createKey(keyName, value.getBytes(UTF_8).length, ReplicationType.RATIS, + ReplicationFactor.THREE, new HashMap<>()); + out.write(value.getBytes(UTF_8)); + out.hsync(); + + if (forceRecovery) { + fs.recoverLease(file); + } else { + assertThrows(OMException.class, () -> fs.recoverLease(file)); + } + } finally { + if (out != null) { + if (forceRecovery) { + // close failure because the key is already committed + assertThrows(OMException.class, out::close); + } else { + out.close(); + } + } + } + } + + @ParameterizedTest + @ValueSource(ints = {1 << 24, (1 << 24) + 1, (1 << 24) - 1}) + public void testPreallocateFileRecovery(long dataSize) throws Exception { + cleanupDeletedTable(); + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + + final byte[] data = new byte[(int) dataSize]; + ThreadLocalRandom.current().nextBytes(data); + + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + long nsQuota = 100; + long spaceQuota = 1 * 1024 * 1024 * 1024; + volume.createBucket(bucketName, new BucketArgs.Builder().setBucketLayout(FILE_SYSTEM_OPTIMIZED) + .setQuotaInNamespace(nsQuota).setQuotaInBytes(spaceQuota).build()); + OzoneBucket bucket = volume.getBucket(bucketName); + + String keyName = UUID.randomUUID().toString(); + final String dir = OZONE_ROOT + bucket.getVolumeName() + OZONE_URI_DELIMITER + bucket.getName(); + final Path file = new Path(dir, keyName); + + // Set the fs.defaultFS + final String rootPath = String.format("%s://%s/", + OZONE_OFS_URI_SCHEME, conf.get(OZONE_OM_ADDRESS_KEY)); + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); + RootedOzoneFileSystem fs = (RootedOzoneFileSystem) FileSystem.get(conf); + OzoneOutputStream out = null; + long totalBlock = 10; + long usedBlock = (dataSize - 1) / fs.getDefaultBlockSize() + 1; + long fileSize = fs.getDefaultBlockSize() * totalBlock; + OMMetrics metrics = ozoneManager.getMetrics(); + long committedBytes = metrics.getDataCommittedBytes(); + try { + out = bucket.createKey(keyName, fileSize, ReplicationType.RATIS, + ReplicationFactor.THREE, new HashMap<>()); + // init used quota check + bucket = volume.getBucket(bucketName); + assertEquals(0, bucket.getUsedNamespace()); + assertEquals(0, bucket.getUsedBytes()); + + out.write(data); + out.hsync(); + fs.recoverLease(file); + + // check file length + FileStatus fileStatus = fs.getFileStatus(file); + assertEquals(dataSize, fileStatus.getLen()); + // check committed bytes + assertEquals(committedBytes + dataSize, + ozoneManager.getMetrics().getDataCommittedBytes()); + // check used quota + bucket = volume.getBucket(bucketName); + assertEquals(1, bucket.getUsedNamespace()); + assertEquals(dataSize * ReplicationFactor.THREE.getValue(), bucket.getUsedBytes()); + + // check unused pre-allocated blocks are reclaimed + Table deletedTable = ozoneManager.getMetadataManager().getDeletedTable(); + try (TableIterator> + keyIter = deletedTable.iterator()) { + while (keyIter.hasNext()) { + Table.KeyValue kv = keyIter.next(); + OmKeyInfo key = kv.getValue().getOmKeyInfoList().get(0); + assertEquals(totalBlock - usedBlock, key.getKeyLocationVersions().get(0).getLocationListCount()); + } + } + } finally { + if (out != null) { + // close failure because the key is already committed + assertThrows(OMException.class, out::close); + } + } + } + + private void cleanupDeletedTable() throws IOException { + Table deletedTable = ozoneManager.getMetadataManager().getDeletedTable(); + List nameList = new ArrayList<>(); + try (TableIterator> + keyIter = deletedTable.iterator()) { + while (keyIter.hasNext()) { + Table.KeyValue kv = keyIter.next(); + nameList.add(kv.getKey()); + } + } + nameList.forEach(k -> { + try { + deletedTable.delete(k); + } catch (IOException e) { + // do nothing + } + }); + } + private void assertTokenIsNull(OmKeyInfo value) { value.getKeyLocationVersions() .forEach( diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestFinalizeBlock.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestFinalizeBlock.java new file mode 100644 index 000000000000..e60b1581b324 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestFinalizeBlock.java @@ -0,0 +1,321 @@ +/** + * 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.ozone.container.common.statemachine.commandhandler; + +import jakarta.annotation.Nonnull; +import org.apache.hadoop.hdds.HddsConfigKeys; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.ScmConfig; +import org.apache.hadoop.hdds.scm.ScmConfigKeys; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.ozone.HddsDatanodeService; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneTestUtils; +import org.apache.hadoop.ozone.client.ObjectStore; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.OzoneClientFactory; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; +import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; +import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_BLOCK; +import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_CHUNK; +import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.CONTAINER_SCHEMA_V3_ENABLED; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +/** + * Tests FinalizeBlock. + */ +@Timeout(300) +public class TestFinalizeBlock { + + private OzoneClient client; + private MiniOzoneCluster cluster; + private OzoneConfiguration conf; + private ObjectStore objectStore; + private static String volumeName = UUID.randomUUID().toString(); + private static String bucketName = UUID.randomUUID().toString(); + + public static Stream dnLayoutParams() { + return Stream.of( + arguments(false, FILE_PER_CHUNK), + arguments(true, FILE_PER_CHUNK), + arguments(false, FILE_PER_BLOCK), + arguments(true, FILE_PER_BLOCK) + ); + } + + private void setup(boolean enableSchemaV3, ContainerLayoutVersion version) throws Exception { + conf = new OzoneConfiguration(); + conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB"); + conf.setStorageSize(OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN, + 0, StorageUnit.MB); + conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); + conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 100, TimeUnit.MILLISECONDS); + conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, 1, SECONDS); + conf.setTimeDuration(HDDS_PIPELINE_REPORT_INTERVAL, 1, SECONDS); + conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 1, SECONDS); + conf.setTimeDuration(HDDS_NODE_REPORT_INTERVAL, 1, SECONDS); + conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); + conf.setBoolean(CONTAINER_SCHEMA_V3_ENABLED, enableSchemaV3); + conf.setEnum(ScmConfigKeys.OZONE_SCM_CONTAINER_LAYOUT_KEY, version); + + DatanodeConfiguration datanodeConfiguration = conf.getObject( + DatanodeConfiguration.class); + datanodeConfiguration.setBlockDeletionInterval(Duration.ofMillis(100)); + conf.setFromObject(datanodeConfiguration); + ScmConfig scmConfig = conf.getObject(ScmConfig.class); + scmConfig.setBlockDeletionInterval(Duration.ofMillis(100)); + conf.setFromObject(scmConfig); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(1).build(); + cluster.waitForClusterToBeReady(); + cluster.waitForPipelineTobeReady(ONE, 30000); + + client = OzoneClientFactory.getRpcClient(conf); + objectStore = client.getObjectStore(); + objectStore.createVolume(volumeName); + objectStore.getVolume(volumeName).createBucket(bucketName); + } + + @AfterEach + public void shutdown() { + IOUtils.closeQuietly(client); + if (cluster != null) { + try { + cluster.shutdown(); + } catch (Exception e) { + // do nothing. + } + } + } + + @ParameterizedTest + @MethodSource("dnLayoutParams") + public void testFinalizeBlock(boolean enableSchemaV3, ContainerLayoutVersion version) + throws Exception { + setup(enableSchemaV3, version); + String keyName = UUID.randomUUID().toString(); + // create key + createKey(keyName); + + ContainerID containerId = cluster.getStorageContainerManager() + .getContainerManager().getContainers().get(0).containerID(); + + OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName) + .setBucketName(bucketName).setKeyName(keyName).setDataSize(0) + .build(); + List omKeyLocationInfoGroupList = + cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions(); + + ContainerInfo container = cluster.getStorageContainerManager() + .getContainerManager().getContainer(containerId); + Pipeline pipeline = cluster.getStorageContainerManager() + .getPipelineManager().getPipeline(container.getPipelineID()); + + XceiverClientManager xceiverClientManager = new XceiverClientManager(conf); + XceiverClientSpi xceiverClient = + xceiverClientManager.acquireClient(pipeline); + + // Before finalize block WRITE chunk on the same block should pass through + ContainerProtos.ContainerCommandRequestProto request = + ContainerTestHelper.getWriteChunkRequest(pipeline, ( + new BlockID(containerId.getId(), omKeyLocationInfoGroupList.get(0) + .getLocationList().get(0).getLocalID())), 100); + xceiverClient.sendCommand(request); + + // Before finalize block PUT block on the same block should pass through + request = ContainerTestHelper.getPutBlockRequest(request); + xceiverClient.sendCommand(request); + + // Now Finalize Block + request = getFinalizeBlockRequest(omKeyLocationInfoGroupList, container); + ContainerProtos.ContainerCommandResponseProto response = + xceiverClient.sendCommand(request); + + assertTrue(response.getFinalizeBlock() + .getBlockData().getBlockID().getLocalID() + == omKeyLocationInfoGroupList.get(0) + .getLocationList().get(0).getLocalID()); + + assertTrue(((KeyValueContainerData)getContainerfromDN( + cluster.getHddsDatanodes().get(0), + containerId.getId()).getContainerData()) + .getFinalizedBlockSet().size() == 1); + + testRejectPutAndWriteChunkAfterFinalizeBlock(containerId, pipeline, xceiverClient, omKeyLocationInfoGroupList); + testFinalizeBlockReloadAfterDNRestart(containerId); + testFinalizeBlockClearAfterCloseContainer(containerId); + } + + private void testFinalizeBlockReloadAfterDNRestart(ContainerID containerId) { + try { + cluster.restartHddsDatanode(0, true); + } catch (Exception e) { + fail("Fail to restart Datanode"); + } + + // After restart DN, finalizeBlock should be loaded into memory + assertTrue(((KeyValueContainerData) + getContainerfromDN(cluster.getHddsDatanodes().get(0), + containerId.getId()).getContainerData()) + .getFinalizedBlockSet().size() == 1); + } + + private void testFinalizeBlockClearAfterCloseContainer(ContainerID containerId) + throws InterruptedException, TimeoutException { + OzoneTestUtils.closeAllContainers(cluster.getStorageContainerManager().getEventQueue(), + cluster.getStorageContainerManager()); + + // Finalize Block should be cleared from container data. + GenericTestUtils.waitFor(() -> ( + (KeyValueContainerData) getContainerfromDN(cluster.getHddsDatanodes().get(0), + containerId.getId()).getContainerData()).getFinalizedBlockSet().size() == 0, + 100, 10 * 1000); + try { + // Restart DataNode + cluster.restartHddsDatanode(0, true); + } catch (Exception e) { + fail("Fail to restart Datanode"); + } + + // After DN restart also there should not be any finalizeBlock + assertTrue(((KeyValueContainerData)getContainerfromDN( + cluster.getHddsDatanodes().get(0), + containerId.getId()).getContainerData()) + .getFinalizedBlockSet().size() == 0); + } + + private void testRejectPutAndWriteChunkAfterFinalizeBlock(ContainerID containerId, Pipeline pipeline, + XceiverClientSpi xceiverClient, List omKeyLocationInfoGroupList) + throws IOException { + // Try doing WRITE chunk on the already finalized block + ContainerProtos.ContainerCommandRequestProto request = + ContainerTestHelper.getWriteChunkRequest(pipeline, + (new BlockID(containerId.getId(), omKeyLocationInfoGroupList.get(0) + .getLocationList().get(0).getLocalID())), 100); + + try { + xceiverClient.sendCommand(request); + fail("Write chunk should fail."); + } catch (IOException e) { + assertTrue(e.getCause().getMessage() + .contains("Block already finalized")); + } + + // Try doing PUT block on the already finalized block + request = ContainerTestHelper.getPutBlockRequest(request); + try { + xceiverClient.sendCommand(request); + fail("Put block should fail."); + } catch (IOException e) { + assertTrue(e.getCause().getMessage() + .contains("Block already finalized")); + } + } + + @Nonnull + private ContainerProtos.ContainerCommandRequestProto getFinalizeBlockRequest( + List omKeyLocationInfoGroupList, + ContainerInfo container) { + final ContainerProtos.ContainerCommandRequestProto.Builder builder = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.FinalizeBlock) + .setContainerID(container.getContainerID()) + .setDatanodeUuid(cluster.getHddsDatanodes() + .get(0).getDatanodeDetails().getUuidString()); + + final ContainerProtos.DatanodeBlockID blockId = + ContainerProtos.DatanodeBlockID.newBuilder() + .setContainerID(container.getContainerID()).setLocalID( + omKeyLocationInfoGroupList.get(0) + .getLocationList().get(0).getLocalID()) + .setBlockCommitSequenceId(0).build(); + + builder.setFinalizeBlock(ContainerProtos.FinalizeBlockRequestProto + .newBuilder().setBlockID(blockId).build()); + return builder.build(); + } + + /** + * create a key with specified name. + * @param keyName + * @throws IOException + */ + private void createKey(String keyName) throws IOException { + OzoneOutputStream key = objectStore.getVolume(volumeName) + .getBucket(bucketName) + .createKey(keyName, 1024, ReplicationType.RATIS, + ReplicationFactor.ONE, new HashMap<>()); + key.write("test".getBytes(UTF_8)); + key.close(); + } + + /** + * Return the container for the given containerID from the given DN. + */ + private Container getContainerfromDN(HddsDatanodeService hddsDatanodeService, + long containerID) { + return hddsDatanodeService.getDatanodeStateMachine().getContainer() + .getContainerSet().getContainer(containerID); + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java index dbb354eccfd2..c24cf748ddb1 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java @@ -22,6 +22,8 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.fs.LeaseRecoverable; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -40,7 +42,6 @@ import org.apache.hadoop.ozone.TestDataUtil; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; -import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.BucketLayout; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME; @@ -48,6 +49,7 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Test cases for LeaseRecoverer. @@ -69,6 +71,10 @@ public class TestLeaseRecoverer { public static void init() throws Exception { conf = new OzoneConfiguration(); conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); + conf.set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "0s"); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setStreamBufferFlushDelay(false); + conf.setFromObject(clientConfig); // Set the number of keys to be processed during batch operate. cluster = MiniOzoneCluster.newBuilder(conf).build(); cluster.waitForClusterToBeReady(); @@ -122,20 +128,27 @@ public void testCLI() throws IOException { // make sure file is visible and closed FileStatus fileStatus = fs.getFileStatus(file); assertEquals(dataSize, fileStatus.getLen()); - // make sure the writer can not write again. - // TODO: write does not fail here. Looks like a bug. HDDS-8439 to fix it. + // write data os.write(data); + // flush should fail since flush will call writeChunk and putBlock + assertThrows(IOException.class, os::flush); + fileStatus = fs.getFileStatus(file); assertEquals(dataSize, fileStatus.getLen()); // make sure hsync fails - assertThrows(OMException.class, os::hsync); + assertThrows(IOException.class, os::hsync); // make sure length remains the same fileStatus = fs.getFileStatus(file); assertEquals(dataSize, fileStatus.getLen()); - // make sure close fails - assertThrows(OMException.class, os::close); + // close succeeds since it's already closed in failure handling of flush + assertTrue(((LeaseRecoverable)fs).isFileClosed(file)); + os.close(); // make sure length remains the same fileStatus = fs.getFileStatus(file); assertEquals(dataSize, fileStatus.getLen()); + + // recover the same file second time should succeed + cmd.execute(args); + assertEquals("", stderr.toString()); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java new file mode 100644 index 000000000000..d049a7e320cf --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java @@ -0,0 +1,111 @@ +/* + * 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.ozone.freon; + +import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; +import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.container.common.SCMTestUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import picocli.CommandLine; + +import java.time.Duration; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Tests Freon, with MiniOzoneCluster and validate data. + */ +public class TestDNRPCLoadGenerator { + + private static MiniOzoneCluster cluster = null; + private static ContainerWithPipeline container; + + private static void startCluster(OzoneConfiguration conf) throws Exception { + DatanodeRatisServerConfig ratisServerConfig = + conf.getObject(DatanodeRatisServerConfig.class); + ratisServerConfig.setRequestTimeOut(Duration.ofSeconds(3)); + ratisServerConfig.setWatchTimeOut(Duration.ofSeconds(10)); + conf.setFromObject(ratisServerConfig); + + RatisClientConfig.RaftConfig raftClientConfig = + conf.getObject(RatisClientConfig.RaftConfig.class); + raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3)); + raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(10)); + conf.setFromObject(raftClientConfig); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(5).build(); + cluster.waitForClusterToBeReady(); + cluster.waitForPipelineTobeReady(HddsProtos.ReplicationFactor.THREE, + 180000); + + StorageContainerLocationProtocolClientSideTranslatorPB + storageContainerLocationClient = cluster + .getStorageContainerLocationClient(); + container = + storageContainerLocationClient.allocateContainer( + SCMTestUtils.getReplicationType(conf), + HddsProtos.ReplicationFactor.ONE, OzoneConsts.OZONE); + XceiverClientManager xceiverClientManager = new XceiverClientManager(conf); + XceiverClientSpi client = xceiverClientManager + .acquireClient(container.getPipeline()); + ContainerProtocolCalls.createContainer(client, + container.getContainerInfo().getContainerID(), null); + } + + static void shutdownCluster() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @BeforeAll + public static void init() throws Exception { + OzoneConfiguration conf = new OzoneConfiguration(); + startCluster(conf); + } + + @AfterAll + public static void shutdown() { + shutdownCluster(); + } + + @Test + public void test() { + DNRPCLoadGenerator randomKeyGenerator = + new DNRPCLoadGenerator(cluster.getConf()); + CommandLine cmd = new CommandLine(randomKeyGenerator); + int exitCode = cmd.execute( + "--container-id", Long.toString(container.getContainerInfo().getContainerID()), + "--clients", "5", + "-t", "10"); + assertEquals(0, exitCode); + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMRatisSnapshots.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMRatisSnapshots.java index 42708b0b1607..bd5046bfc0bf 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMRatisSnapshots.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMRatisSnapshots.java @@ -51,6 +51,7 @@ import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServerConfig; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; import org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils; +import org.apache.hadoop.utils.FaultInjectorImpl; import org.apache.ozone.test.GenericTestUtils; import org.apache.ratis.server.protocol.TermIndex; import org.assertj.core.api.Fail; @@ -78,7 +79,6 @@ import java.util.List; import java.util.Objects; import java.util.Set; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -404,7 +404,7 @@ public void testInstallIncrementalSnapshot(@TempDir Path tempDir) OzoneManager followerOM = cluster.getOzoneManager(followerNodeId); // Set fault injector to pause before install - FaultInjector faultInjector = new SnapshotPauseInjector(); + FaultInjector faultInjector = new FaultInjectorImpl(); followerOM.getOmSnapshotProvider().setInjector(faultInjector); // Do some transactions so that the log index increases @@ -613,7 +613,7 @@ public void testInstallIncrementalSnapshotWithFailure() throws Exception { OzoneManager followerOM = cluster.getOzoneManager(followerNodeId); // Set fault injector to pause before install - FaultInjector faultInjector = new SnapshotPauseInjector(); + FaultInjector faultInjector = new FaultInjectorImpl(); followerOM.getOmSnapshotProvider().setInjector(faultInjector); // Do some transactions so that the log index increases @@ -1129,48 +1129,6 @@ public void exitSystem(int status, String message, Throwable throwable, } } - private static class SnapshotPauseInjector extends FaultInjector { - private CountDownLatch ready; - private CountDownLatch wait; - - SnapshotPauseInjector() { - init(); - } - - @Override - public void init() { - this.ready = new CountDownLatch(1); - this.wait = new CountDownLatch(1); - } - - @Override - public void pause() throws IOException { - ready.countDown(); - try { - wait.await(); - } catch (InterruptedException e) { - throw new IOException(e); - } - } - - @Override - public void resume() throws IOException { - // Make sure injector pauses before resuming. - try { - ready.await(); - } catch (InterruptedException e) { - e.printStackTrace(); - assertTrue(Fail.fail("resume interrupted")); - } - wait.countDown(); - } - - @Override - public void reset() throws IOException { - init(); - } - } - // Interrupts the tarball download process to test creation of // multiple tarballs as needed when the tarball size exceeds the // max. diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java index 2ae69dc3c96f..e773bf7ed7f2 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java @@ -607,18 +607,40 @@ private ContainerProtos.DatanodeBlockID createBlockId(long containerId, private void mockWriteChunkResponse(XceiverClientSpi mockDnProtocol) throws IOException, ExecutionException, InterruptedException { - ContainerCommandResponseProto writeResponse = - ContainerCommandResponseProto.newBuilder() - .setWriteChunk(WriteChunkResponseProto.newBuilder().build()) - .setResult(Result.SUCCESS) - .setCmdType(Type.WriteChunk) - .build(); doAnswer(invocation -> - new XceiverClientReply(completedFuture(writeResponse))) + new XceiverClientReply( + completedFuture( + createWriteChunkResponse( + (ContainerCommandRequestProto)invocation.getArgument(0))))) .when(mockDnProtocol) .sendCommandAsync(argThat(matchCmd(Type.WriteChunk))); } + ContainerCommandResponseProto createWriteChunkResponse( + ContainerCommandRequestProto request) { + ContainerProtos.WriteChunkRequestProto writeChunk = request.getWriteChunk(); + + WriteChunkResponseProto.Builder builder = + WriteChunkResponseProto.newBuilder(); + if (writeChunk.hasBlock()) { + ContainerProtos.BlockData + blockData = writeChunk.getBlock().getBlockData(); + + GetCommittedBlockLengthResponseProto response = + GetCommittedBlockLengthResponseProto.newBuilder() + .setBlockID(blockData.getBlockID()) + .setBlockLength(blockData.getSize()) + .build(); + + builder.setCommittedBlockLength(response); + } + return ContainerCommandResponseProto.newBuilder() + .setWriteChunk(builder.build()) + .setResult(Result.SUCCESS) + .setCmdType(Type.WriteChunk) + .build(); + } + private ArgumentMatcher matchCmd(Type type) { return argument -> argument != null && argument.getCmdType() == type; } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java index 47f5314bd6c5..29ee6c6e04a4 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java @@ -359,7 +359,7 @@ public void testKeyOps() throws Exception { omMetrics = getMetrics("OMMetrics"); - assertEquals(initialNumKeyOps + 7, getLongCounter("NumKeyOps", omMetrics)); + assertEquals(initialNumKeyOps + 8, getLongCounter("NumKeyOps", omMetrics)); assertEquals(initialNumKeyAllocate + 1, getLongCounter("NumKeyAllocate", omMetrics)); assertEquals(initialNumKeyLookup + 1, getLongCounter("NumKeyLookup", omMetrics)); assertEquals(initialNumKeyDeletes + 1, getLongCounter("NumKeyDeletes", omMetrics)); @@ -425,7 +425,7 @@ public void testKeyOps() throws Exception { doKeyOps(keyArgs); omMetrics = getMetrics("OMMetrics"); - assertEquals(initialNumKeyOps + 28, getLongCounter("NumKeyOps", omMetrics)); + assertEquals(initialNumKeyOps + 31, getLongCounter("NumKeyOps", omMetrics)); assertEquals(initialNumKeyAllocate + 6, getLongCounter("NumKeyAllocate", omMetrics)); assertEquals(initialNumKeyLookup + 3, getLongCounter("NumKeyLookup", omMetrics)); assertEquals(initialNumKeyDeletes + 4, getLongCounter("NumKeyDeletes", omMetrics)); @@ -857,6 +857,11 @@ private void doKeyOps(OmKeyArgs keyArgs) { writeClient.initiateMultipartUpload(keyArgs); } catch (IOException ignored) { } + + try { + writeClient.listOpenFiles("", 100, ""); + } catch (IOException ignored) { + } } private OmKeyArgs createKeyArgs(String volumeName, String bucketName, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java index f86fd46946c4..085858f71179 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.UUID; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.crypto.key.KeyProvider; import org.apache.hadoop.crypto.key.kms.KMSClientProvider; import org.apache.hadoop.crypto.key.kms.server.MiniKMS; @@ -77,9 +78,10 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY; import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_LISTING_PAGE_SIZE; +import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME; - import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_EMPTY; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND; @@ -91,6 +93,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -101,7 +104,10 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.MethodOrderer.OrderAnnotation; +import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -120,6 +126,7 @@ * Inspired by TestS3Shell */ @Timeout(300) +@TestMethodOrder(OrderAnnotation.class) public class TestOzoneShellHA { private static final Logger LOG = @@ -154,6 +161,7 @@ public class TestOzoneShellHA { @BeforeAll public static void init() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); + conf.setBoolean(OZONE_FS_HSYNC_ENABLED, true); startKMS(); startCluster(conf); } @@ -572,6 +580,134 @@ public void testOzoneAdminCmdList() throws UnsupportedEncodingException { execute(ozoneAdminShell, args); } + @Test + public void testAdminCmdListOpenFiles() + throws IOException, InterruptedException { + + OzoneConfiguration conf = cluster.getConf(); + final String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + omServiceId; + + OzoneConfiguration clientConf = getClientConfForOFS(hostPrefix, conf); + clientConf.setBoolean(OZONE_FS_HSYNC_ENABLED, true); + FileSystem fs = FileSystem.get(clientConf); + + assertNotEquals(fs.getConf().get(OZONE_FS_HSYNC_ENABLED), + "false", OZONE_FS_HSYNC_ENABLED + " is set to false " + + "by external force. Must be true to allow hsync to function"); + + final String volumeName = "volume-lof"; + final String bucketName = "buck1"; + + String dir1 = hostPrefix + + OM_KEY_PREFIX + volumeName + + OM_KEY_PREFIX + bucketName + + OM_KEY_PREFIX + "dir1"; + // Create volume, bucket, dir + assertTrue(fs.mkdirs(new Path(dir1))); + String keyPrefix = OM_KEY_PREFIX + "key"; + + final int numKeys = 5; + String[] keys = new String[numKeys]; + + for (int i = 0; i < numKeys; i++) { + keys[i] = dir1 + keyPrefix + i; + } + + int pageSize = 3; + String pathToBucket = "/" + volumeName + "/" + bucketName; + FSDataOutputStream[] streams = new FSDataOutputStream[numKeys]; + + try { + // Create multiple keys and hold them open + for (int i = 0; i < numKeys; i++) { + streams[i] = fs.create(new Path(keys[i])); + streams[i].write(1); + } + + // Wait for DB flush + cluster.getOzoneManager().awaitDoubleBufferFlush(); + + String[] args = new String[] {"om", "lof", + "--service-id", omServiceId, + "-l", String.valueOf(numKeys + 1), // pagination + "-p", pathToBucket}; + // Run listopenfiles + execute(ozoneAdminShell, args); + String cmdRes = getStdOut(); + // Should have retrieved all 5 open keys + for (int i = 0; i < numKeys; i++) { + assertTrue(cmdRes.contains(keyPrefix + i)); + } + + // Try pagination + args = new String[] {"om", "lof", + "--service-id", omServiceId, + "-l", String.valueOf(pageSize), // pagination + "-p", pathToBucket}; + execute(ozoneAdminShell, args); + cmdRes = getStdOut(); + + // Should have retrieved the 1st page only (3 keys) + for (int i = 0; i < pageSize; i++) { + assertTrue(cmdRes.contains(keyPrefix + i)); + } + for (int i = pageSize; i < numKeys; i++) { + assertFalse(cmdRes.contains(keyPrefix + i)); + } + // No hsync'ed file/key at this point + assertFalse(cmdRes.contains("\tYes\t")); + + // Get last line of the output which has the continuation token + String[] lines = cmdRes.split("\n"); + String nextCmd = lines[lines.length - 1].trim(); + String kw = "--start="; + String contToken = + nextCmd.substring(nextCmd.lastIndexOf(kw) + kw.length()); + + args = new String[] {"om", "lof", + "--service-id", omServiceId, + "-l", String.valueOf(pageSize), // pagination + "-p", pathToBucket, + "-s", contToken}; + execute(ozoneAdminShell, args); + cmdRes = getStdOut(); + + // Should have retrieved the 2nd page only (2 keys) + for (int i = 0; i < pageSize - 1; i++) { + assertFalse(cmdRes.contains(keyPrefix + i)); + } + // Note: key2 is shown in the continuation token prompt + for (int i = pageSize - 1; i < numKeys; i++) { + assertTrue(cmdRes.contains(keyPrefix + i)); + } + + // hsync last key + streams[numKeys - 1].hsync(); + // Wait for flush + cluster.getOzoneManager().awaitDoubleBufferFlush(); + + execute(ozoneAdminShell, args); + cmdRes = getStdOut(); + + // Verify that only one key is hsync'ed + assertTrue(cmdRes.contains("\tYes\t"), "One key should be hsync'ed"); + assertTrue(cmdRes.contains("\tNo\t"), "One key should not be hsync'ed"); + } finally { + // Cleanup + IOUtils.closeQuietly(streams); + } + + } + + /** + * Return stdout as a String, then clears existing output. + */ + private String getStdOut() throws UnsupportedEncodingException { + String res = out.toString(UTF_8.name()); + out.reset(); + return res; + } + /** * Helper function to retrieve Ozone client configuration for trash testing. * @param hostPrefix Scheme + Authority. e.g. ofs://om-service-test1 @@ -1552,6 +1688,8 @@ public void testKeyDeleteWhenTrashEnableOBS() } @Test + // Run this UT last. This interferes with testAdminCmdListOpenFiles + @Order(Integer.MAX_VALUE) public void testRecursiveBucketDelete() throws Exception { String volume1 = "volume50"; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHAWithFSO.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHAWithFSO.java index 3d1757ecbd9c..a0ad35500caf 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHAWithFSO.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHAWithFSO.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.shell; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -37,6 +38,7 @@ public static void init() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT, OMConfigKeys.OZONE_BUCKET_LAYOUT_FILE_SYSTEM_OPTIMIZED); + conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); startKMS(); startCluster(conf); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/utils/FaultInjectorImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/utils/FaultInjectorImpl.java new file mode 100644 index 000000000000..8656811fa87c --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/utils/FaultInjectorImpl.java @@ -0,0 +1,83 @@ +/* + * 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.utils; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hdds.utils.FaultInjector; +import org.assertj.core.api.Fail; +import org.junit.jupiter.api.Assertions; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; + +/** + * A general FaultInjector implementation. + */ +public class FaultInjectorImpl extends FaultInjector { + private CountDownLatch ready; + private CountDownLatch wait; + private Throwable ex; + + public FaultInjectorImpl() { + init(); + } + + @Override + public void init() { + this.ready = new CountDownLatch(1); + this.wait = new CountDownLatch(1); + } + + @Override + public void pause() throws IOException { + ready.countDown(); + try { + wait.await(); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + + @Override + public void resume() throws IOException { + // Make sure injector pauses before resuming. + try { + ready.await(); + } catch (InterruptedException e) { + e.printStackTrace(); + Assertions.assertTrue(Fail.fail("resume interrupted")); + } + wait.countDown(); + } + + @Override + public void reset() throws IOException { + init(); + } + + @VisibleForTesting + public void setException(Throwable e) { + ex = e; + } + + @VisibleForTesting + public Throwable getException() { + return ex; + } +} + diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto index b0d26020c8d2..2106628a568c 100644 --- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto +++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto @@ -147,6 +147,8 @@ enum Type { ListStatusLight = 129; GetSnapshotInfo = 130; RenameSnapshot = 131; + + ListOpenFiles = 132; } enum SafeMode { @@ -283,6 +285,8 @@ message OMRequest { optional SetSnapshotPropertyRequest SetSnapshotPropertyRequest = 127; optional SnapshotInfoRequest SnapshotInfoRequest = 128; optional RenameSnapshotRequest RenameSnapshotRequest = 129; + + optional ListOpenFilesRequest ListOpenFilesRequest = 130; } message OMResponse { @@ -406,6 +410,8 @@ message OMResponse { optional SnapshotInfoResponse SnapshotInfoResponse = 130; optional OMLockDetailsProto omLockDetails = 131; optional RenameSnapshotResponse RenameSnapshotResponse = 132; + + optional ListOpenFilesResponse ListOpenFilesResponse = 133; } enum Status { @@ -529,6 +535,11 @@ enum Status { INVALID_PATH = 93; TOO_MANY_BUCKETS = 94; + + KEY_UNDER_LEASE_RECOVERY = 95; + KEY_ALREADY_CLOSED = 96; + KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD = 97; + } /** @@ -1420,6 +1431,7 @@ message CommitKeyRequest { required KeyArgs keyArgs = 1; required uint64 clientID = 2; optional bool hsync = 3; + optional bool recovery = 4; } message CommitKeyResponse { @@ -1527,6 +1539,24 @@ message CancelPrepareResponse { } +message ListOpenFilesRequest { + optional string path = 1; + optional uint32 count = 2; + optional string token = 3; +} + +message ListOpenFilesResponse { + // size of openKeyTable and openFileTable combined + optional uint64 totalOpenKeyCount = 1; + // indicates if there are more entries to be retrieved under the given path + optional bool hasMore = 2; + // continuation token should match a dbKey in openKeyTable or openFileTable + optional string continuationToken = 3; + // result + repeated uint64 clientID = 4; + repeated KeyInfo keyInfo = 5; +} + message ServicePort { enum Type { RPC = 1; @@ -2099,10 +2129,13 @@ message RecoverLeaseRequest { optional string volumeName = 1; optional string bucketName = 2; optional string keyName = 3; + optional bool force = 4; } message RecoverLeaseResponse { - optional bool response = 1; + optional bool response = 1 [deprecated=true]; + optional KeyInfo keyInfo = 2; + optional bool isKeyInfo = 3 [default = true]; } message SetTimesRequest { diff --git a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java index f00383cc1b37..51e0808b17de 100644 --- a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java +++ b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.utils.db.cache.CacheValue; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.ozone.om.helpers.ListKeysResult; +import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmDBAccessIdInfo; import org.apache.hadoop.ozone.om.helpers.OmDBUserPrincipalInfo; @@ -55,6 +56,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.ozone.compaction.log.CompactionLogEntry; +import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; + /** * OM metadata manager interface. */ @@ -124,6 +127,15 @@ public interface OMMetadataManager extends DBStoreHAManager { */ String getOzoneKey(String volume, String bucket, String key); + /** + * Get DB key for a key or prefix in an FSO bucket given existing + * volume and bucket names. + */ + String getOzoneKeyFSO(String volumeName, + String bucketName, + String keyPrefix) + throws IOException; + /** * Given a volume, bucket and a key, return the corresponding DB directory * key. @@ -162,6 +174,17 @@ default String getOpenKey(String volume, String bucket, String key, long id) { */ String getOpenKey(String volume, String bucket, String key, String clientId); + /** + * Returns client ID in Long of an OpenKeyTable DB Key String. + * @param dbOpenKeyName An OpenKeyTable DB Key String. + * @return Client ID (Long) + */ + static long getClientIDFromOpenKeyDBKey(String dbOpenKeyName) { + final int lastPrefix = dbOpenKeyName.lastIndexOf(OM_KEY_PREFIX); + final String clientIdString = dbOpenKeyName.substring(lastPrefix + 1); + return Long.parseLong(clientIdString); + } + /** * Given a volume, check if it is empty, i.e there are no buckets inside it. * @@ -201,6 +224,24 @@ List listBuckets(String volumeName, String startBucket, boolean hasSnapshot) throws IOException; + /** + * Inner implementation of listOpenFiles. Called after all the arguments are + * checked and processed by Ozone Manager. + * @param bucketLayout + * @param maxKeys + * @param dbOpenKeyPrefix + * @param hasContToken + * @param dbContTokenPrefix + * @return ListOpenFilesResult + * @throws IOException + */ + ListOpenFilesResult listOpenFiles(BucketLayout bucketLayout, + int maxKeys, + String dbOpenKeyPrefix, + boolean hasContToken, + String dbContTokenPrefix) + throws IOException; + /** * Returns a list of keys represented by {@link OmKeyInfo} in the given * bucket. @@ -291,6 +332,12 @@ boolean recoverTrash(String volumeName, String bucketName, List listVolumes(String userName, String prefix, String startKey, int maxKeys) throws IOException; + /** + * Get total open key count (estimated, due to the nature of RocksDB impl) + * of both OpenKeyTable and OpenFileTable. + */ + long getTotalOpenKeyCount() throws IOException; + /** * Returns the names of up to {@code count} open keys whose age is * greater than or equal to {@code expireThreshold}. @@ -302,7 +349,7 @@ List listVolumes(String userName, String prefix, * @throws IOException */ ExpiredOpenKeys getExpiredOpenKeys(Duration expireThreshold, int count, - BucketLayout bucketLayout) throws IOException; + BucketLayout bucketLayout, Duration leaseThreshold) throws IOException; /** * Returns the names of up to {@code count} MPU key whose age is greater diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java index 4804b317bc7d..54de09f3328b 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java @@ -103,7 +103,9 @@ public enum OMAction implements AuditAction { SNAPSHOT_INFO, SET_TIMES, - ABORT_EXPIRED_MULTIPART_UPLOAD; + ABORT_EXPIRED_MULTIPART_UPLOAD, + + LIST_OPEN_FILES; @Override public String getAction() { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java index 4378701426c2..7a3312c0685a 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java @@ -145,11 +145,12 @@ List listTrash(String volumeName, String bucketName, * @param count The maximum number of expired open keys to return. * @param expireThreshold The threshold of open key expiration age. * @param bucketLayout The type of open keys to get (e.g. DEFAULT or FSO). + * @param leaseThreshold The threshold of hsync key. * @return the expired open keys. * @throws IOException */ ExpiredOpenKeys getExpiredOpenKeys(Duration expireThreshold, int count, - BucketLayout bucketLayout) throws IOException; + BucketLayout bucketLayout, Duration leaseThreshold) throws IOException; /** * Returns the MPU infos of up to {@code count} whose age is greater diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java index d2ca26e3fc08..2527f1ec9a6c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java @@ -686,9 +686,9 @@ public PendingKeysDeletion getPendingDeletionKeys(final int count) @Override public ExpiredOpenKeys getExpiredOpenKeys(Duration expireThreshold, - int count, BucketLayout bucketLayout) throws IOException { + int count, BucketLayout bucketLayout, Duration leaseThreshold) throws IOException { return metadataManager.getExpiredOpenKeys(expireThreshold, count, - bucketLayout); + bucketLayout, leaseThreshold); } @Override diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java index 7003f818bf1a..1c0ec78cfb22 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java @@ -84,6 +84,7 @@ public class OMMetrics implements OmMetadataReaderMetrics { private @Metric MutableCounterLong numCreateFile; private @Metric MutableCounterLong numLookupFile; private @Metric MutableCounterLong numListStatus; + private @Metric MutableCounterLong numListOpenFiles; private @Metric MutableCounterLong numOpenKeyDeleteRequests; private @Metric MutableCounterLong numOpenKeysSubmittedForDeletion; @@ -182,6 +183,7 @@ public class OMMetrics implements OmMetadataReaderMetrics { private @Metric MutableCounterLong numCreateFileFails; private @Metric MutableCounterLong numLookupFileFails; private @Metric MutableCounterLong numListStatusFails; + private @Metric MutableCounterLong numListOpenFilesFails; private @Metric MutableCounterLong getNumGetKeyInfoFails; private @Metric MutableCounterLong numRecoverLeaseFails; @@ -434,10 +436,18 @@ public void incNumListS3Buckets() { } public void incNumListS3BucketsFails() { - numBucketOps.incr(); numBucketS3ListFails.incr(); } + public void incNumListOpenFiles() { + numKeyOps.incr(); + numListOpenFiles.incr(); + } + + public void incNumListOpenFilesFails() { + numListOpenFilesFails.incr(); + } + public void incNumInitiateMultipartUploads() { numKeyOps.incr(); numInitiateMultipartUploads.incr(); @@ -1124,6 +1134,11 @@ public long getNumKeyHSyncs() { return numKeyHSyncs.value(); } + @VisibleForTesting + public void resetNumKeyHSyncs() { + numKeyHSyncs.incr(-numKeyHSyncs.value()); + } + @VisibleForTesting public long getNumKeyCommitFails() { return numKeyCommitFails.value(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index e35ab118e63f..afc9eae859a5 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -64,6 +64,7 @@ import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; import org.apache.hadoop.ozone.om.helpers.ListKeysResult; +import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmDBAccessIdInfo; import org.apache.hadoop.ozone.om.helpers.OmDBUserPrincipalInfo; @@ -75,6 +76,7 @@ import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo; import org.apache.hadoop.ozone.om.helpers.OmDBTenantState; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.S3SecretValue; @@ -272,14 +274,14 @@ public class OmMetadataManagerImpl implements OMMetadataManager, private Table bucketTable; private Table keyTable; private Table deletedTable; - private Table openKeyTable; + private Table openKeyTable; private Table multipartInfoTable; private Table s3SecretTable; private Table dTokenTable; private Table prefixTable; private Table dirTable; private Table fileTable; - private Table openFileTable; + private Table openFileTable; private Table transactionInfoTable; private Table metaTable; @@ -475,7 +477,7 @@ public Table getDeletedDirTable() { } @Override - public Table getOpenKeyTable(BucketLayout bucketLayout) { + public Table getOpenKeyTable(BucketLayout bucketLayout) { if (bucketLayout.isFileSystemOptimized()) { return openFileTable; } @@ -842,6 +844,18 @@ public String getOzoneKey(String volume, String bucket, String key) { return builder.toString(); } + @Override + public String getOzoneKeyFSO(String volumeName, + String bucketName, + String keyPrefix) + throws IOException { + final long volumeId = getVolumeId(volumeName); + final long bucketId = getBucketId(volumeName, bucketName); + // FSO keyPrefix could look like: -9223372036854774527/key1 + return getOzoneKey(Long.toString(volumeId), + Long.toString(bucketId), keyPrefix); + } + @Override public String getOzoneDirKey(String volume, String bucket, String key) { key = OzoneFSUtils.addTrailingSlashIfNeeded(key); @@ -1158,6 +1172,69 @@ public List listBuckets(final String volumeName, return keyTable.iterator(); } + @Override + public ListOpenFilesResult listOpenFiles(BucketLayout bucketLayout, + int maxKeys, + String dbOpenKeyPrefix, + boolean hasContToken, + String dbContTokenPrefix) + throws IOException { + + List openKeySessionList = new ArrayList<>(); + int currentCount = 0; + final boolean hasMore; + final String retContToken; + + // TODO: If we want "better" results, we want to iterate cache like + // listKeys do. But that complicates the iteration logic by quite a bit. + // And if we do that, we need to refactor listKeys as well to dedup. + + final Table okTable; + okTable = getOpenKeyTable(bucketLayout); + + // No lock required since table iterator creates a "snapshot" + try (TableIterator> + openKeyIter = okTable.iterator()) { + KeyValue kv; + kv = openKeyIter.seek(dbContTokenPrefix); + if (hasContToken && kv.getKey().equals(dbContTokenPrefix)) { + // Skip one entry when cont token is specified and the current entry + // key is exactly the same as cont token. + openKeyIter.next(); + } + while (currentCount < maxKeys && openKeyIter.hasNext()) { + kv = openKeyIter.next(); + if (kv != null && kv.getKey().startsWith(dbOpenKeyPrefix)) { + String dbKey = kv.getKey(); + long clientID = OMMetadataManager.getClientIDFromOpenKeyDBKey(dbKey); + OmKeyInfo omKeyInfo = kv.getValue(); + // Note with HDDS-10077, there is no need to check KeyTable for hsync metadata + openKeySessionList.add( + new OpenKeySession(clientID, omKeyInfo, + omKeyInfo.getLatestVersionLocations().getVersion())); + currentCount++; + } + } + + // Set hasMore flag as a hint for client-side pagination + if (openKeyIter.hasNext()) { + KeyValue nextKv = openKeyIter.next(); + hasMore = nextKv != null && nextKv.getKey().startsWith(dbOpenKeyPrefix); + } else { + hasMore = false; + } + + // Set continuation token + retContToken = hasMore ? kv.getKey() : null; + } + + return new ListOpenFilesResult( + getTotalOpenKeyCount(), + hasMore, + retContToken, + openKeySessionList); + } + @Override public ListKeysResult listKeys(String volumeName, String bucketName, String startKey, String keyPrefix, int maxKeys) @@ -1712,9 +1789,16 @@ private boolean isOpenMultipartKey(OmKeyInfo openKeyInfo, String openDbKey) return getMultipartInfoTable().isExist(multipartInfoDbKey); } + @Override + public long getTotalOpenKeyCount() throws IOException { + // Get an estimated key count of OpenKeyTable + OpenFileTable + return openKeyTable.getEstimatedKeyCount() + + openFileTable.getEstimatedKeyCount(); + } + @Override public ExpiredOpenKeys getExpiredOpenKeys(Duration expireThreshold, - int count, BucketLayout bucketLayout) throws IOException { + int count, BucketLayout bucketLayout, Duration leaseThreshold) throws IOException { final ExpiredOpenKeys expiredKeys = new ExpiredOpenKeys(); final Table kt = getKeyTable(bucketLayout); @@ -1727,6 +1811,8 @@ public ExpiredOpenKeys getExpiredOpenKeys(Duration expireThreshold, final long expiredCreationTimestamp = expireThreshold.negated().plusMillis(Time.now()).toMillis(); + final long expiredLeaseTimestamp = + leaseThreshold.negated().plusMillis(Time.now()).toMillis(); int num = 0; while (num < count && keyValueTableIterator.hasNext()) { @@ -1741,22 +1827,25 @@ public ExpiredOpenKeys getExpiredOpenKeys(Duration expireThreshold, continue; } - if (openKeyInfo.getCreationTime() <= expiredCreationTimestamp) { + if (openKeyInfo.getCreationTime() <= expiredCreationTimestamp || + openKeyInfo.getModificationTime() <= expiredLeaseTimestamp) { final String clientIdString = dbOpenKeyName.substring(lastPrefix + 1); - final OmKeyInfo info = kt.get(dbKeyName); - final boolean isHsync = java.util.Optional.ofNullable(info) + final boolean isHsync = java.util.Optional.of(openKeyInfo) .map(WithMetadata::getMetadata) .map(meta -> meta.get(OzoneConsts.HSYNC_CLIENT_ID)) .filter(id -> id.equals(clientIdString)) .isPresent(); - if (!isHsync) { + if (!isHsync && openKeyInfo.getCreationTime() <= expiredCreationTimestamp) { // add non-hsync'ed keys expiredKeys.addOpenKey(openKeyInfo, dbOpenKeyName); - } else { + num++; + } else if (isHsync && openKeyInfo.getModificationTime() <= expiredLeaseTimestamp && + !openKeyInfo.getMetadata().containsKey(OzoneConsts.LEASE_RECOVERY)) { // add hsync'ed keys + final OmKeyInfo info = kt.get(dbKeyName); final KeyArgs.Builder keyArgs = KeyArgs.newBuilder() .setVolumeName(info.getVolumeName()) .setBucketName(info.getBucketName()) @@ -1773,8 +1862,8 @@ public ExpiredOpenKeys getExpiredOpenKeys(Duration expireThreshold, info.getReplicationConfig(), keyArgs); expiredKeys.addHsyncKey(keyArgs, Long.parseLong(clientIdString)); + num++; } - num++; } } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index 680ebe063156..5966d969de70 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -44,6 +44,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.StringTokenizer; import java.util.Timer; import java.util.TimerTask; import java.util.UUID; @@ -90,6 +91,8 @@ import org.apache.hadoop.hdds.utils.db.Table.KeyValue; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.OzoneManagerVersion; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; import org.apache.hadoop.ozone.om.helpers.SnapshotDiffJob; import org.apache.hadoop.ozone.om.lock.OMLockDetails; import org.apache.hadoop.ozone.om.ratis_snapshot.OmRatisSnapshotProvider; @@ -261,6 +264,7 @@ import static org.apache.hadoop.ozone.OzoneConsts.DB_TRANSIENT_MARKER; import static org.apache.hadoop.ozone.OzoneConsts.DEFAULT_OM_UPDATE_ID; import static org.apache.hadoop.ozone.OzoneConsts.LAYOUT_VERSION_KEY; +import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; import static org.apache.hadoop.ozone.OzoneConsts.OM_METRICS_FILE; import static org.apache.hadoop.ozone.OzoneConsts.OM_METRICS_TEMP_FILE; import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR; @@ -299,7 +303,9 @@ import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FEATURE_NOT_ENABLED; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INTERNAL_ERROR; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_PATH; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_REQUEST; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.PERMISSION_DENIED; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_ERROR_OTHER; import static org.apache.hadoop.ozone.om.s3.S3SecretStoreConfigurationKeys.DEFAULT_SECRET_STORAGE_TYPE; @@ -1651,13 +1657,12 @@ public void start() throws IOException { metadataManager.start(configuration); + startSecretManagerIfNecessary(); // Start Ratis services if (omRatisServer != null) { omRatisServer.start(); } - startSecretManagerIfNecessary(); - upgradeFinalizer.runPrefinalizeStateActions(omStorage, this); Integer layoutVersionInDB = getLayoutVersionInDB(); if (layoutVersionInDB == null || @@ -3221,14 +3226,100 @@ public ServiceInfoEx getServiceInfo() throws IOException { } @Override - public void transferLeadership(String newLeaderId) + public ListOpenFilesResult listOpenFiles(String path, + int maxKeys, + String contToken) throws IOException { - final UserGroupInformation ugi = getRemoteUser(); - if (!isAdmin(ugi)) { - throw new OMException( - "Only Ozone admins are allowed to transfer raft leadership.", - PERMISSION_DENIED); + + metrics.incNumListOpenFiles(); + checkAdminUserPrivilege("list open files."); + + // Using final to make sure they are assigned once and only once in + // every branch. + final String dbOpenKeyPrefix, dbContTokenPrefix; + final String volumeName, bucketName; + final BucketLayout bucketLayout; + + // Process path prefix + if (path == null || path.isEmpty() || path.equals(OM_KEY_PREFIX)) { + // path is root + dbOpenKeyPrefix = ""; + volumeName = ""; + bucketName = ""; + // default to FSO's OpenFileTable. TODO: client option to pass OBS/LEGACY? + bucketLayout = BucketLayout.FILE_SYSTEM_OPTIMIZED; + } else { + // path is bucket or key prefix, break it down to volume, bucket, prefix + StringTokenizer tokenizer = new StringTokenizer(path, OM_KEY_PREFIX); + // Validate path to avoid NoSuchElementException + if (tokenizer.countTokens() < 2) { + metrics.incNumListOpenFilesFails(); + throw new OMException("Invalid path: " + path + ". " + + "Only root level or bucket level path is supported at this time", + INVALID_PATH); + } + + volumeName = tokenizer.nextToken(); + bucketName = tokenizer.nextToken(); + + OmBucketInfo bucketInfo; + try { + // as expected, getBucketInfo throws if volume or bucket does not exist + bucketInfo = getBucketInfo(volumeName, bucketName); + } catch (OMException ex) { + metrics.incNumListOpenFilesFails(); + throw ex; + } catch (IOException ex) { + // Wrap IOException in OMException + metrics.incNumListOpenFilesFails(); + throw new OMException(ex.getMessage(), NOT_SUPPORTED_OPERATION); + } + + final String keyPrefix; + if (tokenizer.hasMoreTokens()) { + // Collect the rest but trim the leading "/" + keyPrefix = tokenizer.nextToken("").substring(1); + } else { + keyPrefix = ""; + } + + // Determine dbKey prefix based on the bucket type + bucketLayout = bucketInfo.getBucketLayout(); + switch (bucketLayout) { + case FILE_SYSTEM_OPTIMIZED: + dbOpenKeyPrefix = metadataManager.getOzoneKeyFSO( + volumeName, bucketName, keyPrefix); + break; + case OBJECT_STORE: + case LEGACY: + dbOpenKeyPrefix = metadataManager.getOzoneKey( + volumeName, bucketName, keyPrefix); + break; + default: + metrics.incNumListOpenFilesFails(); + throw new OMException("Unsupported bucket layout: " + + bucketInfo.getBucketLayout(), NOT_SUPPORTED_OPERATION); + } + } + + // Process cont. token + if (contToken == null || contToken.isEmpty()) { + // if a continuation token is not specified + dbContTokenPrefix = dbOpenKeyPrefix; + } else { + dbContTokenPrefix = contToken; } + + // arg processing done. call inner impl (table iteration) + return metadataManager.listOpenFiles( + bucketLayout, maxKeys, dbOpenKeyPrefix, + !StringUtils.isEmpty(contToken), dbContTokenPrefix); + } + + @Override + public void transferLeadership(String newLeaderId) + throws IOException { + checkAdminUserPrivilege("transfer raft leadership."); if (!isRatisEnabled) { throw new IOException("OM HA not enabled."); } @@ -3345,7 +3436,7 @@ public TenantStateList listTenant() throws IOException { if (!isAdmin(ugi)) { final OMException omEx = new OMException( "Only Ozone admins are allowed to list tenants.", PERMISSION_DENIED); - AUDIT.logWriteFailure(buildAuditMessageForFailure( + AUDIT.logReadFailure(buildAuditMessageForFailure( OMAction.LIST_TENANT, new LinkedHashMap<>(), omEx)); throw omEx; } @@ -4626,9 +4717,8 @@ public EchoRPCResponse echoRPCReq(byte[] payloadReq, int payloadSizeResp, } @Override - public boolean recoverLease(String volumeName, String bucketName, - String keyName) { - return false; + public LeaseKeyInfo recoverLease(String volumeName, String bucketName, String keyName, boolean force) { + return null; } @Override @@ -4811,13 +4901,7 @@ public List listSnapshotDiffJobs(String volume, public String printCompactionLogDag(String fileNamePrefix, String graphType) throws IOException { - - final UserGroupInformation ugi = getRemoteUser(); - if (!isAdmin(ugi)) { - throw new OMException( - "Only Ozone admins are allowed to print compaction DAG.", - PERMISSION_DENIED); - } + checkAdminUserPrivilege("print compaction DAG."); if (StringUtils.isBlank(fileNamePrefix)) { fileNamePrefix = "dag-"; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMRecoverLeaseRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMRecoverLeaseRequest.java index 763c8641c0c9..6116ed81e879 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMRecoverLeaseRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMRecoverLeaseRequest.java @@ -20,7 +20,8 @@ import com.google.common.base.Preconditions; -import org.apache.ratis.server.protocol.TermIndex; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.security.token.OzoneBlockTokenSecretManager; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.audit.OMAction; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -30,35 +31,44 @@ import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmFSOFile; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.om.request.key.OMKeyRequest; import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; import org.apache.hadoop.ozone.om.response.OMClientResponse; import org.apache.hadoop.ozone.om.response.file.OMRecoverLeaseResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .OMResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .OMRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .RecoverLeaseRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .RecoverLeaseResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RecoverLeaseRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RecoverLeaseResponse; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto.READ; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto.WRITE; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT_DEFAULT; +import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.RecoverLease; import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; import org.apache.hadoop.ozone.security.acl.OzoneObj; import org.apache.hadoop.util.Time; +import org.apache.ratis.server.protocol.TermIndex; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.file.InvalidPathException; +import java.util.EnumSet; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_ALREADY_CLOSED; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; -import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .Type.RecoverLease; /** * Perform actions for RecoverLease requests. @@ -72,6 +82,9 @@ public class OMRecoverLeaseRequest extends OMKeyRequest { private String keyName; private OmKeyInfo keyInfo; private String dbFileKey; + private OmKeyInfo openKeyInfo; + private String dbOpenFileKey; + private boolean force; private OMMetadataManager omMetadataManager; @@ -84,6 +97,7 @@ public OMRecoverLeaseRequest(OMRequest omRequest) { volumeName = recoverLeaseRequest.getVolumeName(); bucketName = recoverLeaseRequest.getBucketName(); keyName = recoverLeaseRequest.getKeyName(); + force = recoverLeaseRequest.getForce(); } @Override @@ -139,28 +153,20 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn acquiredLock = getOmLockDetails().isLockAcquired(); validateBucketAndVolume(omMetadataManager, volumeName, bucketName); - String openKeyEntryName = doWork(ozoneManager, termIndex.getIndex()); + RecoverLeaseResponse recoverLeaseResponse = doWork(ozoneManager, termIndex.getIndex()); // Prepare response - boolean responseCode = true; - omResponse - .setRecoverLeaseResponse( - RecoverLeaseResponse.newBuilder() - .setResponse(responseCode) - .build()) - .setCmdType(RecoverLease); - omClientResponse = - new OMRecoverLeaseResponse(omResponse.build(), getBucketLayout(), - keyInfo, dbFileKey, openKeyEntryName); + omResponse.setRecoverLeaseResponse(recoverLeaseResponse).setCmdType(RecoverLease); + omClientResponse = new OMRecoverLeaseResponse(omResponse.build(), getBucketLayout(), + dbOpenFileKey, openKeyInfo); omMetrics.incNumRecoverLease(); - LOG.debug("Key recovered. Volume:{}, Bucket:{}, Key:{}", volumeName, - bucketName, keyName); + LOG.debug("Key recovered. Volume:{}, Bucket:{}, Key:{}", + volumeName, bucketName, keyName); } catch (IOException | InvalidPathException ex) { LOG.error("Fail for recovering lease. Volume:{}, Bucket:{}, Key:{}", volumeName, bucketName, keyName, ex); exception = ex; omMetrics.incNumRecoverLeaseFails(); - omResponse.setCmdType(RecoverLease); omClientResponse = new OMRecoverLeaseResponse( createErrorOMResponse(omResponse, exception), getBucketLayout()); } finally { @@ -182,10 +188,10 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn return omClientResponse; } - private String doWork(OzoneManager ozoneManager, long transactionLogIndex) - throws IOException { - - String errMsg = "Cannot recover file : " + keyName + private RecoverLeaseResponse doWork(OzoneManager ozoneManager, + long transactionLogIndex) throws IOException { + + String errMsg = "Cannot recover file : " + keyName + " as parent directory doesn't exist"; OmFSOFile fsoFile = new OmFSOFile.Builder() @@ -195,48 +201,94 @@ private String doWork(OzoneManager ozoneManager, long transactionLogIndex) .setOmMetadataManager(omMetadataManager) .setErrMsg(errMsg) .build(); - - String fileName = fsoFile.getFileName(); + dbFileKey = fsoFile.getOzonePathKey(); keyInfo = getKey(dbFileKey); if (keyInfo == null) { - throw new OMException("Key:" + keyName + " not found", KEY_NOT_FOUND); + throw new OMException("Key:" + keyName + " not found in keyTable.", KEY_NOT_FOUND); } - final String clientId = keyInfo.getMetadata().remove( - OzoneConsts.HSYNC_CLIENT_ID); - if (clientId == null) { + + final String writerId = keyInfo.getMetadata().get(OzoneConsts.HSYNC_CLIENT_ID); + if (writerId == null) { // if file is closed, do nothing and return right away. - LOG.warn("Key:" + keyName + " is already closed"); - return null; + throw new OMException("Key: " + keyName + " is already closed", KEY_ALREADY_CLOSED); } - String openFileDBKey = fsoFile.getOpenFileName(Long.parseLong(clientId)); - if (openFileDBKey != null) { - commitKey(dbFileKey, keyInfo, fileName, ozoneManager, - transactionLogIndex); - removeOpenKey(openFileDBKey, fileName, transactionLogIndex); + + dbOpenFileKey = fsoFile.getOpenFileName(Long.parseLong(writerId)); + openKeyInfo = omMetadataManager.getOpenKeyTable(getBucketLayout()).get(dbOpenFileKey); + if (openKeyInfo == null) { + throw new OMException("Open Key " + dbOpenFileKey + " not found in openKeyTable", KEY_NOT_FOUND); } - return openFileDBKey; - } + long openKeyModificationTime = openKeyInfo.getModificationTime(); + if (openKeyInfo.getMetadata().containsKey(OzoneConsts.LEASE_RECOVERY)) { + LOG.debug("Key: " + keyName + " is already under recovery"); + } else { + final long leaseSoftLimit = ozoneManager.getConfiguration() + .getTimeDuration(OZONE_OM_LEASE_SOFT_LIMIT, OZONE_OM_LEASE_SOFT_LIMIT_DEFAULT, TimeUnit.MILLISECONDS); + if (!force && Time.now() < openKeyInfo.getModificationTime() + leaseSoftLimit) { + throw new OMException("Open Key " + keyName + " updated recently and is inside soft limit period", + KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD); + } + openKeyInfo.getMetadata().put(OzoneConsts.LEASE_RECOVERY, "true"); + openKeyInfo.setUpdateID(transactionLogIndex, ozoneManager.isRatisEnabled()); + openKeyInfo.setModificationTime(Time.now()); + // add to cache. + omMetadataManager.getOpenKeyTable(getBucketLayout()).addCacheEntry( + dbOpenFileKey, openKeyInfo, transactionLogIndex); + } + // override key name with normalizedKeyPath + keyInfo.setKeyName(keyName); + openKeyInfo.setKeyName(keyName); + + OmKeyLocationInfoGroup keyLatestVersionLocations = keyInfo.getLatestVersionLocations(); + List keyLocationInfoList = keyLatestVersionLocations.getLocationList(); + OmKeyLocationInfoGroup openKeyLatestVersionLocations = openKeyInfo.getLatestVersionLocations(); + List openKeyLocationInfoList = openKeyLatestVersionLocations.getLocationList(); + + OmKeyLocationInfo finalBlock = null; + OmKeyLocationInfo penultimateBlock = null; + boolean returnKeyInfo = true; + if (openKeyLocationInfoList.size() > keyLocationInfoList.size() && + openKeyModificationTime > keyInfo.getModificationTime() && + openKeyLocationInfoList.size() > 0) { + finalBlock = openKeyLocationInfoList.get(openKeyLocationInfoList.size() - 1); + if (openKeyLocationInfoList.size() > 1) { + penultimateBlock = openKeyLocationInfoList.get(openKeyLocationInfoList.size() - 2); + } + returnKeyInfo = false; + } else if (keyLocationInfoList.size() > 0) { + finalBlock = keyLocationInfoList.get(keyLocationInfoList.size() - 1); + } + updateBlockInfo(ozoneManager, finalBlock); + updateBlockInfo(ozoneManager, penultimateBlock); - private OmKeyInfo getKey(String dbOzoneKey) throws IOException { - return omMetadataManager.getKeyTable(getBucketLayout()).get(dbOzoneKey); - } + RecoverLeaseResponse.Builder rb = RecoverLeaseResponse.newBuilder(); + rb.setKeyInfo(returnKeyInfo ? keyInfo.getNetworkProtobuf(getOmRequest().getVersion(), true) : + openKeyInfo.getNetworkProtobuf(getOmRequest().getVersion(), true)); + rb.setIsKeyInfo(returnKeyInfo); - private void commitKey(String dbOzoneKey, OmKeyInfo omKeyInfo, - String fileName, OzoneManager ozoneManager, - long transactionLogIndex) throws IOException { - omKeyInfo.setModificationTime(Time.now()); - omKeyInfo.setUpdateID(transactionLogIndex, ozoneManager.isRatisEnabled()); + return rb.build(); + } - OMFileRequest.addFileTableCacheEntry(omMetadataManager, dbOzoneKey, - omKeyInfo, fileName, transactionLogIndex); + private void updateBlockInfo(OzoneManager ozoneManager, OmKeyLocationInfo blockInfo) throws IOException { + if (blockInfo != null) { + // set token to last block if enabled + if (ozoneManager.isGrpcBlockTokenEnabled()) { + String remoteUser = getRemoteUser().getShortUserName(); + OzoneBlockTokenSecretManager secretManager = ozoneManager.getBlockTokenSecretManager(); + blockInfo.setToken(secretManager.generateToken(remoteUser, blockInfo.getBlockID(), + EnumSet.of(READ, WRITE), blockInfo.getLength())); + } + // refresh last block pipeline + ContainerWithPipeline containerWithPipeline = + ozoneManager.getScmClient().getContainerClient().getContainerWithPipeline(blockInfo.getContainerID()); + blockInfo.setPipeline(containerWithPipeline.getPipeline()); + } } - private void removeOpenKey(String openKeyName, String fileName, - long transactionLogIndex) { - OMFileRequest.addOpenFileTableCacheEntry(omMetadataManager, - openKeyName, null, fileName, transactionLogIndex); + private OmKeyInfo getKey(String dbOzoneKey) throws IOException { + return omMetadataManager.getKeyTable(getBucketLayout()).get(dbOzoneKey); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java index ac4d9ab6244f..d47b2b7a57e2 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java @@ -65,6 +65,7 @@ import org.apache.hadoop.hdds.utils.db.cache.CacheValue; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_UNDER_LEASE_RECOVERY; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; /** @@ -106,7 +107,7 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { // BlockOutputStreamEntryPool, so we are fine for now. But if one some // one uses direct omclient we might be in trouble. - UserInfo userInfo = getUserInfo(); + UserInfo userInfo = getUserIfNotExists(ozoneManager); ReplicationConfig repConfig = ReplicationConfig.fromProto(keyArgs.getType(), keyArgs.getFactor(), keyArgs.getEcReplicationConfig()); // To allocate atleast one block passing requested size and scmBlockSize @@ -203,6 +204,10 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn KEY_NOT_FOUND); } + if (openKeyInfo.getMetadata().containsKey(OzoneConsts.LEASE_RECOVERY)) { + throw new OMException("Open Key " + openKeyName + " is under lease recovery", + KEY_UNDER_LEASE_RECOVERY); + } List newLocationList = Collections.singletonList( OmKeyLocationInfo.getFromProtobuf(blockLocation)); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestWithFSO.java index c4bb52f177e9..4f0c9fe60248 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestWithFSO.java @@ -56,6 +56,7 @@ import java.util.Map; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_UNDER_LEASE_RECOVERY; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; /** @@ -125,7 +126,10 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn throw new OMException("Open Key not found " + openKeyName, KEY_NOT_FOUND); } - + if (openKeyInfo.getMetadata().containsKey(OzoneConsts.LEASE_RECOVERY)) { + throw new OMException("Open Key " + openKeyName + " is under lease recovery", + KEY_UNDER_LEASE_RECOVERY); + } List newLocationList = Collections.singletonList( OmKeyLocationInfo.getFromProtobuf(blockLocation)); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java index 5c3593b07839..830ab70827cb 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java @@ -42,6 +42,7 @@ import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.WithMetadata; import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; +import org.apache.hadoop.ozone.om.request.util.OmKeyHSyncUtil; import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator; import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase; import org.apache.hadoop.ozone.om.request.validation.ValidationCondition; @@ -68,7 +69,9 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; import org.apache.hadoop.util.Time; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_ALREADY_CLOSED; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_UNDER_LEASE_RECOVERY; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_A_FILE; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; @@ -79,8 +82,7 @@ public class OMKeyCommitRequest extends OMKeyRequest { @VisibleForTesting - public static final Logger LOG = - LoggerFactory.getLogger(OMKeyCommitRequest.class); + public static final Logger LOG = LoggerFactory.getLogger(OMKeyCommitRequest.class); public OMKeyCommitRequest(OMRequest omRequest, BucketLayout bucketLayout) { super(omRequest, bucketLayout); @@ -102,12 +104,15 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { OmUtils.validateKeyName(StringUtils.removeEnd(keyArgs.getKeyName(), OzoneConsts.FS_FILE_COPYING_TEMP_SUFFIX)); } - boolean isHsync = commitKeyRequest.hasHsync() && - commitKeyRequest.getHsync(); + boolean isHsync = commitKeyRequest.hasHsync() && commitKeyRequest.getHsync(); + boolean isRecovery = commitKeyRequest.hasRecovery() && commitKeyRequest.getRecovery(); boolean enableHsync = ozoneManager.getConfiguration().getBoolean( OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED_DEFAULT); - if (isHsync && !enableHsync) { + + // If hsynced is called for a file, then this file is hsynced, otherwise it's not hsynced. + // Currently, file lease recovery by design only supports recover hsynced file + if ((isHsync || isRecovery) && !enableHsync) { throw new OMException("Hsync is not enabled. To enable, " + "set ozone.fs.hsync.enabled = true", NOT_SUPPORTED_OPERATION); } @@ -160,24 +165,24 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager(); - boolean isHSync = commitKeyRequest.hasHsync() && - commitKeyRequest.getHsync(); - + boolean isHSync = commitKeyRequest.hasHsync() && commitKeyRequest.getHsync(); + boolean isRecovery = commitKeyRequest.hasRecovery() && commitKeyRequest.getRecovery(); + // isHsync = true, a commit request as a result of client side hsync call + // isRecovery = true, a commit request as a result of client side recoverLease call + // none of isHsync and isRecovery is true, a commit request as a result of client side normal + // outputStream#close call. if (isHSync) { omMetrics.incNumKeyHSyncs(); } else { omMetrics.incNumKeyCommits(); } - LOG.debug("isHSync = {}, volumeName = {}, bucketName = {}, keyName = {}", - isHSync, volumeName, bucketName, keyName); + LOG.debug("isHSync = {}, isRecovery = {}, volumeName = {}, bucketName = {}, keyName = {}", + isHSync, isRecovery, volumeName, bucketName, keyName); try { String dbOzoneKey = - omMetadataManager.getOzoneKey(volumeName, bucketName, - keyName); - String dbOpenKey = omMetadataManager.getOpenKey(volumeName, bucketName, - keyName, commitKeyRequest.getClientID()); + omMetadataManager.getOzoneKey(volumeName, bucketName, keyName); List locationInfoList = getOmKeyLocationInfos(ozoneManager, commitKeyArgs); @@ -220,34 +225,59 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn Map oldKeyVersionsToDeleteMap = null; OmKeyInfo keyToDelete = omMetadataManager.getKeyTable(getBucketLayout()).get(dbOzoneKey); + long writerClientId = commitKeyRequest.getClientID(); + if (isRecovery && keyToDelete != null) { + String clientId = keyToDelete.getMetadata().get(OzoneConsts.HSYNC_CLIENT_ID); + if (clientId == null) { + throw new OMException("Failed to recovery key, as " + + dbOzoneKey + " is already closed", KEY_ALREADY_CLOSED); + } + writerClientId = Long.parseLong(clientId); + } + + final String clientIdString = String.valueOf(writerClientId); if (null != keyToDelete) { - final String clientIdString - = String.valueOf(commitKeyRequest.getClientID()); - isPreviousCommitHsync = java.util.Optional.ofNullable(keyToDelete) + isPreviousCommitHsync = java.util.Optional.of(keyToDelete) .map(WithMetadata::getMetadata) .map(meta -> meta.get(OzoneConsts.HSYNC_CLIENT_ID)) .filter(id -> id.equals(clientIdString)) .isPresent(); } + String dbOpenKey = omMetadataManager.getOpenKey(volumeName, bucketName, + keyName, writerClientId); omKeyInfo = omMetadataManager.getOpenKeyTable(getBucketLayout()).get(dbOpenKey); if (omKeyInfo == null) { - String action = "commit"; - if (isHSync) { - action = "hsync"; - } + String action = isRecovery ? "recovery" : isHSync ? "hsync" : "commit"; throw new OMException("Failed to " + action + " key, as " + dbOpenKey + " entry is not found in the OpenKey table", KEY_NOT_FOUND); } - omKeyInfo.getMetadata().putAll(KeyValueUtil.getFromProtobuf( - commitKeyArgs.getMetadataList())); + if (omKeyInfo.getMetadata().containsKey(OzoneConsts.LEASE_RECOVERY) && + omKeyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID)) { + if (!isRecovery) { + throw new OMException("Cannot commit key " + dbOpenKey + " with " + OzoneConsts.LEASE_RECOVERY + + " metadata while recovery flag is not set in request", KEY_UNDER_LEASE_RECOVERY); + } + } + + omKeyInfo.setModificationTime(commitKeyArgs.getModificationTime()); + + // non-null indicates it is necessary to update the open key + OmKeyInfo newOpenKeyInfo = null; + if (isHSync) { - omKeyInfo.getMetadata().put(OzoneConsts.HSYNC_CLIENT_ID, - String.valueOf(commitKeyRequest.getClientID())); + if (!OmKeyHSyncUtil.isHSyncedPreviously(omKeyInfo, clientIdString, dbOpenKey)) { + // Update open key as well if it is the first hsync of this key + omKeyInfo.getMetadata().put(OzoneConsts.HSYNC_CLIENT_ID, clientIdString); + newOpenKeyInfo = omKeyInfo.copyObject(); + } } + + omKeyInfo.getMetadata().putAll(KeyValueUtil.getFromProtobuf( + commitKeyArgs.getMetadataList())); omKeyInfo.setDataSize(commitKeyArgs.getDataSize()); - omKeyInfo.setModificationTime(commitKeyArgs.getModificationTime()); + // Update the block length for each block, return the allocated but // uncommitted blocks List uncommitted = @@ -311,11 +341,21 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn // Add to cache of open key table and key table. if (!isHSync) { - // If isHSync = false, put a tombstone in OpenKeyTable cache, + // If !isHSync = true, put a tombstone in OpenKeyTable cache, // indicating the key is removed from OpenKeyTable. // So that this key can't be committed again. omMetadataManager.getOpenKeyTable(getBucketLayout()).addCacheEntry( dbOpenKey, trxnLogIndex); + + // Prevent hsync metadata from getting committed to the final key + omKeyInfo.getMetadata().remove(OzoneConsts.HSYNC_CLIENT_ID); + if (isRecovery) { + omKeyInfo.getMetadata().remove(OzoneConsts.LEASE_RECOVERY); + } + } else if (newOpenKeyInfo != null) { + // isHSync is true and newOpenKeyInfo is set, update OpenKeyTable + omMetadataManager.getOpenKeyTable(getBucketLayout()).addCacheEntry( + dbOpenKey, newOpenKeyInfo, trxnLogIndex); } omMetadataManager.getKeyTable(getBucketLayout()).addCacheEntry( @@ -325,7 +365,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn omClientResponse = new OMKeyCommitResponse(omResponse.build(), omKeyInfo, dbOzoneKey, dbOpenKey, omBucketInfo.copyObject(), - oldKeyVersionsToDeleteMap, isHSync); + oldKeyVersionsToDeleteMap, isHSync, newOpenKeyInfo); result = Result.SUCCESS; } catch (IOException | InvalidPathException ex) { @@ -344,12 +384,12 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn } // Debug logging for any key commit operation, successful or not - LOG.debug("Key commit {} with isHSync = {}, omKeyInfo = {}", - result == Result.SUCCESS ? "succeeded" : "failed", isHSync, omKeyInfo); + LOG.debug("Key commit {} with isHSync = {}, isRecovery = {}, omKeyInfo = {}", + result == Result.SUCCESS ? "succeeded" : "failed", isHSync, isRecovery, omKeyInfo); if (!isHSync) { auditLog(auditLogger, buildAuditMessage(OMAction.COMMIT_KEY, auditMap, - exception, getOmRequest().getUserInfo())); + exception, getOmRequest().getUserInfo())); processResult(commitKeyRequest, volumeName, bucketName, keyName, omMetrics, exception, omKeyInfo, result); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java index 704e9e91c47d..0362f068e8e1 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java @@ -22,6 +22,7 @@ import java.util.HashMap; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.ozone.om.request.util.OmKeyHSyncUtil; import org.apache.ratis.server.protocol.TermIndex; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.audit.AuditLogger; @@ -53,7 +54,9 @@ import java.util.List; import java.util.Map; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_ALREADY_CLOSED; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_UNDER_LEASE_RECOVERY; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; /** @@ -98,17 +101,20 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn OMClientResponse omClientResponse = null; boolean bucketLockAcquired = false; Result result; - boolean isHSync = commitKeyRequest.hasHsync() && - commitKeyRequest.getHsync(); - + boolean isHSync = commitKeyRequest.hasHsync() && commitKeyRequest.getHsync(); + boolean isRecovery = commitKeyRequest.hasRecovery() && commitKeyRequest.getRecovery(); + // isHsync = true, a commit request as a result of client side hsync call + // isRecovery = true, a commit request as a result of client side recoverLease call + // none of isHsync and isRecovery is true, a commit request as a result of client side normal + // outputStream#close call. if (isHSync) { omMetrics.incNumKeyHSyncs(); } else { omMetrics.incNumKeyCommits(); } - LOG.debug("isHSync = {}, volumeName = {}, bucketName = {}, keyName = {}", - isHSync, volumeName, bucketName, keyName); + LOG.debug("isHSync = {}, isRecovery = {}, volumeName = {}, bucketName = {}, keyName = {}", + isHSync, isRecovery, volumeName, bucketName, keyName); OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager(); @@ -138,30 +144,52 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn String fileName = fsoFile.getFileName(); long volumeId = fsoFile.getVolumeId(); String dbFileKey = fsoFile.getOzonePathKey(); - dbOpenFileKey = fsoFile.getOpenFileName(commitKeyRequest.getClientID()); - + OmKeyInfo keyToDelete = + omMetadataManager.getKeyTable(getBucketLayout()).get(dbFileKey); + long writerClientId = commitKeyRequest.getClientID(); + if (isRecovery && keyToDelete != null) { + String clientId = keyToDelete.getMetadata().get(OzoneConsts.HSYNC_CLIENT_ID); + if (clientId == null) { + throw new OMException("Failed to recovery key, as " + + dbFileKey + " is already closed", KEY_ALREADY_CLOSED); + } + writerClientId = Long.parseLong(clientId); + } + dbOpenFileKey = fsoFile.getOpenFileName(writerClientId); omKeyInfo = OMFileRequest.getOmKeyInfoFromFileTable(true, omMetadataManager, dbOpenFileKey, keyName); if (omKeyInfo == null) { - String action = "commit"; - if (isHSync) { - action = "hsync"; - } + String action = isRecovery ? "recovery" : isHSync ? "hsync" : "commit"; throw new OMException("Failed to " + action + " key, as " + - dbOpenFileKey + "entry is not found in the OpenKey table", - KEY_NOT_FOUND); + dbOpenFileKey + " entry is not found in the OpenKey table", KEY_NOT_FOUND); } - omKeyInfo.getMetadata().putAll(KeyValueUtil.getFromProtobuf( - commitKeyArgs.getMetadataList())); + + if (omKeyInfo.getMetadata().containsKey(OzoneConsts.LEASE_RECOVERY) && + omKeyInfo.getMetadata().containsKey(OzoneConsts.HSYNC_CLIENT_ID)) { + if (!isRecovery) { + throw new OMException("Cannot commit key " + dbOpenFileKey + " with " + OzoneConsts.LEASE_RECOVERY + + " metadata while recovery flag is not set in request", KEY_UNDER_LEASE_RECOVERY); + } + } + + omKeyInfo.setModificationTime(commitKeyArgs.getModificationTime()); + + final String clientIdString = String.valueOf(writerClientId); + // non-null indicates it is necessary to update the open key + OmKeyInfo newOpenKeyInfo = null; + if (isHSync) { - omKeyInfo.getMetadata().put(OzoneConsts.HSYNC_CLIENT_ID, - String.valueOf(commitKeyRequest.getClientID())); + if (!OmKeyHSyncUtil.isHSyncedPreviously(omKeyInfo, clientIdString, dbOpenFileKey)) { + // Update open key as well if it is the first hsync of this key + omKeyInfo.getMetadata().put(OzoneConsts.HSYNC_CLIENT_ID, clientIdString); + newOpenKeyInfo = omKeyInfo.copyObject(); + } } + omKeyInfo.getMetadata().putAll(KeyValueUtil.getFromProtobuf( + commitKeyArgs.getMetadataList())); omKeyInfo.setDataSize(commitKeyArgs.getDataSize()); - omKeyInfo.setModificationTime(commitKeyArgs.getModificationTime()); - List uncommitted = omKeyInfo.updateLocationInfoList(locationInfoList, false); @@ -174,12 +202,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn // creation after the knob turned on. boolean isPreviousCommitHsync = false; Map oldKeyVersionsToDeleteMap = null; - OmKeyInfo keyToDelete = - omMetadataManager.getKeyTable(getBucketLayout()).get(dbFileKey); if (null != keyToDelete) { - final String clientIdString - = String.valueOf(commitKeyRequest.getClientID()); - isPreviousCommitHsync = java.util.Optional.ofNullable(keyToDelete) + isPreviousCommitHsync = java.util.Optional.of(keyToDelete) .map(WithMetadata::getMetadata) .map(meta -> meta.get(OzoneConsts.HSYNC_CLIENT_ID)) .filter(id -> id.equals(clientIdString)) @@ -251,6 +275,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn // So that this key can't be committed again. OMFileRequest.addOpenFileTableCacheEntry(omMetadataManager, dbOpenFileKey, null, fileName, trxnLogIndex); + + // Prevent hsync metadata from getting committed to the final key + omKeyInfo.getMetadata().remove(OzoneConsts.HSYNC_CLIENT_ID); + if (isRecovery) { + omKeyInfo.getMetadata().remove(OzoneConsts.LEASE_RECOVERY); + } + } else if (newOpenKeyInfo != null) { + // isHSync is true and newOpenKeyInfo is set, update OpenKeyTable + OMFileRequest.addOpenFileTableCacheEntry(omMetadataManager, + dbOpenFileKey, newOpenKeyInfo, fileName, trxnLogIndex); } OMFileRequest.addFileTableCacheEntry(omMetadataManager, dbFileKey, @@ -259,8 +293,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn omBucketInfo.incrUsedBytes(correctedSpace); omClientResponse = new OMKeyCommitResponseWithFSO(omResponse.build(), - omKeyInfo, dbFileKey, dbOpenFileKey, omBucketInfo.copyObject(), - oldKeyVersionsToDeleteMap, volumeId, isHSync); + omKeyInfo, dbFileKey, dbOpenFileKey, omBucketInfo.copyObject(), + oldKeyVersionsToDeleteMap, volumeId, isHSync, newOpenKeyInfo); result = Result.SUCCESS; } catch (IOException | InvalidPathException ex) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java index b72300533e54..ae3715be7bfd 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java @@ -234,7 +234,7 @@ protected List< OmKeyLocationInfo > allocateBlock(ScmClient scmClient, /* Optimize ugi lookup for RPC operations to avoid a trip through * UGI.getCurrentUser which is synch'ed. */ - private UserGroupInformation getRemoteUser() throws IOException { + protected UserGroupInformation getRemoteUser() throws IOException { UserGroupInformation ugi = Server.getRemoteUser(); return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser(); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OmKeysDeleteRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OmKeysDeleteRequestWithFSO.java index 255a3db9fef3..b90fd15b267f 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OmKeysDeleteRequestWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OmKeysDeleteRequestWithFSO.java @@ -31,9 +31,9 @@ import org.apache.hadoop.ozone.om.response.OMClientResponse; import org.apache.hadoop.ozone.om.response.key.OMKeysDeleteResponseWithFSO; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; -import jakarta.annotation.Nonnull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import jakarta.annotation.Nonnull; import java.io.IOException; import java.util.List; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/OmKeyHSyncUtil.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/OmKeyHSyncUtil.java new file mode 100644 index 000000000000..e32b2e21770f --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/util/OmKeyHSyncUtil.java @@ -0,0 +1,56 @@ +/** + * 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.ozone.om.request.util; + +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Helper methods related to OM key HSync. + */ +public final class OmKeyHSyncUtil { + + public static final Logger LOG = LoggerFactory.getLogger(OmKeyHSyncUtil.class); + + private OmKeyHSyncUtil() { + } + + /** + * Returns true if the key has been hsync'ed before (has metadata HSYNC_CLIENT_ID). + * @param omKeyInfo OmKeyInfo + * @param clientIdString Client ID String + * @param dbOpenKey dbOpenKey + */ + public static boolean isHSyncedPreviously(OmKeyInfo omKeyInfo, String clientIdString, String dbOpenKey) { + // Check whether the key has been hsync'ed before + final String previousHsyncClientId = omKeyInfo.getMetadata().get(OzoneConsts.HSYNC_CLIENT_ID); + if (previousHsyncClientId != null) { + if (clientIdString.equals(previousHsyncClientId)) { + // Same client ID, no need to update OpenKeyTable. One less DB write + return true; + } else { + // Sanity check. Should never enter + LOG.warn("Client ID '{}' currently hsync'ing key does not match previous hsync client ID '{}'. dbOpenKey='{}'", + clientIdString, previousHsyncClientId, dbOpenKey); + } + } + return false; + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/file/OMRecoverLeaseResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/file/OMRecoverLeaseResponse.java index 792cadf22245..690d9bf0d4e4 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/file/OMRecoverLeaseResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/file/OMRecoverLeaseResponse.java @@ -25,7 +25,7 @@ import org.apache.hadoop.ozone.om.response.CleanupTableInfo; import org.apache.hadoop.ozone.om.response.key.OmKeyResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .OMResponse; + .OMResponse; import jakarta.annotation.Nonnull; import java.io.IOException; @@ -39,16 +39,14 @@ @CleanupTableInfo(cleanupTables = {FILE_TABLE, OPEN_FILE_TABLE}) public class OMRecoverLeaseResponse extends OmKeyResponse { - private OmKeyInfo keyInfo; - private String dbFileKey; + private OmKeyInfo openKeyInfo; private String openKeyName; + public OMRecoverLeaseResponse(@Nonnull OMResponse omResponse, - BucketLayout bucketLayout, OmKeyInfo keyInfo, String dbFileKey, - String openKeyName) { + BucketLayout bucketLayout, String openKeyName, OmKeyInfo openKeyInfo) { super(omResponse, bucketLayout); - this.keyInfo = keyInfo; - this.dbFileKey = dbFileKey; this.openKeyName = openKeyName; + this.openKeyInfo = openKeyInfo; } /** @@ -64,12 +62,11 @@ public OMRecoverLeaseResponse(@Nonnull OMResponse omResponse, @Override protected void addToDBBatch(OMMetadataManager omMetadataManager, BatchOperation batchOperation) throws IOException { - // Delete from OpenKey table + // Update OpenKey table if (openKeyName != null) { - omMetadataManager.getOpenKeyTable(getBucketLayout()).deleteWithBatch( - batchOperation, openKeyName); - omMetadataManager.getKeyTable(getBucketLayout()) - .putWithBatch(batchOperation, dbFileKey, keyInfo); + // In INIT stage, update the keyInfo in openKeyTable + omMetadataManager.getOpenKeyTable(getBucketLayout()).putWithBatch( + batchOperation, openKeyName, openKeyInfo); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java index a9f32420dbef..685b2969808b 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java @@ -49,15 +49,17 @@ public class OMKeyCommitResponse extends OmKeyResponse { private String openKeyName; private OmBucketInfo omBucketInfo; private Map keyToDeleteMap; - private boolean isHSync; + private OmKeyInfo newOpenKeyInfo; + @SuppressWarnings("checkstyle:ParameterNumber") public OMKeyCommitResponse( @Nonnull OMResponse omResponse, @Nonnull OmKeyInfo omKeyInfo, String ozoneKeyName, String openKeyName, @Nonnull OmBucketInfo omBucketInfo, Map keyToDeleteMap, - boolean isHSync) { + boolean isHSync, + OmKeyInfo newOpenKeyInfo) { super(omResponse, omBucketInfo.getBucketLayout()); this.omKeyInfo = omKeyInfo; this.ozoneKeyName = ozoneKeyName; @@ -65,6 +67,7 @@ public OMKeyCommitResponse( this.omBucketInfo = omBucketInfo; this.keyToDeleteMap = keyToDeleteMap; this.isHSync = isHSync; + this.newOpenKeyInfo = newOpenKeyInfo; } /** @@ -84,7 +87,10 @@ public void addToDBBatch(OMMetadataManager omMetadataManager, // Delete from OpenKey table if (!isHSync()) { omMetadataManager.getOpenKeyTable(getBucketLayout()) - .deleteWithBatch(batchOperation, openKeyName); + .deleteWithBatch(batchOperation, openKeyName); + } else if (newOpenKeyInfo != null) { + omMetadataManager.getOpenKeyTable(getBucketLayout()).putWithBatch( + batchOperation, openKeyName, newOpenKeyInfo); } omMetadataManager.getKeyTable(getBucketLayout()) @@ -133,4 +139,8 @@ protected void updateDeletedTable(OMMetadataManager omMetadataManager, protected boolean isHSync() { return isHSync; } + + public OmKeyInfo getNewOpenKeyInfo() { + return newOpenKeyInfo; + } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponseWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponseWithFSO.java index 552d93861de5..13034f77dfb9 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponseWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponseWithFSO.java @@ -47,16 +47,17 @@ public class OMKeyCommitResponseWithFSO extends OMKeyCommitResponse { private long volumeId; - @SuppressWarnings("parameternumber") + @SuppressWarnings("checkstyle:ParameterNumber") public OMKeyCommitResponseWithFSO( @Nonnull OMResponse omResponse, @Nonnull OmKeyInfo omKeyInfo, String ozoneKeyName, String openKeyName, @Nonnull OmBucketInfo omBucketInfo, Map deleteKeyMap, long volumeId, - boolean isHSync) { + boolean isHSync, + OmKeyInfo newOpenKeyInfo) { super(omResponse, omKeyInfo, ozoneKeyName, openKeyName, - omBucketInfo, deleteKeyMap, isHSync); + omBucketInfo, deleteKeyMap, isHSync, newOpenKeyInfo); this.volumeId = volumeId; } @@ -77,18 +78,21 @@ public void addToDBBatch(OMMetadataManager omMetadataManager, // Delete from OpenKey table if commit if (!this.isHSync()) { omMetadataManager.getOpenKeyTable(getBucketLayout()) - .deleteWithBatch(batchOperation, getOpenKeyName()); + .deleteWithBatch(batchOperation, getOpenKeyName()); + } else if (getNewOpenKeyInfo() != null) { + omMetadataManager.getOpenKeyTable(getBucketLayout()).putWithBatch( + batchOperation, getOpenKeyName(), getNewOpenKeyInfo()); } OMFileRequest.addToFileTable(omMetadataManager, batchOperation, - getOmKeyInfo(), volumeId, getOmBucketInfo().getObjectID()); + getOmKeyInfo(), volumeId, getOmBucketInfo().getObjectID()); updateDeletedTable(omMetadataManager, batchOperation); // update bucket usedBytes. omMetadataManager.getBucketTable().putWithBatch(batchOperation, - omMetadataManager.getBucketKey(getOmBucketInfo().getVolumeName(), - getOmBucketInfo().getBucketName()), getOmBucketInfo()); + omMetadataManager.getBucketKey(getOmBucketInfo().getVolumeName(), + getOmBucketInfo().getBucketName()), getOmBucketInfo()); } @Override diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OpenKeyCleanupService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OpenKeyCleanupService.java index 0f2c1f84a6ca..ab5562301940 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OpenKeyCleanupService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/OpenKeyCleanupService.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.utils.BackgroundTask; import org.apache.hadoop.hdds.utils.BackgroundTaskQueue; import org.apache.hadoop.hdds.utils.BackgroundTaskResult; +import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.om.ExpiredOpenKeys; import org.apache.hadoop.ozone.om.KeyManager; import org.apache.hadoop.ozone.om.OMConfigKeys; @@ -76,6 +77,7 @@ public class OpenKeyCleanupService extends BackgroundService { // service, not the client. private final ClientId clientId = ClientId.randomId(); private final Duration expireThreshold; + private final Duration leaseThreshold; private final int cleanupLimitPerTask; private final AtomicLong submittedOpenKeyCount; private final AtomicLong runCount; @@ -96,6 +98,18 @@ public OpenKeyCleanupService(long interval, TimeUnit unit, long timeout, TimeUnit.MILLISECONDS); this.expireThreshold = Duration.ofMillis(expireMillis); + long leaseHardMillis = conf.getTimeDuration(OMConfigKeys.OZONE_OM_LEASE_HARD_LIMIT, + OMConfigKeys.OZONE_OM_LEASE_HARD_LIMIT_DEFAULT, TimeUnit.MILLISECONDS); + long leaseSoftMillis = conf.getTimeDuration(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, + OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT_DEFAULT, TimeUnit.MILLISECONDS); + + if (leaseHardMillis < leaseSoftMillis) { + String msg = "Hard lease limit cannot be less than Soft lease limit. " + + "LeaseHardLimit: " + leaseHardMillis + " LeaseSoftLimit: " + leaseSoftMillis; + throw new IllegalArgumentException(msg); + } + this.leaseThreshold = Duration.ofMillis(leaseHardMillis); + this.cleanupLimitPerTask = conf.getInt( OMConfigKeys.OZONE_OM_OPEN_KEY_CLEANUP_LIMIT_PER_TASK, OMConfigKeys.OZONE_OM_OPEN_KEY_CLEANUP_LIMIT_PER_TASK_DEFAULT); @@ -178,13 +192,12 @@ public BackgroundTaskResult call() throws Exception { if (!shouldRun()) { return BackgroundTaskResult.EmptyTaskResult.newResult(); } - runCount.incrementAndGet(); long startTime = Time.monotonicNow(); final ExpiredOpenKeys expiredOpenKeys; try { expiredOpenKeys = keyManager.getExpiredOpenKeys(expireThreshold, - cleanupLimitPerTask, bucketLayout); + cleanupLimitPerTask, bucketLayout, leaseThreshold); } catch (IOException e) { LOG.error("Unable to get hanging open keys, retry in next interval", e); return BackgroundTaskResult.EmptyTaskResult.newResult(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java index 5acb9f365107..5339c7400d30 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java @@ -51,6 +51,7 @@ import org.apache.hadoop.ozone.om.helpers.ListKeysResult; import org.apache.hadoop.ozone.om.helpers.DBUpdates; import org.apache.hadoop.ozone.om.helpers.KeyInfoWithVolumeContext; +import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; @@ -58,6 +59,7 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts; import org.apache.hadoop.ozone.om.helpers.OmPartInfo; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus; import org.apache.hadoop.ozone.om.helpers.OzoneFileStatusLight; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; @@ -80,6 +82,8 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CancelSnapshotDiffRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CancelSnapshotDiffResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListOpenFilesRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListOpenFilesResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListSnapshotDiffJobRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListSnapshotDiffJobResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CheckVolumeAccessRequest; @@ -245,6 +249,11 @@ public OMResponse handleReadRequest(OMRequest request) { listMultipartUploads(request.getListMultipartUploadsRequest()); responseBuilder.setListMultipartUploadsResponse(response); break; + case ListOpenFiles: + ListOpenFilesResponse listOpenFilesResponse = listOpenFiles( + request.getListOpenFilesRequest(), request.getVersion()); + responseBuilder.setListOpenFilesResponse(listOpenFilesResponse); + break; case ServiceList: ServiceListResponse serviceListResponse = getServiceList( request.getServiceListRequest()); @@ -904,6 +913,31 @@ public static OMResponse disallowListTrashWithBucketLayout( return resp; } + private ListOpenFilesResponse listOpenFiles(ListOpenFilesRequest req, + int clientVersion) + throws IOException { + ListOpenFilesResponse.Builder resp = ListOpenFilesResponse.newBuilder(); + + ListOpenFilesResult res = + impl.listOpenFiles(req.getPath(), req.getCount(), req.getToken()); + // TODO: Is there a clean way to avoid ser-de for responses: + // OM does: ListOpenFilesResult -> ListOpenFilesResponse + // Client : ListOpenFilesResponse -> ListOpenFilesResult + + resp.setTotalOpenKeyCount(res.getTotalOpenKeyCount()); + resp.setHasMore(res.hasMore()); + if (res.getContinuationToken() != null) { + resp.setContinuationToken(res.getContinuationToken()); + } + + for (OpenKeySession e : res.getOpenKeys()) { + resp.addClientID(e.getId()); + resp.addKeyInfo(e.getKeyInfo().getProtobuf(clientVersion)); + } + + return resp.build(); + } + private ServiceListResponse getServiceList(ServiceListRequest request) throws IOException { ServiceListResponse.Builder resp = ServiceListResponse.newBuilder(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java index e0d5ef4084da..33a33ad807d4 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java @@ -96,7 +96,7 @@ private List createInputStreams(String dataString) { private BlockInputStream createStream(byte[] buf, int offset) { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setChecksumVerify(true); - return new BlockInputStream(null, 100, null, null, null, + return new BlockInputStream(null, 100L, null, null, null, clientConfig) { private long pos; private final ByteArrayInputStream in = diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java index 1e2fb6a60a80..c4913879ae90 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java @@ -25,12 +25,14 @@ import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.hdds.utils.TransactionInfo; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; @@ -46,6 +48,7 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import java.io.File; @@ -562,36 +565,120 @@ public void testListKeysWithFewDeleteEntriesInCache() throws Exception { } - private static BucketLayout getDefaultBucketLayout() { - return BucketLayout.DEFAULT; - } + /** + * Tests inner impl of listOpenFiles with different bucket types with and + * without pagination. NOTE: This UT does NOT test hsync here since the hsync + * status check is done purely on the client side. + * @param bucketLayout BucketLayout + */ + @ParameterizedTest + @EnumSource + public void testListOpenFiles(BucketLayout bucketLayout) throws Exception { + final long clientID = 1000L; - @Test - public void testGetExpiredOpenKeys() throws Exception { - testGetExpiredOpenKeys(BucketLayout.DEFAULT); - } + String volumeName = "volume-lof"; + String bucketName = "bucket-" + bucketLayout.name().toLowerCase(); + String keyPrefix = "key"; - @Test - public void testGetExpiredOpenKeysExcludeMPUs() throws Exception { - testGetExpiredOpenKeysExcludeMPUKeys(BucketLayout.DEFAULT); - } + OMRequestTestUtils.addVolumeAndBucketToDB(volumeName, bucketName, + omMetadataManager, bucketLayout); - @Test - public void testGetExpiredOpenKeysFSO() throws Exception { - testGetExpiredOpenKeys(BucketLayout.FILE_SYSTEM_OPTIMIZED); - } + long volumeId = -1L, bucketId = -1L; + if (bucketLayout.isFileSystemOptimized()) { + volumeId = omMetadataManager.getVolumeId(volumeName); + bucketId = omMetadataManager.getBucketId(volumeName, bucketName); + } - @Test - public void testGetExpiredOpenKeysExcludeMPUsFSO() throws Exception { - testGetExpiredOpenKeysExcludeMPUKeys(BucketLayout.FILE_SYSTEM_OPTIMIZED); + int numOpenKeys = 3; + List openKeys = new ArrayList<>(); + for (int i = 0; i < numOpenKeys; i++) { + final OmKeyInfo keyInfo = OMRequestTestUtils.createOmKeyInfo(volumeName, bucketName, keyPrefix + i, + RatisReplicationConfig.getInstance(ONE)) + .build(); + + final String dbOpenKeyName; + if (bucketLayout.isFileSystemOptimized()) { + keyInfo.setParentObjectID(i); + keyInfo.setFileName(OzoneFSUtils.getFileName(keyInfo.getKeyName())); + OMRequestTestUtils.addFileToKeyTable(true, false, + keyInfo.getFileName(), keyInfo, clientID, 0L, omMetadataManager); + dbOpenKeyName = omMetadataManager.getOpenFileName(volumeId, bucketId, + keyInfo.getParentObjectID(), keyInfo.getFileName(), clientID); + } else { + OMRequestTestUtils.addKeyToTable(true, false, + keyInfo, clientID, 0L, omMetadataManager); + dbOpenKeyName = omMetadataManager.getOpenKey(volumeName, bucketName, + keyInfo.getKeyName(), clientID); + } + openKeys.add(dbOpenKeyName); + } + + String dbPrefix; + if (bucketLayout.isFileSystemOptimized()) { + dbPrefix = omMetadataManager.getOzoneKeyFSO(volumeName, bucketName, ""); + } else { + dbPrefix = omMetadataManager.getOzoneKey(volumeName, bucketName, ""); + } + + // Without pagination + ListOpenFilesResult res = omMetadataManager.listOpenFiles( + bucketLayout, 100, dbPrefix, false, dbPrefix); + + assertEquals(numOpenKeys, res.getTotalOpenKeyCount()); + assertEquals(false, res.hasMore()); + List keySessionList = res.getOpenKeys(); + assertEquals(numOpenKeys, keySessionList.size()); + // Verify that every single open key shows up in the result, and in order + for (int i = 0; i < numOpenKeys; i++) { + OpenKeySession keySession = keySessionList.get(i); + assertEquals(keyPrefix + i, keySession.getKeyInfo().getKeyName()); + assertEquals(clientID, keySession.getId()); + assertEquals(0, keySession.getOpenVersion()); + } + + // With pagination + int pageSize = 2; + int numExpectedKeys = pageSize; + res = omMetadataManager.listOpenFiles( + bucketLayout, pageSize, dbPrefix, false, dbPrefix); + // total open key count should still be 3 + assertEquals(numOpenKeys, res.getTotalOpenKeyCount()); + // hasMore should have been set + assertEquals(true, res.hasMore()); + keySessionList = res.getOpenKeys(); + assertEquals(numExpectedKeys, keySessionList.size()); + for (int i = 0; i < numExpectedKeys; i++) { + OpenKeySession keySession = keySessionList.get(i); + assertEquals(keyPrefix + i, keySession.getKeyInfo().getKeyName()); + assertEquals(clientID, keySession.getId()); + assertEquals(0, keySession.getOpenVersion()); + } + + // Get the second page + res = omMetadataManager.listOpenFiles( + bucketLayout, pageSize, dbPrefix, true, res.getContinuationToken()); + numExpectedKeys = numOpenKeys - pageSize; + // total open key count should still be 3 + assertEquals(numOpenKeys, res.getTotalOpenKeyCount()); + assertEquals(false, res.hasMore()); + keySessionList = res.getOpenKeys(); + assertEquals(numExpectedKeys, keySessionList.size()); + for (int i = 0; i < numExpectedKeys; i++) { + OpenKeySession keySession = keySessionList.get(i); + assertEquals(keyPrefix + (pageSize + i), + keySession.getKeyInfo().getKeyName()); + assertEquals(clientID, keySession.getId()); + assertEquals(0, keySession.getOpenVersion()); + } } - @Test - public void testGetExpiredMultipartUploads() throws Exception { - testGetExpiredMPUs(); + private static BucketLayout getDefaultBucketLayout() { + return BucketLayout.DEFAULT; } - private void testGetExpiredOpenKeys(BucketLayout bucketLayout) + @ParameterizedTest + @EnumSource + public void testGetExpiredOpenKeys(BucketLayout bucketLayout) throws Exception { final String bucketName = UUID.randomUUID().toString(); final String volumeName = UUID.randomUUID().toString(); @@ -648,7 +735,7 @@ private void testGetExpiredOpenKeys(BucketLayout bucketLayout) // Test retrieving fewer expired keys than actually exist. final Collection someExpiredKeys = omMetadataManager.getExpiredOpenKeys(expireThreshold, - numExpiredOpenKeys - 1, bucketLayout).getOpenKeyBuckets(); + numExpiredOpenKeys - 1, bucketLayout, expireThreshold).getOpenKeyBuckets(); List names = getOpenKeyNames(someExpiredKeys); assertEquals(numExpiredOpenKeys - 1, names.size()); assertThat(expiredKeys).containsAll(names); @@ -656,7 +743,7 @@ private void testGetExpiredOpenKeys(BucketLayout bucketLayout) // Test attempting to retrieving more expired keys than actually exist. Collection allExpiredKeys = omMetadataManager.getExpiredOpenKeys(expireThreshold, - numExpiredOpenKeys + 1, bucketLayout).getOpenKeyBuckets(); + numExpiredOpenKeys + 1, bucketLayout, expireThreshold).getOpenKeyBuckets(); names = getOpenKeyNames(allExpiredKeys); assertEquals(numExpiredOpenKeys, names.size()); assertThat(expiredKeys).containsAll(names); @@ -664,13 +751,15 @@ private void testGetExpiredOpenKeys(BucketLayout bucketLayout) // Test retrieving exact amount of expired keys that exist. allExpiredKeys = omMetadataManager.getExpiredOpenKeys(expireThreshold, - numExpiredOpenKeys, bucketLayout).getOpenKeyBuckets(); + numExpiredOpenKeys, bucketLayout, expireThreshold).getOpenKeyBuckets(); names = getOpenKeyNames(allExpiredKeys); assertEquals(numExpiredOpenKeys, names.size()); assertThat(expiredKeys).containsAll(names); } - private void testGetExpiredOpenKeysExcludeMPUKeys( + @ParameterizedTest + @EnumSource + public void testGetExpiredOpenKeysExcludeMPUKeys( BucketLayout bucketLayout) throws Exception { final String bucketName = UUID.randomUUID().toString(); final String volumeName = UUID.randomUUID().toString(); @@ -721,7 +810,7 @@ private void testGetExpiredOpenKeysExcludeMPUKeys( // Return empty since only MPU-related open keys exist. assertTrue(omMetadataManager.getExpiredOpenKeys(expireThreshold, - numExpiredMPUOpenKeys, bucketLayout).getOpenKeyBuckets().isEmpty()); + numExpiredMPUOpenKeys, bucketLayout, expireThreshold).getOpenKeyBuckets().isEmpty()); // This is for MPU-related open keys prior to isMultipartKey fix in @@ -755,11 +844,12 @@ private void testGetExpiredOpenKeysExcludeMPUKeys( // MPU-related open keys should not be fetched regardless of isMultipartKey // flag if has the multipart upload characteristics assertTrue(omMetadataManager.getExpiredOpenKeys(expireThreshold, - numExpiredMPUOpenKeys, bucketLayout).getOpenKeyBuckets() + numExpiredMPUOpenKeys, bucketLayout, expireThreshold).getOpenKeyBuckets() .isEmpty()); } - private void testGetExpiredMPUs() throws Exception { + @Test + public void testGetExpiredMPUs() throws Exception { final String bucketName = UUID.randomUUID().toString(); final String volumeName = UUID.randomUUID().toString(); final int numExpiredMPUs = 4; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java index 0ebd6946bd29..86263683682a 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java @@ -1512,7 +1512,7 @@ public static long addParentsToDirTable(String volumeName, String bucketName, OMRequestTestUtils.createOmDirectoryInfo(pathElement, ++objectId, parentId); OMRequestTestUtils.addDirKeyToDirTable(true, omDirInfo, - volumeName, bucketName, txnID, omMetaMgr); + volumeName, bucketName, ++txnID, omMetaMgr); parentId = omDirInfo.getObjectID(); } return parentId; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMRecoverLeaseRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMRecoverLeaseRequest.java index 294281555a56..59d3e211efdb 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMRecoverLeaseRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMRecoverLeaseRequest.java @@ -18,40 +18,49 @@ package org.apache.hadoop.ozone.om.request.file; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.ozone.ClientVersion; +import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.KeyValueUtil; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils; import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; +import org.apache.hadoop.ozone.om.request.key.OMAllocateBlockRequestWithFSO; +import org.apache.hadoop.ozone.om.request.key.OMKeyCommitRequestWithFSO; import org.apache.hadoop.ozone.om.request.key.TestOMKeyRequest; import org.apache.hadoop.ozone.om.response.OMClientResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .KeyLocation; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .RecoverLeaseRequest; -import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; -import org.apache.hadoop.ozone.security.acl.OzoneObj; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RecoverLeaseRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RecoverLeaseResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CommitKeyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AllocateBlockRequest; import jakarta.annotation.Nonnull; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; -import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; import java.util.UUID; import java.util.stream.Collectors; +import static org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB.setReplicationConfig; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyString; -import static org.mockito.Mockito.when; /** * Tests OMRecoverLeaseRequest. @@ -59,6 +68,7 @@ public class TestOMRecoverLeaseRequest extends TestOMKeyRequest { private long parentId; + private boolean forceRecovery = false; @Override public BucketLayout getBucketLayout() { @@ -72,34 +82,190 @@ public BucketLayout getBucketLayout() { */ @Test public void testRecoverHsyncFile() throws Exception { - when(ozoneManager.getAclsEnabled()).thenReturn(true); - when(ozoneManager.getVolumeOwner( - anyString(), - any(IAccessAuthorizer.ACLType.class), any( - OzoneObj.ResourceType.class))) - .thenReturn("user"); - InetSocketAddress address = new InetSocketAddress("localhost", 10000); - when(ozoneManager.getOmRpcServerAddr()).thenReturn(address); - populateNamespace(true, true); + populateNamespace(true, true, true, true); OMClientResponse omClientResponse = validateAndUpdateCache(); + OMResponse omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + RecoverLeaseResponse recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + OmKeyInfo omKeyInfo = OmKeyInfo.getFromProtobuf(keyInfo); - assertEquals(OzoneManagerProtocolProtos.Status.OK, + omClientResponse = validateAndUpdateCacheForCommit(getNewKeyArgs(omKeyInfo, 0)); + omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + + verifyTables(true, false); + } + + /** + * Verify that RecoverLease request is idempotent. + * @throws Exception + */ + @Test + public void testInitStageIdempotent() throws Exception { + populateNamespace(true, true, true, true); + + // call recovery first time + OMClientResponse omClientResponse = validateAndUpdateCache(); + OMResponse omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + RecoverLeaseResponse recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo1 = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo1); + + // call recovery second time + omClientResponse = validateAndUpdateCache(); + omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo2 = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo2); + assertEquals(keyInfo1.getKeyName(), keyInfo2.getKeyName()); + } + + /** + * Verify that COMMIT request for recovery is not idempotent. + * @throws Exception + */ + @Test + public void testCommitStageNotIdempotent() throws Exception { + populateNamespace(true, true, true, true); + + // call recovery + OMClientResponse omClientResponse = validateAndUpdateCache(); + OMResponse omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + RecoverLeaseResponse recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + OmKeyInfo omKeyInfo = OmKeyInfo.getFromProtobuf(keyInfo); + + KeyArgs newKeyArgs = getNewKeyArgs(omKeyInfo, 0); + + // call commit first time + omClientResponse = validateAndUpdateCacheForCommit(newKeyArgs); + omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + + // call commit second time + omClientResponse = validateAndUpdateCacheForCommit(newKeyArgs); + omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.KEY_ALREADY_CLOSED, omResponse.getStatus()); + } + + /** + * Verify that RecoverLease COMMIT request has a new file length. + * @throws Exception + */ + @Test + public void testRecoverWithNewFileLength() throws Exception { + populateNamespace(true, true, true, true); + + // call recovery + OMClientResponse omClientResponse = validateAndUpdateCache(); + OMResponse omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + RecoverLeaseResponse recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + OmKeyInfo omKeyInfo = OmKeyInfo.getFromProtobuf(keyInfo); + + // call commit + long deltaLength = 100; + KeyArgs newKeyArgs = getNewKeyArgs(omKeyInfo, deltaLength); + omClientResponse = validateAndUpdateCacheForCommit(newKeyArgs); + omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + + // get file length and check the length is as expected + String ozoneKey = getFileName(); + OmKeyInfo omKeyInfoFetched = omMetadataManager.getKeyTable(getBucketLayout()).get(ozoneKey); + assertEquals(omKeyInfo.getDataSize(), omKeyInfoFetched.getDataSize()); + + // check the final block length is as expected + List locationInfoListFetched = + omKeyInfoFetched.getLatestVersionLocations().getBlocksLatestVersionOnly(); + List omKeyLocationInfos = omKeyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly(); + assertEquals(omKeyLocationInfos.get(omKeyLocationInfos.size() - 1).getLength(), + locationInfoListFetched.get(locationInfoListFetched.size() - 1).getLength()); + + // check the committed file doesn't have HSYNC_CLIENT_ID and LEASE_RECOVERY metadata + assertNull(omKeyInfoFetched.getMetadata().get(OzoneConsts.HSYNC_CLIENT_ID)); + assertNull(omKeyInfoFetched.getMetadata().get(OzoneConsts.LEASE_RECOVERY)); + } + + /** + * Verify that RecoverLease COMMIT request has a new client ID. + * @throws Exception + */ + @Test + public void testRecoverWithNewClientID() throws Exception { + populateNamespace(true, true, true, true); + + // call recovery + OMClientResponse omClientResponse = validateAndUpdateCache(); + OMResponse omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + RecoverLeaseResponse recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + OmKeyInfo omKeyInfo = OmKeyInfo.getFromProtobuf(keyInfo); + + // call commit + KeyArgs newKeyArgs = getNewKeyArgs(omKeyInfo, 0); + omClientResponse = validateAndUpdateCacheForCommit(newKeyArgs, true, true); + omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + } + + /** + * Verify that an under recovery file will reject allocate block and further hsync call(commit). + * @throws Exception + */ + @Test + public void testRejectAllocateBlockAndHsync() throws Exception { + populateNamespace(true, true, true, true); + + // call recovery + OMClientResponse omClientResponse = validateAndUpdateCache(); + OMResponse omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + RecoverLeaseResponse recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + OmKeyInfo omKeyInfo = OmKeyInfo.getFromProtobuf(keyInfo); + + // call allocate block + OMRequest request = createAllocateBlockRequest(volumeName, bucketName, keyName); + OMAllocateBlockRequestWithFSO omAllocateBlockRequest = + new OMAllocateBlockRequestWithFSO(request, getBucketLayout()); + request = omAllocateBlockRequest.preExecute(ozoneManager); + assertNotNull(request.getUserInfo()); + omAllocateBlockRequest = new OMAllocateBlockRequestWithFSO(request, getBucketLayout()); + omClientResponse = omAllocateBlockRequest.validateAndUpdateCache( + ozoneManager, 100L); + assertEquals(OzoneManagerProtocolProtos.Status.KEY_UNDER_LEASE_RECOVERY, omClientResponse.getOMResponse().getStatus()); - verifyTables(true, true); + // call commit(hsync calls commit) + KeyArgs newKeyArgs = getNewKeyArgs(omKeyInfo, 0); + omClientResponse = validateAndUpdateCacheForCommit(newKeyArgs, false, false); + assertEquals(OzoneManagerProtocolProtos.Status.KEY_UNDER_LEASE_RECOVERY, + omClientResponse.getOMResponse().getStatus()); } /** - * verify that recover a closed file should be allowed (essentially no-op). - */ + * verify that recover a closed file. + **/ @Test public void testRecoverClosedFile() throws Exception { - populateNamespace(true, false); + populateNamespace(true, false, false, false); OMClientResponse omClientResponse = validateAndUpdateCache(); - assertEquals(OzoneManagerProtocolProtos.Status.OK, + assertEquals(OzoneManagerProtocolProtos.Status.KEY_ALREADY_CLOSED, omClientResponse.getOMResponse().getStatus()); verifyTables(true, false); @@ -110,7 +276,7 @@ public void testRecoverClosedFile() throws Exception { */ @Test public void testRecoverOpenFile() throws Exception { - populateNamespace(false, true); + populateNamespace(false, false, true, false); OMClientResponse omClientResponse = validateAndUpdateCache(); @@ -126,7 +292,7 @@ public void testRecoverOpenFile() throws Exception { */ @Test public void testRecoverAbsentFile() throws Exception { - populateNamespace(false, false); + populateNamespace(false, false, false, false); OMClientResponse omClientResponse = validateAndUpdateCache(); @@ -136,8 +302,91 @@ public void testRecoverAbsentFile() throws Exception { verifyTables(false, false); } - private void populateNamespace(boolean addKeyTable, boolean addOpenKeyTable) - throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testLeaseSoftLimitForHsyncRecoverFile(boolean force) throws Exception { + forceRecovery = force; + populateNamespace(true, true, true, true); + + // update soft limit to high value + ozoneManager.getConfiguration().set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "2s"); + OMClientResponse omClientResponse = validateAndUpdateCache(); + OMResponse omResponse = omClientResponse.getOMResponse(); + RecoverLeaseResponse recoverLeaseResponse; + if (force) { + // In case of force it should always succeed irrespective of soft limit value. + assertEquals(OzoneManagerProtocolProtos.Status.OK, + omResponse.getStatus()); + recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + } else { + // Call recovery inside soft limit period it should fail + assertEquals(OzoneManagerProtocolProtos.Status.KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD, + omResponse.getStatus()); + } + omClientResponse = validateAndUpdateCache(); + omResponse = omClientResponse.getOMResponse(); + if (force) { + assertEquals(OzoneManagerProtocolProtos.Status.OK, + omResponse.getStatus()); + recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + } else { + // Call second time inside soft limit period also should fail + assertEquals(OzoneManagerProtocolProtos.Status.KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD, + omResponse.getStatus()); + } + Thread.sleep(2000); + // Call recovery after soft limit period it should succeed + omClientResponse = validateAndUpdateCache(); + omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + KeyInfo keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + + // Call recovery again it should succeed + omClientResponse = validateAndUpdateCache(); + omResponse = omClientResponse.getOMResponse(); + assertEquals(OzoneManagerProtocolProtos.Status.OK, omResponse.getStatus()); + recoverLeaseResponse = omResponse.getRecoverLeaseResponse(); + keyInfo = recoverLeaseResponse.getKeyInfo(); + assertNotNull(keyInfo); + } + + private KeyArgs getNewKeyArgs(OmKeyInfo omKeyInfo, long deltaLength) throws IOException { + OmKeyLocationInfoGroup omKeyLocationInfoGroup = omKeyInfo.getLatestVersionLocations(); + List omKeyLocationInfoList = omKeyLocationInfoGroup.getBlocksLatestVersionOnly(); + long lastBlockLength = omKeyLocationInfoList.get(omKeyLocationInfoList.size() - 1).getLength(); + omKeyLocationInfoList.get(omKeyLocationInfoList.size() - 1).setLength(lastBlockLength + deltaLength); + + long fileLength = omKeyLocationInfoList.stream().mapToLong(OmKeyLocationInfo::getLength).sum(); + omKeyInfo.setDataSize(fileLength); + OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(omKeyInfo.getVolumeName()) + .setBucketName(omKeyInfo.getBucketName()).setKeyName(omKeyInfo.getKeyName()) + .setReplicationConfig(omKeyInfo.getReplicationConfig()).setDataSize(fileLength) + .setLocationInfoList(omKeyLocationInfoList).setLatestVersionLocation(true) + .build(); + + List locationInfoList = keyArgs.getLocationInfoList(); + Preconditions.checkNotNull(locationInfoList); + KeyArgs.Builder keyArgsBuilder = KeyArgs.newBuilder() + .setVolumeName(keyArgs.getVolumeName()) + .setBucketName(keyArgs.getBucketName()) + .setKeyName(keyArgs.getKeyName()) + .setDataSize(keyArgs.getDataSize()) + .addAllMetadata(KeyValueUtil.toProtobuf(keyArgs.getMetadata())) + .addAllKeyLocations(locationInfoList.stream() + .map(info -> info.getProtobuf(ClientVersion.CURRENT_VERSION)) + .collect(Collectors.toList())); + setReplicationConfig(keyArgs.getReplicationConfig(), keyArgsBuilder); + return keyArgsBuilder.build(); + } + + private void populateNamespace(boolean addKeyTable, boolean keyInfoWithHsyncFlag, + boolean addOpenKeyTable, boolean openKeyInfoWithHsyncFlag) throws Exception { String parentDir = "c/d/e"; String fileName = "f"; keyName = parentDir + "/" + fileName; @@ -152,40 +401,55 @@ private void populateNamespace(boolean addKeyTable, boolean addOpenKeyTable) OmKeyInfo omKeyInfo; if (addKeyTable) { - String ozoneKey = addToFileTable(allocatedLocationList); + String ozoneKey = addToFileTable(allocatedLocationList, keyInfoWithHsyncFlag); omKeyInfo = omMetadataManager.getKeyTable(getBucketLayout()) .get(ozoneKey); assertNotNull(omKeyInfo); } if (addOpenKeyTable) { - String openKey = addToOpenFileTable(allocatedLocationList); + String openKey = addToOpenFileTable(allocatedLocationList, openKeyInfoWithHsyncFlag); omKeyInfo = omMetadataManager.getOpenKeyTable(getBucketLayout()) .get(openKey); assertNotNull(omKeyInfo); } + + // Set lease soft limit to 0 + ozoneManager.getConfiguration().set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "0s"); } - @Nonnull - protected OMRequest createRecoverLeaseRequest( - String volumeName, String bucketName, String keyName) { + protected OMRequest createAllocateBlockRequest(String volumeName, String bucketName, String keyName) { + KeyArgs keyArgs = KeyArgs.newBuilder() + .setVolumeName(volumeName).setBucketName(bucketName) + .setKeyName(keyName) + .setFactor(((RatisReplicationConfig) replicationConfig).getReplicationFactor()) + .setType(replicationConfig.getReplicationType()) + .build(); - RecoverLeaseRequest recoverLeaseRequest = RecoverLeaseRequest.newBuilder() - .setVolumeName(volumeName) - .setBucketName(bucketName) - .setKeyName(keyName).build(); + AllocateBlockRequest allocateBlockRequest = + AllocateBlockRequest.newBuilder().setClientID(clientID).setKeyArgs(keyArgs).build(); + return OMRequest.newBuilder() + .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock) + .setClientId(UUID.randomUUID().toString()) + .setAllocateBlockRequest(allocateBlockRequest).build(); + } + + @Nonnull + protected OMRequest createRecoverLeaseRequest( + String volumeName, String bucketName, String keyName, boolean force) { + RecoverLeaseRequest.Builder rb = RecoverLeaseRequest.newBuilder(); + rb.setVolumeName(volumeName).setBucketName(bucketName).setKeyName(keyName).setForce(force); return OMRequest.newBuilder() .setCmdType(OzoneManagerProtocolProtos.Type.RecoverLease) .setClientId(UUID.randomUUID().toString()) - .setRecoverLeaseRequest(recoverLeaseRequest).build(); + .setRecoverLeaseRequest(rb.build()).build(); } - private OMClientResponse validateAndUpdateCache() throws Exception { OMRequest modifiedOmRequest = doPreExecute(createRecoverLeaseRequest( - volumeName, bucketName, keyName)); + volumeName, bucketName, keyName, forceRecovery)); assertNotNull(modifiedOmRequest.getUserInfo()); OMRecoverLeaseRequest omRecoverLeaseRequest = getOmRecoverLeaseRequest( @@ -196,6 +460,34 @@ private OMClientResponse validateAndUpdateCache() throws Exception { return omClientResponse; } + @Nonnull + protected OMRequest createKeyCommitRequest(KeyArgs keyArgs, boolean newClientID, boolean recovery) { + CommitKeyRequest.Builder rb = + CommitKeyRequest.newBuilder().setKeyArgs(keyArgs).setRecovery(recovery); + rb.setClientID(newClientID ? clientID + 1 : clientID); + return OMRequest.newBuilder() + .setCmdType(OzoneManagerProtocolProtos.Type.CommitKey) + .setClientId(UUID.randomUUID().toString()) + .setCommitKeyRequest(rb.build()).build(); + } + + private OMClientResponse validateAndUpdateCacheForCommit(KeyArgs keyArgs) throws Exception { + return validateAndUpdateCacheForCommit(keyArgs, false, true); + } + + private OMClientResponse validateAndUpdateCacheForCommit(KeyArgs keyArgs, boolean newClientID, + boolean recovery) throws Exception { + OMRequest omRequest = createKeyCommitRequest(keyArgs, newClientID, recovery); + OMKeyCommitRequestWithFSO omKeyCommitRequest = new OMKeyCommitRequestWithFSO(omRequest, getBucketLayout()); + OMRequest modifiedOmRequest = omKeyCommitRequest.preExecute(ozoneManager); + assertNotNull(modifiedOmRequest.getUserInfo()); + + omKeyCommitRequest = new OMKeyCommitRequestWithFSO(modifiedOmRequest, getBucketLayout()); + OMClientResponse omClientResponse = + omKeyCommitRequest.validateAndUpdateCache(ozoneManager, 100L); + return omClientResponse; + } + private void verifyTables(boolean hasKey, boolean hasOpenKey) throws IOException { // Now entry should be created in key Table. @@ -238,8 +530,7 @@ String getFileName() throws IOException { fileName); } - protected OMRecoverLeaseRequest getOmRecoverLeaseRequest( - OMRequest omRequest) { + protected OMRecoverLeaseRequest getOmRecoverLeaseRequest(OMRequest omRequest) { return new OMRecoverLeaseRequest(omRequest); } @@ -269,15 +560,17 @@ private OMRequest doPreExecute(OMRequest originalOMRequest) throws Exception { return modifiedOmRequest; } - String addToOpenFileTable(List locationList) + String addToOpenFileTable(List locationList, boolean hsyncFlag) throws Exception { OmKeyInfo omKeyInfo = OMRequestTestUtils.createOmKeyInfo(volumeName, bucketName, keyName, replicationConfig, new OmKeyLocationInfoGroup(version, new ArrayList<>(), false)) .setParentObjectID(parentId) .build(); omKeyInfo.appendNewBlocks(locationList, false); - omKeyInfo.getMetadata().put(OzoneConsts.HSYNC_CLIENT_ID, - String.valueOf(clientID)); + if (hsyncFlag) { + omKeyInfo.getMetadata().put(OzoneConsts.HSYNC_CLIENT_ID, + String.valueOf(clientID)); + } OMRequestTestUtils.addFileToKeyTable( true, false, omKeyInfo.getFileName(), @@ -292,13 +585,17 @@ bucketName, keyName, replicationConfig, new OmKeyLocationInfoGroup(version, new omKeyInfo.getParentObjectID(), omKeyInfo.getFileName(), clientID); } - String addToFileTable(List locationList) + String addToFileTable(List locationList, boolean hsyncFlag) throws Exception { OmKeyInfo omKeyInfo = OMRequestTestUtils.createOmKeyInfo(volumeName, bucketName, keyName, replicationConfig, new OmKeyLocationInfoGroup(version, new ArrayList<>(), false)) .setParentObjectID(parentId) .build(); omKeyInfo.appendNewBlocks(locationList, false); + if (hsyncFlag) { + omKeyInfo.getMetadata().put(OzoneConsts.HSYNC_CLIENT_ID, + String.valueOf(clientID)); + } OMRequestTestUtils.addFileToKeyTable( false, false, omKeyInfo.getFileName(), diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java index 9719865db196..1c38287f55e8 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java @@ -49,14 +49,10 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.response.OMClientResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .CommitKeyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .KeyArgs; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .KeyLocation; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .OMRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CommitKeyRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java index fde83d7b7697..e5c3f19b8506 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java @@ -27,7 +27,10 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; +import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OMPerformanceMetrics; @@ -70,6 +73,7 @@ import org.apache.hadoop.ozone.om.ScmClient; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenSecretManager; import org.apache.hadoop.util.Time; +import org.mockito.Mockito; import org.slf4j.event.Level; import static org.apache.hadoop.ozone.om.request.OMRequestTestUtils.setupReplicationConfigValidation; @@ -101,6 +105,7 @@ public class TestOMKeyRequest { protected ScmClient scmClient; protected OzoneBlockTokenSecretManager ozoneBlockTokenSecretManager; protected ScmBlockLocationProtocol scmBlockLocationProtocol; + protected StorageContainerLocationProtocol scmContainerLocationProtocol; protected OMPerformanceMetrics metrics; protected static final long CONTAINER_ID = 1000L; @@ -163,6 +168,9 @@ public void setup() throws Exception { when(ozoneManager.getOMServiceId()).thenReturn( UUID.randomUUID().toString()); when(scmClient.getBlockClient()).thenReturn(scmBlockLocationProtocol); + scmContainerLocationProtocol = Mockito.mock(StorageContainerLocationProtocol.class); + when(scmClient.getContainerClient()).thenReturn(scmContainerLocationProtocol); + when(ozoneManager.getKeyManager()).thenReturn(keyManager); when(ozoneManager.getAccessAuthorizer()) .thenReturn(new OzoneNativeAuthorizer()); @@ -203,6 +211,9 @@ public void setup() throws Exception { return allocatedBlocks; }); + ContainerWithPipeline containerWithPipeline = + new ContainerWithPipeline(Mockito.mock(ContainerInfo.class), pipeline); + when(scmContainerLocationProtocol.getContainerWithPipeline(anyLong())).thenReturn(containerWithPipeline); volumeName = UUID.randomUUID().toString(); bucketName = UUID.randomUUID().toString(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java index 89b179391cee..c4384c2dc906 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java @@ -65,7 +65,7 @@ public void testAddToDBBatch() throws Exception { String ozoneKey = getOzoneKey(); OMKeyCommitResponse omKeyCommitResponse = getOmKeyCommitResponse( - omKeyInfo, omResponse, openKey, ozoneKey, keysToDelete, false); + omKeyInfo, omResponse, openKey, ozoneKey, keysToDelete, false, null); omKeyCommitResponse.addToDBBatch(omMetadataManager, batchOperation); @@ -94,7 +94,7 @@ public void testAddToDBBatchNoOp() throws Exception { String ozoneKey = getOzoneKey(); OMKeyCommitResponse omKeyCommitResponse = getOmKeyCommitResponse( - omKeyInfo, omResponse, openKey, ozoneKey, null, false); + omKeyInfo, omResponse, openKey, ozoneKey, null, false, null); // As during commit Key, entry will be already there in openKeyTable. // Adding it here. @@ -148,7 +148,7 @@ protected String getOzoneKey() throws IOException { @Nonnull protected OMKeyCommitResponse getOmKeyCommitResponse(OmKeyInfo omKeyInfo, OzoneManagerProtocolProtos.OMResponse omResponse, String openKey, - String ozoneKey, RepeatedOmKeyInfo deleteKeys, Boolean isHSync) + String ozoneKey, RepeatedOmKeyInfo deleteKeys, Boolean isHSync, OmKeyInfo newOpenKeyInfo) throws IOException { assertNotNull(omBucketInfo); Map deleteKeyMap = new HashMap<>(); @@ -158,6 +158,6 @@ protected OMKeyCommitResponse getOmKeyCommitResponse(OmKeyInfo omKeyInfo, new RepeatedOmKeyInfo(e))); } return new OMKeyCommitResponse(omResponse, omKeyInfo, ozoneKey, openKey, - omBucketInfo, deleteKeyMap, isHSync); + omBucketInfo, deleteKeyMap, isHSync, newOpenKeyInfo); } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseWithFSO.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseWithFSO.java index 32d55d3e961c..62998d87b72c 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseWithFSO.java @@ -44,7 +44,7 @@ public class TestOMKeyCommitResponseWithFSO extends TestOMKeyCommitResponse { @Override protected OMKeyCommitResponse getOmKeyCommitResponse(OmKeyInfo omKeyInfo, OzoneManagerProtocolProtos.OMResponse omResponse, String openKey, - String ozoneKey, RepeatedOmKeyInfo deleteKeys, Boolean isHSync) + String ozoneKey, RepeatedOmKeyInfo deleteKeys, Boolean isHSync, OmKeyInfo newOpenKeyInfo) throws IOException { assertNotNull(omBucketInfo); long volumeId = omMetadataManager.getVolumeId(omKeyInfo.getVolumeName()); @@ -57,7 +57,7 @@ protected OMKeyCommitResponse getOmKeyCommitResponse(OmKeyInfo omKeyInfo, new RepeatedOmKeyInfo(e))); } return new OMKeyCommitResponseWithFSO(omResponse, omKeyInfo, ozoneKey, - openKey, omBucketInfo, deleteKeyMap, volumeId, isHSync); + openKey, omBucketInfo, deleteKeyMap, volumeId, isHSync, newOpenKeyInfo); } @Nonnull diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestOpenKeyCleanupService.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestOpenKeyCleanupService.java index 2ef6c341ae73..aa51e2be7b67 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestOpenKeyCleanupService.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestOpenKeyCleanupService.java @@ -22,11 +22,18 @@ import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.server.ServerUtils; import org.apache.hadoop.hdds.utils.db.DBConfigFromFile; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.om.ExpiredOpenKeys; @@ -38,6 +45,7 @@ import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.ozone.om.helpers.OpenKeySession; @@ -47,11 +55,16 @@ import org.apache.ratis.util.ExitUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.MethodOrderer.OrderAnnotation; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.TestMethodOrder; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,15 +74,20 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.UUID; import java.util.concurrent.TimeUnit; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_LEASE_HARD_LIMIT; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_OPEN_KEY_CLEANUP_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_OPEN_KEY_EXPIRE_THRESHOLD; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.when; @TestInstance(TestInstance.Lifecycle.PER_CLASS) +@TestMethodOrder(OrderAnnotation.class) class TestOpenKeyCleanupService { private OzoneManagerProtocol writeClient; private OzoneManager om; @@ -96,6 +114,9 @@ void setup(@TempDir Path tempDir) throws Exception { SERVICE_INTERVAL, TimeUnit.MILLISECONDS); conf.setTimeDuration(OZONE_OM_OPEN_KEY_EXPIRE_THRESHOLD, EXPIRE_THRESHOLD_MS, TimeUnit.MILLISECONDS); + conf.setTimeDuration(OZONE_OM_LEASE_HARD_LIMIT, + EXPIRE_THRESHOLD_MS, TimeUnit.MILLISECONDS); + conf.set(OzoneConfigKeys.OZONE_OM_LEASE_SOFT_LIMIT, "0s"); conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true); conf.setQuietMode(false); OmTestManagers omTestManagers = new OmTestManagers(conf); @@ -150,8 +171,8 @@ public void testCleanupExpiredOpenKeys( long numOpenKeysCleaned = metrics.getNumOpenKeysCleaned(); long numOpenKeysHSyncCleaned = metrics.getNumOpenKeysHSyncCleaned(); final int keyCount = numDEFKeys + numFSOKeys; - createOpenKeys(numDEFKeys, false, BucketLayout.DEFAULT); - createOpenKeys(numFSOKeys, hsync, BucketLayout.FILE_SYSTEM_OPTIMIZED); + createOpenKeys(numDEFKeys, false, BucketLayout.DEFAULT, false); + createOpenKeys(numFSOKeys, hsync, BucketLayout.FILE_SYSTEM_OPTIMIZED, false); // wait for open keys to expire Thread.sleep(EXPIRE_THRESHOLD_MS); @@ -183,6 +204,62 @@ public void testCleanupExpiredOpenKeys( } } + /** + * In this test, we create a bunch of hsync keys with some keys having recover flag set. + * OpenKeyCleanupService should commit keys which don't have recovery flag and has expired. + * Keys with recovery flag and expired should be ignored by OpenKeyCleanupService. + * @throws IOException - on Failure. + */ + @Test + @Timeout(300) + // Run this test first to avoid any lingering keys generated by other tests. + @Order(1) + public void testIgnoreExpiredRecoverhsyncKeys() throws Exception { + OpenKeyCleanupService openKeyCleanupService = + (OpenKeyCleanupService) keyManager.getOpenKeyCleanupService(); + + openKeyCleanupService.suspend(); + // wait for submitted tasks to complete + Thread.sleep(SERVICE_INTERVAL); + final long oldkeyCount = openKeyCleanupService.getSubmittedOpenKeyCount(); + LOG.info("oldkeyCount={}", oldkeyCount); + assertEquals(0, oldkeyCount); + + final OMMetrics metrics = om.getMetrics(); + assertEquals(0, metrics.getNumOpenKeysHSyncCleaned()); + int keyCount = 10; + Pipeline pipeline = Pipeline.newBuilder() + .setState(Pipeline.PipelineState.OPEN) + .setId(PipelineID.randomId()) + .setReplicationConfig( + StandaloneReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE)) + .setNodes(new ArrayList<>()) + .build(); + + when(om.getScmClient().getContainerClient().getContainerWithPipeline(anyLong())) + .thenReturn(new ContainerWithPipeline(Mockito.mock(ContainerInfo.class), pipeline)); + + createOpenKeys(keyCount, true, BucketLayout.FILE_SYSTEM_OPTIMIZED, false); + // create 2 more key and mark recovery flag set + createOpenKeys(2, true, BucketLayout.FILE_SYSTEM_OPTIMIZED, true); + + // wait for open keys to expire + Thread.sleep(EXPIRE_THRESHOLD_MS); + + // Only 10 keys should be returned after hard limit period, as 2 key is having recovery flag set + assertEquals(keyCount, getExpiredOpenKeys(true, BucketLayout.FILE_SYSTEM_OPTIMIZED)); + assertExpiredOpenKeys(false, true, + BucketLayout.FILE_SYSTEM_OPTIMIZED); + + openKeyCleanupService.resume(); + + // 10 keys should be recovered and there should not be any expired key pending + waitForOpenKeyCleanup(true, BucketLayout.FILE_SYSTEM_OPTIMIZED); + + // 2 keys should still remain in openKey table + assertEquals(2, getOpenKeyInfo(BucketLayout.FILE_SYSTEM_OPTIMIZED).size()); + } + /** * In this test, we create a bunch of incomplete MPU keys and try to run * openKeyCleanupService on it. We make sure that none of these incomplete @@ -323,7 +400,7 @@ private void assertExpiredOpenKeys(boolean expectedToEmpty, boolean hsync, private int getExpiredOpenKeys(boolean hsync, BucketLayout layout) { try { final ExpiredOpenKeys expired = keyManager.getExpiredOpenKeys( - EXPIRE_THRESHOLD, 100, layout); + EXPIRE_THRESHOLD, 100, layout, EXPIRE_THRESHOLD); return (hsync ? expired.getHsyncKeys() : expired.getOpenKeyBuckets()) .size(); } catch (IOException e) { @@ -331,6 +408,23 @@ private int getExpiredOpenKeys(boolean hsync, BucketLayout layout) { } } + private List getOpenKeyInfo(BucketLayout bucketLayout) { + List omKeyInfo = new ArrayList<>(); + + Table openFileTable = + om.getMetadataManager().getOpenKeyTable(bucketLayout); + try (TableIterator> + iterator = openFileTable.iterator()) { + while (iterator.hasNext()) { + omKeyInfo.add(iterator.next().getValue()); + } + + } catch (Exception e) { + } + return omKeyInfo; + } + + void waitForOpenKeyCleanup(boolean hsync, BucketLayout layout) throws Exception { GenericTestUtils.waitFor(() -> 0 == getExpiredOpenKeys(hsync, layout), @@ -338,7 +432,7 @@ void waitForOpenKeyCleanup(boolean hsync, BucketLayout layout) } private void createOpenKeys(int keyCount, boolean hsync, - BucketLayout bucketLayout) throws IOException { + BucketLayout bucketLayout, boolean recovery) throws IOException { String volume = UUID.randomUUID().toString(); String bucket = UUID.randomUUID().toString(); for (int x = 0; x < keyCount; x++) { @@ -351,9 +445,9 @@ private void createOpenKeys(int keyCount, boolean hsync, String key = UUID.randomUUID().toString(); createVolumeAndBucket(volume, bucket, bucketLayout); - final int numBlocks = RandomUtils.nextInt(0, 3); + final int numBlocks = RandomUtils.nextInt(1, 3); // Create the key - createOpenKey(volume, bucket, key, numBlocks, hsync); + createOpenKey(volume, bucket, key, numBlocks, hsync, recovery); } } @@ -377,7 +471,7 @@ private void createVolumeAndBucket(String volumeName, String bucketName, } private void createOpenKey(String volumeName, String bucketName, - String keyName, int numBlocks, boolean hsync) throws IOException { + String keyName, int numBlocks, boolean hsync, boolean recovery) throws IOException { OmKeyArgs keyArg = new OmKeyArgs.Builder() .setVolumeName(volumeName) @@ -397,6 +491,9 @@ private void createOpenKey(String volumeName, String bucketName, } if (hsync) { writeClient.hsyncKey(keyArg, session.getId()); + if (recovery) { + writeClient.recoverLease(volumeName, bucketName, keyName, false); + } } } diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ByteBufferPositionedReadable.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ByteBufferPositionedReadable.java new file mode 100644 index 000000000000..f5d0c8521334 --- /dev/null +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ByteBufferPositionedReadable.java @@ -0,0 +1,92 @@ +/** + * 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.fs; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * FIXME: Hack: This is copied from Hadoop 3.3.6. Remove this interface once + * we drop Hadoop 3.1, 3.2 support. + * Implementers of this interface provide a positioned read API that writes to a + * {@link ByteBuffer} rather than a {@code byte[]}. + * + * @see PositionedReadable + * @see ByteBufferReadable + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface ByteBufferPositionedReadable { + /** + * Reads up to {@code buf.remaining()} bytes into buf from a given position + * in the file and returns the number of bytes read. Callers should use + * {@code buf.limit(...)} to control the size of the desired read and + * {@code buf.position(...)} to control the offset into the buffer the data + * should be written to. + *

+ * After a successful call, {@code buf.position()} will be advanced by the + * number of bytes read and {@code buf.limit()} will be unchanged. + *

+ * In the case of an exception, the state of the buffer (the contents of the + * buffer, the {@code buf.position()}, the {@code buf.limit()}, etc.) is + * undefined, and callers should be prepared to recover from this + * eventuality. + *

+ * Callers should use {@link StreamCapabilities#hasCapability(String)} with + * {@link StreamCapabilities#PREADBYTEBUFFER} to check if the underlying + * stream supports this interface, otherwise they might get a + * {@link UnsupportedOperationException}. + *

+ * Implementations should treat 0-length requests as legitimate, and must not + * signal an error upon their receipt. + *

+ * This does not change the current offset of a file, and is thread-safe. + * + * @param position position within file + * @param buf the ByteBuffer to receive the results of the read operation. + * @return the number of bytes read, possibly zero, or -1 if reached + * end-of-stream + * @throws IOException if there is some error performing the read + */ + int read(long position, ByteBuffer buf) throws IOException; + + /** + * Reads {@code buf.remaining()} bytes into buf from a given position in + * the file or until the end of the data was reached before the read + * operation completed. Callers should use {@code buf.limit(...)} to + * control the size of the desired read and {@code buf.position(...)} to + * control the offset into the buffer the data should be written to. + *

+ * This operation provides similar semantics to + * {@link #read(long, ByteBuffer)}, the difference is that this method is + * guaranteed to read data until the {@link ByteBuffer} is full, or until + * the end of the data stream is reached. + * + * @param position position within file + * @param buf the ByteBuffer to receive the results of the read operation. + * @throws IOException if there is some error performing the read + * @throws EOFException the end of the data was reached before + * the read operation completed + * @see #read(long, ByteBuffer) + */ + void readFully(long position, ByteBuffer buf) throws IOException; +} diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java index 3d426ed03498..d44055236d8b 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java @@ -36,12 +36,20 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; import org.apache.hadoop.fs.SafeModeAction; +import org.apache.hadoop.hdds.client.ReplicatedReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.pipeline.PipelineID; +import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.io.Text; @@ -56,7 +64,11 @@ import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.client.rpc.RpcClient; +import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; @@ -690,11 +702,64 @@ private SnapshotDiffReportOzone getSnapshotDiffReportOnceComplete( } @Override - public boolean recoverLease(final String pathStr) throws IOException { - incrementCounter(Statistic.INVOCATION_RECOVER_LEASE, 1); + public LeaseKeyInfo recoverFilePrepare(final String pathStr, boolean force) throws IOException { + incrementCounter(Statistic.INVOCATION_RECOVER_FILE_PREPARE, 1); return ozoneClient.getProxy().getOzoneManagerClient().recoverLease( - volume.getName(), bucket.getName(), pathStr); + volume.getName(), bucket.getName(), pathStr, force); + } + + @Override + public void recoverFile(OmKeyArgs keyArgs) throws IOException { + incrementCounter(Statistic.INVOCATION_RECOVER_FILE, 1); + + ozoneClient.getProxy().getOzoneManagerClient().recoverKey(keyArgs, 0L); + } + + @Override + public long finalizeBlock(OmKeyLocationInfo block) throws IOException { + incrementCounter(Statistic.INVOCATION_FINALIZE_BLOCK, 1); + RpcClient rpcClient = (RpcClient) ozoneClient.getProxy(); + XceiverClientFactory xceiverClientFactory = rpcClient.getXceiverClientManager(); + Pipeline pipeline = block.getPipeline(); + XceiverClientSpi client = null; + try { + // If pipeline is still open + if (pipeline.isOpen()) { + client = xceiverClientFactory.acquireClient(pipeline); + ContainerProtos.FinalizeBlockResponseProto finalizeBlockResponseProto = + ContainerProtocolCalls.finalizeBlock(client, block.getBlockID().getDatanodeBlockIDProtobuf(), + block.getToken()); + return BlockData.getFromProtoBuf(finalizeBlockResponseProto.getBlockData()).getSize(); + } + } catch (IOException e) { + LOG.warn("Failed to execute finalizeBlock command", e); + } finally { + if (client != null) { + xceiverClientFactory.releaseClient(client, false); + } + } + + // Try fetch block committed length from DN + ReplicationConfig replicationConfig = pipeline.getReplicationConfig(); + if (!(replicationConfig instanceof ReplicatedReplicationConfig)) { + throw new IOException("ReplicationConfig type " + replicationConfig.getClass().getSimpleName() + + " is not supported in finalizeBlock"); + } + StandaloneReplicationConfig newConfig = StandaloneReplicationConfig.getInstance( + ((ReplicatedReplicationConfig) replicationConfig).getReplicationFactor()); + Pipeline.Builder builder = Pipeline.newBuilder().setReplicationConfig(newConfig).setId(PipelineID.randomId()) + .setNodes(block.getPipeline().getNodes()).setState(Pipeline.PipelineState.OPEN); + try { + client = xceiverClientFactory.acquireClientForReadData(builder.build()); + ContainerProtos.GetCommittedBlockLengthResponseProto responseProto = + ContainerProtocolCalls.getCommittedBlockLength(client, block.getBlockID(), block.getToken()); + return responseProto.getBlockLength(); + } finally { + if (client != null) { + xceiverClientFactory.releaseClient(client, false); + } + } } @Override diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java index 3bbae134db51..de9603c475f7 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java @@ -43,13 +43,21 @@ import org.apache.hadoop.fs.PathPermissionException; import org.apache.hadoop.fs.SafeModeAction; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdds.client.ReplicatedReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.pipeline.PipelineID; +import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.io.Text; @@ -67,8 +75,12 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.client.BucketArgs; +import org.apache.hadoop.ozone.client.rpc.RpcClient; +import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; @@ -1363,14 +1375,67 @@ public boolean isFileClosed(String pathStr) throws IOException { } @Override - public boolean recoverLease(final String pathStr) throws IOException { - incrementCounter(Statistic.INVOCATION_RECOVER_LEASE, 1); + public LeaseKeyInfo recoverFilePrepare(final String pathStr, boolean force) throws IOException { + incrementCounter(Statistic.INVOCATION_RECOVER_FILE_PREPARE, 1); OFSPath ofsPath = new OFSPath(pathStr, config); OzoneVolume volume = objectStore.getVolume(ofsPath.getVolumeName()); OzoneBucket bucket = getBucket(ofsPath, false); return ozoneClient.getProxy().getOzoneManagerClient().recoverLease( - volume.getName(), bucket.getName(), ofsPath.getKeyName()); + volume.getName(), bucket.getName(), ofsPath.getKeyName(), force); + } + + @Override + public void recoverFile(OmKeyArgs keyArgs) throws IOException { + incrementCounter(Statistic.INVOCATION_RECOVER_FILE, 1); + + ozoneClient.getProxy().getOzoneManagerClient().recoverKey(keyArgs, 0L); + } + + @Override + public long finalizeBlock(OmKeyLocationInfo block) throws IOException { + incrementCounter(Statistic.INVOCATION_FINALIZE_BLOCK, 1); + RpcClient rpcClient = (RpcClient) ozoneClient.getProxy(); + XceiverClientFactory xceiverClientFactory = rpcClient.getXceiverClientManager(); + Pipeline pipeline = block.getPipeline(); + XceiverClientSpi client = null; + try { + // If pipeline is still open + if (pipeline.isOpen()) { + client = xceiverClientFactory.acquireClient(pipeline); + ContainerProtos.FinalizeBlockResponseProto finalizeBlockResponseProto = + ContainerProtocolCalls.finalizeBlock(client, block.getBlockID().getDatanodeBlockIDProtobuf(), + block.getToken()); + return BlockData.getFromProtoBuf(finalizeBlockResponseProto.getBlockData()).getSize(); + } + } catch (IOException e) { + LOG.warn("Failed to execute finalizeBlock command", e); + } finally { + if (client != null) { + xceiverClientFactory.releaseClient(client, false); + } + } + + // Try fetch block committed length from DN + ReplicationConfig replicationConfig = pipeline.getReplicationConfig(); + if (!(replicationConfig instanceof ReplicatedReplicationConfig)) { + throw new IOException("ReplicationConfig type " + replicationConfig.getClass().getSimpleName() + + " is not supported in finalizeBlock"); + } + StandaloneReplicationConfig newConfig = StandaloneReplicationConfig.getInstance( + ((ReplicatedReplicationConfig) replicationConfig).getReplicationFactor()); + Pipeline.Builder builder = Pipeline.newBuilder().setReplicationConfig(newConfig).setId(PipelineID.randomId()) + .setNodes(block.getPipeline().getNodes()).setState(Pipeline.PipelineState.OPEN); + try { + client = xceiverClientFactory.acquireClientForReadData(builder.build()); + ContainerProtos.GetCommittedBlockLengthResponseProto responseProto = + ContainerProtocolCalls.getCommittedBlockLength(client, block.getBlockID(), block.getToken()); + return responseProto.getBlockLength(); + } finally { + if (client != null) { + xceiverClientFactory.releaseClient(client, false); + } + } } @Override diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSInputStream.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSInputStream.java index 290546e4a104..30e0c32265bf 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSInputStream.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/CapableOzoneFSInputStream.java @@ -35,6 +35,7 @@ public boolean hasCapability(String capability) { switch (StringUtils.toLowerCase(capability)) { case StreamCapabilities.READBYTEBUFFER: case StreamCapabilities.UNBUFFER: + case StreamCapabilities.PREADBYTEBUFFER: return true; default: return false; diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java index 1a6462c1bb3a..7e78d6650ee3 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java @@ -28,6 +28,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.SafeModeAction; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.security.OzoneTokenIdentifier; import org.apache.hadoop.security.token.Token; @@ -97,7 +100,11 @@ SnapshotDiffReport getSnapshotDiffReport(Path snapshotDir, String fromSnapshot, String toSnapshot) throws IOException, InterruptedException; - boolean recoverLease(String pathStr) throws IOException; + LeaseKeyInfo recoverFilePrepare(String pathStr, boolean force) throws IOException; + + void recoverFile(OmKeyArgs keyArgs) throws IOException; + + long finalizeBlock(OmKeyLocationInfo block) throws IOException; void setTimes(String key, long mtime, long atime) throws IOException; diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java index 35ee20d56c34..4dc70bfa569d 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.ozone; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; @@ -30,6 +31,7 @@ import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.apache.hadoop.fs.ByteBufferReadable; +import org.apache.hadoop.fs.ByteBufferPositionedReadable; import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.Seekable; @@ -44,7 +46,7 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class OzoneFSInputStream extends FSInputStream - implements ByteBufferReadable, CanUnbuffer { + implements ByteBufferReadable, CanUnbuffer, ByteBufferPositionedReadable { private final InputStream inputStream; private final Statistics statistics; @@ -161,4 +163,49 @@ public void unbuffer() { ((CanUnbuffer) inputStream).unbuffer(); } } + + /** + * @param buf the ByteBuffer to receive the results of the read operation. + * @param position offset + * @return the number of bytes read, possibly zero, or -1 if + * reach end-of-stream + * @throws IOException if there is some error performing the read + */ + @Override + public int read(long position, ByteBuffer buf) throws IOException { + if (!buf.hasRemaining()) { + return 0; + } + long oldPos = this.getPos(); + int bytesRead; + try { + ((Seekable) inputStream).seek(position); + bytesRead = ((ByteBufferReadable) inputStream).read(buf); + } catch (EOFException e) { + // Either position is negative or it has reached EOF + return -1; + } finally { + ((Seekable) inputStream).seek(oldPos); + } + return bytesRead; + } + + /** + * @param buf the ByteBuffer to receive the results of the read operation. + * @param position offset + * @return void + * @throws IOException if there is some error performing the read + * @throws EOFException if end of file reached before reading fully + */ + @Override + public void readFully(long position, ByteBuffer buf) throws IOException { + int bytesRead; + for (int readCount = 0; buf.hasRemaining(); readCount += bytesRead) { + bytesRead = this.read(position + (long)readCount, buf); + if (bytesRead < 0) { + // Still buffer has space to read but stream has already reached EOF + throw new EOFException("End of file reached before reading fully."); + } + } + } } diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/Statistic.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/Statistic.java index aae71e9c4cde..f28f2b7d437c 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/Statistic.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/Statistic.java @@ -77,8 +77,10 @@ public enum Statistic { "Calls of setTimes()"), INVOCATION_IS_FILE_CLOSED("op_is_file_closed", "Calls of isFileClosed()"), - INVOCATION_RECOVER_LEASE("op_recover_lease", - "Calls of recoverLease()"), + INVOCATION_RECOVER_FILE_PREPARE("op_recover_file_prepare", + "Calls of recoverFilePrepare()"), + INVOCATION_RECOVER_FILE("op_recover_file", "Calls of recoverFile()"), + INVOCATION_FINALIZE_BLOCK("op_finalize_block", "Calls of finalizeBlock()"), INVOCATION_SET_SAFE_MODE("op_set_safe_mode", "Calls of setSafeMode()"); diff --git a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java index 203e7ed373eb..4de4b22908d7 100644 --- a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java +++ b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java @@ -21,7 +21,9 @@ import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.util.List; +import com.google.common.base.Strings; import org.apache.hadoop.crypto.key.KeyProvider; import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer; import org.apache.hadoop.fs.FileSystem; @@ -33,8 +35,17 @@ import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.security.token.DelegationTokenIssuer; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; +import static org.apache.hadoop.ozone.OzoneConsts.FORCE_LEASE_RECOVERY_ENV; + /** * The Ozone Filesystem implementation. *

@@ -49,9 +60,12 @@ public class OzoneFileSystem extends BasicOzoneFileSystem implements KeyProviderTokenIssuer, LeaseRecoverable, SafeMode { private OzoneFSStorageStatistics storageStatistics; + private boolean forceRecovery; public OzoneFileSystem() { this.storageStatistics = new OzoneFSStorageStatistics(); + String force = System.getProperty(FORCE_LEASE_RECOVERY_ENV); + forceRecovery = Strings.isNullOrEmpty(force) ? false : Boolean.parseBoolean(force); } @Override @@ -126,10 +140,62 @@ public boolean hasPathCapability(final Path path, final String capability) @Override public boolean recoverLease(Path f) throws IOException { + statistics.incrementWriteOps(1); LOG.trace("recoverLease() path:{}", f); + Path qualifiedPath = makeQualified(f); String key = pathToKey(qualifiedPath); - return getAdapter().recoverLease(key); + LeaseKeyInfo leaseKeyInfo; + try { + leaseKeyInfo = getAdapter().recoverFilePrepare(key, forceRecovery); + } catch (OMException e) { + if (e.getResult() == OMException.ResultCodes.KEY_ALREADY_CLOSED) { + // key is already closed, let's just return success + return true; + } + throw e; + } + + // finalize the final block and get block length + List locationInfoList = leaseKeyInfo.getKeyInfo().getLatestVersionLocations().getLocationList(); + if (!locationInfoList.isEmpty()) { + OmKeyLocationInfo block = locationInfoList.get(locationInfoList.size() - 1); + try { + block.setLength(getAdapter().finalizeBlock(block)); + } catch (Throwable e) { + if (e instanceof StorageContainerException && (((StorageContainerException) e).getResult().equals(NO_SUCH_BLOCK) + || ((StorageContainerException) e).getResult().equals(CONTAINER_NOT_FOUND)) + && !leaseKeyInfo.getIsKeyInfo() && locationInfoList.size() > 1) { + locationInfoList = leaseKeyInfo.getKeyInfo().getLatestVersionLocations().getLocationList().subList(0, + locationInfoList.size() - 1); + block = locationInfoList.get(locationInfoList.size() - 1); + try { + block.setLength(getAdapter().finalizeBlock(block)); + } catch (Throwable exp) { + if (!forceRecovery) { + throw exp; + } + LOG.warn("Failed to finalize block. Continue to recover the file since {} is enabled.", + FORCE_LEASE_RECOVERY_ENV, exp); + } + } else if (!forceRecovery) { + throw e; + } else { + LOG.warn("Failed to finalize block. Continue to recover the file since {} is enabled.", + FORCE_LEASE_RECOVERY_ENV, e); + } + } + } + + // recover and commit file + long keyLength = locationInfoList.stream().mapToLong(OmKeyLocationInfo::getLength).sum(); + OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(leaseKeyInfo.getKeyInfo().getVolumeName()) + .setBucketName(leaseKeyInfo.getKeyInfo().getBucketName()).setKeyName(leaseKeyInfo.getKeyInfo().getKeyName()) + .setReplicationConfig(leaseKeyInfo.getKeyInfo().getReplicationConfig()).setDataSize(keyLength) + .setLocationInfoList(locationInfoList) + .build(); + getAdapter().recoverFile(keyArgs); + return true; } @Override diff --git a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java index 9b1596c05b9e..3025b1af03be 100644 --- a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java +++ b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.ozone; +import com.google.common.base.Strings; import org.apache.hadoop.fs.LeaseRecoverable; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.SafeMode; @@ -29,11 +30,21 @@ import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.security.token.DelegationTokenIssuer; import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.util.List; + +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; +import static org.apache.hadoop.ozone.OzoneConsts.FORCE_LEASE_RECOVERY_ENV; /** * The Rooted Ozone Filesystem (OFS) implementation. @@ -49,9 +60,12 @@ public class RootedOzoneFileSystem extends BasicRootedOzoneFileSystem implements KeyProviderTokenIssuer, LeaseRecoverable, SafeMode { private OzoneFSStorageStatistics storageStatistics; + private boolean forceRecovery; public RootedOzoneFileSystem() { this.storageStatistics = new OzoneFSStorageStatistics(); + String force = System.getProperty(FORCE_LEASE_RECOVERY_ENV); + forceRecovery = Strings.isNullOrEmpty(force) ? false : Boolean.parseBoolean(force); } @Override @@ -131,17 +145,65 @@ public boolean hasPathCapability(final Path path, final String capability) */ @Override public boolean recoverLease(final Path f) throws IOException { - incrementCounter(Statistic.INVOCATION_RECOVER_LEASE, 1); statistics.incrementWriteOps(1); LOG.trace("recoverLease() path:{}", f); Path qualifiedPath = makeQualified(f); String key = pathToKey(qualifiedPath); - return getAdapter().recoverLease(key); + LeaseKeyInfo leaseKeyInfo; + try { + leaseKeyInfo = getAdapter().recoverFilePrepare(key, forceRecovery); + } catch (OMException e) { + if (e.getResult() == OMException.ResultCodes.KEY_ALREADY_CLOSED) { + // key is already closed, let's just return success + return true; + } + throw e; + } + + // finalize the final block and get block length + List locationInfoList = leaseKeyInfo.getKeyInfo().getLatestVersionLocations().getLocationList(); + if (!locationInfoList.isEmpty()) { + OmKeyLocationInfo block = locationInfoList.get(locationInfoList.size() - 1); + try { + block.setLength(getAdapter().finalizeBlock(block)); + } catch (Throwable e) { + if (e instanceof StorageContainerException && (((StorageContainerException) e).getResult().equals(NO_SUCH_BLOCK) + || ((StorageContainerException) e).getResult().equals(CONTAINER_NOT_FOUND)) + && !leaseKeyInfo.getIsKeyInfo() && locationInfoList.size() > 1) { + locationInfoList = leaseKeyInfo.getKeyInfo().getLatestVersionLocations().getLocationList().subList(0, + locationInfoList.size() - 1); + block = locationInfoList.get(locationInfoList.size() - 1); + try { + block.setLength(getAdapter().finalizeBlock(block)); + } catch (Throwable exp) { + if (!forceRecovery) { + throw exp; + } + LOG.warn("Failed to finalize block. Continue to recover the file since {} is enabled.", + FORCE_LEASE_RECOVERY_ENV, exp); + } + } else if (!forceRecovery) { + throw e; + } else { + LOG.warn("Failed to finalize block. Continue to recover the file since {} is enabled.", + FORCE_LEASE_RECOVERY_ENV, e); + } + } + } + + // recover and commit file + long keyLength = locationInfoList.stream().mapToLong(OmKeyLocationInfo::getLength).sum(); + OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(leaseKeyInfo.getKeyInfo().getVolumeName()) + .setBucketName(leaseKeyInfo.getKeyInfo().getBucketName()).setKeyName(leaseKeyInfo.getKeyInfo().getKeyName()) + .setReplicationConfig(leaseKeyInfo.getKeyInfo().getReplicationConfig()).setDataSize(keyLength) + .setLocationInfoList(locationInfoList) + .build(); + getAdapter().recoverFile(keyArgs); + return true; } @Override public boolean isFileClosed(Path f) throws IOException { - incrementCounter(Statistic.INVOCATION_IS_FILE_CLOSED, 1); statistics.incrementReadOps(1); LOG.trace("isFileClosed() path:{}", f); Path qualifiedPath = makeQualified(f); diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java index 71f01e441427..4de4b22908d7 100644 --- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java +++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java @@ -21,7 +21,9 @@ import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.util.List; +import com.google.common.base.Strings; import org.apache.hadoop.crypto.key.KeyProvider; import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer; import org.apache.hadoop.fs.FileSystem; @@ -33,8 +35,17 @@ import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.security.token.DelegationTokenIssuer; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; +import static org.apache.hadoop.ozone.OzoneConsts.FORCE_LEASE_RECOVERY_ENV; + /** * The Ozone Filesystem implementation. *

@@ -49,9 +60,12 @@ public class OzoneFileSystem extends BasicOzoneFileSystem implements KeyProviderTokenIssuer, LeaseRecoverable, SafeMode { private OzoneFSStorageStatistics storageStatistics; + private boolean forceRecovery; public OzoneFileSystem() { this.storageStatistics = new OzoneFSStorageStatistics(); + String force = System.getProperty(FORCE_LEASE_RECOVERY_ENV); + forceRecovery = Strings.isNullOrEmpty(force) ? false : Boolean.parseBoolean(force); } @Override @@ -126,10 +140,62 @@ public boolean hasPathCapability(final Path path, final String capability) @Override public boolean recoverLease(Path f) throws IOException { - LOG.trace("isFileClosed() path:{}", f); + statistics.incrementWriteOps(1); + LOG.trace("recoverLease() path:{}", f); + Path qualifiedPath = makeQualified(f); String key = pathToKey(qualifiedPath); - return getAdapter().recoverLease(key); + LeaseKeyInfo leaseKeyInfo; + try { + leaseKeyInfo = getAdapter().recoverFilePrepare(key, forceRecovery); + } catch (OMException e) { + if (e.getResult() == OMException.ResultCodes.KEY_ALREADY_CLOSED) { + // key is already closed, let's just return success + return true; + } + throw e; + } + + // finalize the final block and get block length + List locationInfoList = leaseKeyInfo.getKeyInfo().getLatestVersionLocations().getLocationList(); + if (!locationInfoList.isEmpty()) { + OmKeyLocationInfo block = locationInfoList.get(locationInfoList.size() - 1); + try { + block.setLength(getAdapter().finalizeBlock(block)); + } catch (Throwable e) { + if (e instanceof StorageContainerException && (((StorageContainerException) e).getResult().equals(NO_SUCH_BLOCK) + || ((StorageContainerException) e).getResult().equals(CONTAINER_NOT_FOUND)) + && !leaseKeyInfo.getIsKeyInfo() && locationInfoList.size() > 1) { + locationInfoList = leaseKeyInfo.getKeyInfo().getLatestVersionLocations().getLocationList().subList(0, + locationInfoList.size() - 1); + block = locationInfoList.get(locationInfoList.size() - 1); + try { + block.setLength(getAdapter().finalizeBlock(block)); + } catch (Throwable exp) { + if (!forceRecovery) { + throw exp; + } + LOG.warn("Failed to finalize block. Continue to recover the file since {} is enabled.", + FORCE_LEASE_RECOVERY_ENV, exp); + } + } else if (!forceRecovery) { + throw e; + } else { + LOG.warn("Failed to finalize block. Continue to recover the file since {} is enabled.", + FORCE_LEASE_RECOVERY_ENV, e); + } + } + } + + // recover and commit file + long keyLength = locationInfoList.stream().mapToLong(OmKeyLocationInfo::getLength).sum(); + OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(leaseKeyInfo.getKeyInfo().getVolumeName()) + .setBucketName(leaseKeyInfo.getKeyInfo().getBucketName()).setKeyName(leaseKeyInfo.getKeyInfo().getKeyName()) + .setReplicationConfig(leaseKeyInfo.getKeyInfo().getReplicationConfig()).setDataSize(keyLength) + .setLocationInfoList(locationInfoList) + .build(); + getAdapter().recoverFile(keyArgs); + return true; } @Override diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java index c377128d2940..c06a6b7644e8 100644 --- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java +++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java @@ -18,24 +18,35 @@ package org.apache.hadoop.fs.ozone; +import com.google.common.base.Strings; import io.opentracing.util.GlobalTracer; +import org.apache.hadoop.crypto.key.KeyProvider; +import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LeaseRecoverable; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.SafeMode; import org.apache.hadoop.fs.SafeModeAction; +import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.apache.hadoop.hdds.conf.ConfigurationSource; -import org.apache.hadoop.crypto.key.KeyProvider; -import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.security.token.DelegationTokenIssuer; import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.util.List; + +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; +import static org.apache.hadoop.ozone.OzoneConsts.FORCE_LEASE_RECOVERY_ENV; /** * The Rooted Ozone Filesystem (OFS) implementation. @@ -51,9 +62,12 @@ public class RootedOzoneFileSystem extends BasicRootedOzoneFileSystem implements KeyProviderTokenIssuer, LeaseRecoverable, SafeMode { private OzoneFSStorageStatistics storageStatistics; + private boolean forceRecovery; public RootedOzoneFileSystem() { this.storageStatistics = new OzoneFSStorageStatistics(); + String force = System.getProperty(FORCE_LEASE_RECOVERY_ENV); + forceRecovery = Strings.isNullOrEmpty(force) ? false : Boolean.parseBoolean(force); } @Override @@ -135,7 +149,57 @@ private boolean recoverLeaseTraced(final Path f) throws IOException { LOG.trace("recoverLease() path:{}", f); Path qualifiedPath = makeQualified(f); String key = pathToKey(qualifiedPath); - return getAdapter().recoverLease(key); + LeaseKeyInfo leaseKeyInfo; + try { + leaseKeyInfo = getAdapter().recoverFilePrepare(key, forceRecovery); + } catch (OMException e) { + if (e.getResult() == OMException.ResultCodes.KEY_ALREADY_CLOSED) { + // key is already closed, let's just return success + return true; + } + throw e; + } + + // finalize the final block and get block length + List locationInfoList = leaseKeyInfo.getKeyInfo().getLatestVersionLocations().getLocationList(); + if (!locationInfoList.isEmpty()) { + OmKeyLocationInfo block = locationInfoList.get(locationInfoList.size() - 1); + try { + block.setLength(getAdapter().finalizeBlock(block)); + } catch (Throwable e) { + if (e instanceof StorageContainerException && (((StorageContainerException) e).getResult().equals(NO_SUCH_BLOCK) + || ((StorageContainerException) e).getResult().equals(CONTAINER_NOT_FOUND)) + && !leaseKeyInfo.getIsKeyInfo() && locationInfoList.size() > 1) { + locationInfoList = leaseKeyInfo.getKeyInfo().getLatestVersionLocations().getLocationList().subList(0, + locationInfoList.size() - 1); + block = locationInfoList.get(locationInfoList.size() - 1); + try { + block.setLength(getAdapter().finalizeBlock(block)); + } catch (Throwable exp) { + if (!forceRecovery) { + throw exp; + } + LOG.warn("Failed to finalize block. Continue to recover the file since {} is enabled.", + FORCE_LEASE_RECOVERY_ENV, exp); + } + } else if (!forceRecovery) { + throw e; + } else { + LOG.warn("Failed to finalize block. Continue to recover the file since {} is enabled.", + FORCE_LEASE_RECOVERY_ENV, e); + } + } + } + + // recover and commit file + long keyLength = locationInfoList.stream().mapToLong(OmKeyLocationInfo::getLength).sum(); + OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(leaseKeyInfo.getKeyInfo().getVolumeName()) + .setBucketName(leaseKeyInfo.getKeyInfo().getBucketName()).setKeyName(leaseKeyInfo.getKeyInfo().getKeyName()) + .setReplicationConfig(leaseKeyInfo.getKeyInfo().getReplicationConfig()).setDataSize(keyLength) + .setLocationInfoList(locationInfoList) + .build(); + getAdapter().recoverFile(keyArgs); + return true; } @Override diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java new file mode 100644 index 000000000000..9ede45a80aa6 --- /dev/null +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/ListOpenFilesSubCommand.java @@ -0,0 +1,202 @@ +/** + * 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.ozone.admin.om; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hdds.cli.HddsVersionProvider; +import org.apache.hadoop.hdds.server.JsonUtils; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.om.helpers.ListOpenFilesResult; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import picocli.CommandLine; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.Callable; + +/** + * Handler of ozone admin om list-open-files command. + */ +@CommandLine.Command( + name = "list-open-files", + aliases = {"list-open-keys", "lof", "lok"}, + description = "Lists open files (keys) in Ozone Manager.", + mixinStandardHelpOptions = true, + versionProvider = HddsVersionProvider.class +) +public class ListOpenFilesSubCommand implements Callable { + + @CommandLine.ParentCommand + private OMAdmin parent; + + @CommandLine.Option( + names = {"--service-id", "--om-service-id"}, + description = "Ozone Manager Service ID", + required = false + ) + private String omServiceId; + + @CommandLine.Option( + names = {"--service-host"}, + description = "Ozone Manager Host. If OM HA is enabled, use --service-id instead. " + + "If you must use --service-host with OM HA, this must point directly to the leader OM. " + + "This option is required when --service-id is not provided or when HA is not enabled." + ) + private String omHost; + + @CommandLine.Option(names = { "--json" }, + defaultValue = "false", + description = "Format output as JSON") + private boolean json; + + // Conforms to ListOptions, but not all in ListOptions applies here thus + // not using that directly + @CommandLine.Option( + names = {"-p", "--prefix"}, + description = "Filter results by the specified path on the server side.", + defaultValue = "/" + ) + private String pathPrefix; + + @CommandLine.Option( + names = {"-l", "--length"}, + description = "Maximum number of items to list", + defaultValue = "100" + ) + private int limit; + + @CommandLine.Option( + names = {"-s", "--start"}, + description = "The item to start the listing from.\n" + + "i.e. continuation token. " + + "This will be excluded from the result.", + defaultValue = "" + ) + private String startItem; + + @Override + public Void call() throws Exception { + + if (StringUtils.isEmpty(omServiceId) && StringUtils.isEmpty(omHost)) { + System.err.println("Error: Please specify -id or -host"); + return null; + } + + OzoneManagerProtocol ozoneManagerClient = + parent.createOmClient(omServiceId, omHost, false); + + ListOpenFilesResult res = + ozoneManagerClient.listOpenFiles(pathPrefix, limit, startItem); + + if (json) { + // Print detailed JSON + printOpenKeysListAsJson(res); + } else { + // Human friendly output + printOpenKeysList(res); + } + + return null; + } + + private void printOpenKeysListAsJson(ListOpenFilesResult res) + throws IOException { + System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(res)); + } + + private void printOpenKeysList(ListOpenFilesResult res) { + + List openFileList = res.getOpenKeys(); + + String msg = res.getTotalOpenKeyCount() + + " total open files (est.). Showing " + openFileList.size() + + " open files (limit " + limit + ") under path prefix:\n " + pathPrefix; + + if (startItem != null && !startItem.isEmpty()) { + msg += "\nafter continuation token:\n " + startItem; + } + msg += "\n\nClient ID\t\tCreation time\tHsync'ed\tOpen File Path"; + System.out.println(msg); + + for (OpenKeySession e : openFileList) { + long clientId = e.getId(); + OmKeyInfo omKeyInfo = e.getKeyInfo(); + String line = clientId + "\t" + omKeyInfo.getCreationTime() + "\t"; + + if (omKeyInfo.isHsync()) { + String hsyncClientIdStr = + omKeyInfo.getMetadata().get(OzoneConsts.HSYNC_CLIENT_ID); + long hsyncClientId = Long.parseLong(hsyncClientIdStr); + if (clientId == hsyncClientId) { + line += "Yes\t\t"; + } else { + // last hsync'ed with a different client ID than the client that + // initially opens the file (!) + line += "Yes w/ cid " + hsyncClientIdStr + "\t"; + } + } else { + line += "No\t\t"; + } + + line += getFullPathFromKeyInfo(omKeyInfo); + + System.out.println(line); + } + + // Compose next batch's command + if (res.hasMore()) { + String nextBatchCmd = getCmdForNextBatch(res.getContinuationToken()); + + System.out.println("\n" + + "To get the next batch of open keys, run:\n " + nextBatchCmd); + } else { + System.out.println("\nReached the end of the list."); + } + } + + /** + * @return the command to get the next batch of open keys + */ + private String getCmdForNextBatch(String lastElementFullPath) { + String nextBatchCmd = "ozone admin om lof"; + if (omServiceId != null && !omServiceId.isEmpty()) { + nextBatchCmd += " -id=" + omServiceId; + } + if (omHost != null && !omHost.isEmpty()) { + nextBatchCmd += " -host=" + omHost; + } + if (json) { + nextBatchCmd += " --json"; + } + nextBatchCmd += " --length=" + limit; + if (pathPrefix != null && !pathPrefix.isEmpty()) { + nextBatchCmd += " --prefix=" + pathPrefix; + } + nextBatchCmd += " --start=" + lastElementFullPath; + return nextBatchCmd; + } + + private String getFullPathFromKeyInfo(OmKeyInfo oki) { + return "/" + oki.getVolumeName() + + "/" + oki.getBucketName() + + "/" + oki.getPath(); + } + +} diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/OMAdmin.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/OMAdmin.java index ce7d4ed7a7ca..3162c5563545 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/OMAdmin.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/om/OMAdmin.java @@ -53,6 +53,7 @@ versionProvider = HddsVersionProvider.class, subcommands = { FinalizeUpgradeSubCommand.class, + ListOpenFilesSubCommand.class, GetServiceRolesSubcommand.class, PrepareSubCommand.class, CancelPrepareSubCommand.class, diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java new file mode 100644 index 000000000000..1d1b898a7d95 --- /dev/null +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java @@ -0,0 +1,178 @@ +/* + * 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.ozone.freon; + +import com.codahale.metrics.Timer; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.scm.client.ClientTrustManager; +import org.apache.hadoop.hdds.security.x509.certificate.client.CACertificateProvider; +import org.apache.hadoop.hdds.utils.HAUtils; +import org.apache.hadoop.ozone.OzoneSecurityUtil; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.hadoop.hdds.cli.HddsVersionProvider; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; +import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations; +import picocli.CommandLine; +import picocli.CommandLine.Command; +import picocli.CommandLine.Option; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; + +import static org.apache.hadoop.ozone.common.PayloadUtils.generatePayloadBytes; + +/** + * Utility to generate RPC request to DN. + */ +@Command(name = "dn-echo", + aliases = "dne", + description = + "Generate echo RPC request to DataNode", + versionProvider = HddsVersionProvider.class, + mixinStandardHelpOptions = true, + showDefaultValues = true) +public class DNRPCLoadGenerator extends BaseFreonGenerator + implements Callable { + + private static final int RPC_PAYLOAD_MULTIPLICATION_FACTOR = 1024; + private static final int MAX_SIZE_KB = 2097151; + private Timer timer; + private OzoneConfiguration configuration; + private ByteString payloadReqBytes; + private int payloadRespSize; + private List clients; + private String encodedContainerToken; + @Option(names = {"--payload-req"}, + description = + "Specifies the size of payload in KB in RPC request. ", + defaultValue = "0") + private int payloadReqSizeKB = 0; + + @Option(names = {"--payload-resp"}, + description = + "Specifies the size of payload in KB in RPC response. ", + defaultValue = "0") + private int payloadRespSizeKB = 0; + + @Option(names = {"--container-id"}, + description = "Send echo to DataNodes associated with this container") + private long containerID; + + @Option(names = {"--sleep-time-ms"}, + description = "Let DataNode to pause for a duration (in milliseconds) for each request", + defaultValue = "0") + private int sleepTimeMs = 0; + + @Option(names = {"--clients"}, + description = "number of xceiver clients", + defaultValue = "1") + private int numClients = 1; + + @CommandLine.ParentCommand + private Freon freon; + + // empy constructor for picocli + DNRPCLoadGenerator() { + } + + @VisibleForTesting + DNRPCLoadGenerator(OzoneConfiguration ozoneConfiguration) { + this.configuration = ozoneConfiguration; + } + + @Override + public Void call() throws Exception { + Preconditions.checkArgument(payloadReqSizeKB >= 0, + "OM echo request payload size should be positive value or zero."); + Preconditions.checkArgument(payloadRespSizeKB >= 0, + "OM echo response payload size should be positive value or zero."); + + if (configuration == null) { + configuration = freon.createOzoneConfiguration(); + } + ContainerOperationClient scmClient = new ContainerOperationClient(configuration); + ContainerInfo containerInfo = scmClient.getContainer(containerID); + + List pipelineList = scmClient.listPipelines(); + Pipeline pipeline = pipelineList.stream() + .filter(p -> p.getId().equals(containerInfo.getPipelineID())) + .findFirst() + .orElse(null); + encodedContainerToken = scmClient.getEncodedContainerToken(containerID); + XceiverClientFactory xceiverClientManager; + if (OzoneSecurityUtil.isSecurityEnabled(configuration)) { + CACertificateProvider caCerts = () -> HAUtils.buildCAX509List(null, configuration); + xceiverClientManager = new XceiverClientManager(configuration, + configuration.getObject(XceiverClientManager.ScmClientConfig.class), + new ClientTrustManager(caCerts, null)); + } else { + xceiverClientManager = new XceiverClientManager(configuration); + } + clients = new ArrayList<>(numClients); + for (int i = 0; i < numClients; i++) { + clients.add(xceiverClientManager.acquireClient(pipeline)); + } + + init(); + payloadReqBytes = UnsafeByteOperations.unsafeWrap(generatePayloadBytes(payloadReqSizeKB)); + payloadRespSize = calculateMaxPayloadSize(payloadRespSizeKB); + timer = getMetrics().timer("rpc-payload"); + try { + runTests(this::sendRPCReq); + } finally { + for (XceiverClientSpi client : clients) { + xceiverClientManager.releaseClient(client, false); + } + xceiverClientManager.close(); + scmClient.close(); + } + return null; + } + + private int calculateMaxPayloadSize(int payloadSizeKB) { + if (payloadSizeKB > 0) { + return Math.min( + Math.toIntExact((long)payloadSizeKB * + RPC_PAYLOAD_MULTIPLICATION_FACTOR), + MAX_SIZE_KB); + } + return 0; + } + + private void sendRPCReq(long l) throws Exception { + timer.time(() -> { + int clientIndex = (numClients == 1) ? 0 : (int)l % numClients; + ContainerProtos.EchoResponseProto response = + ContainerProtocolCalls.echo(clients.get(clientIndex), encodedContainerToken, + containerID, payloadReqBytes, payloadRespSize, sleepTimeMs); + return null; + }); + } +} + + diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/Freon.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/Freon.java index bd5510695fa1..349887a776d3 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/Freon.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/Freon.java @@ -73,7 +73,8 @@ OzoneClientKeyReadWriteListOps.class, RangeKeysGenerator.class, DatanodeSimulator.class, - OmMetadataGenerator.class + OmMetadataGenerator.class, + DNRPCLoadGenerator.class }, versionProvider = HddsVersionProvider.class, mixinStandardHelpOptions = true) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmRPCLoadGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmRPCLoadGenerator.java index 958df4c11a14..90807a0e6fe2 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmRPCLoadGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmRPCLoadGenerator.java @@ -19,7 +19,6 @@ import com.codahale.metrics.Timer; import com.google.common.base.Preconditions; -import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB; @@ -27,6 +26,8 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; +import static org.apache.hadoop.ozone.common.PayloadUtils.generatePayloadBytes; + /** * Utility to generate RPC request to OM with or without payload. */ @@ -88,8 +89,7 @@ public Void call() throws Exception { } init(); - payloadReqBytes = RandomUtils.nextBytes( - calculateMaxPayloadSize(payloadReqSizeKB)); + payloadReqBytes = generatePayloadBytes(payloadReqSizeKB); payloadRespSize = calculateMaxPayloadSize(payloadRespSizeKB); timer = getMetrics().timer("rpc-payload"); try { diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/containergenerator/GeneratorDatanode.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/containergenerator/GeneratorDatanode.java index 3a43ddd8ab09..dbca12c8b26d 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/containergenerator/GeneratorDatanode.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/containergenerator/GeneratorDatanode.java @@ -60,7 +60,6 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl; import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerFactory; -import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; import com.codahale.metrics.Timer; @@ -111,6 +110,7 @@ public class GeneratorDatanode extends BaseGenerator { private int overlap; private ChunkManager chunkManager; + private BlockManagerImpl blockManager; private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy; @@ -133,7 +133,7 @@ public Void call() throws Exception { config = createOzoneConfiguration(); - BlockManager blockManager = new BlockManagerImpl(config); + blockManager = new BlockManagerImpl(config); chunkManager = ChunkManagerFactory .createChunkManager(config, blockManager, null); @@ -286,7 +286,7 @@ public void generateData(long index) throws Exception { writtenBytes += currentChunkSize; } - BlockManagerImpl.persistPutBlock(container, blockData, config, true); + blockManager.persistPutBlock(container, blockData, true); }