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 064ce6ef83e5..091d2a83dd33 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 @@ -66,6 +66,37 @@ public enum ChecksumCombineMode { tags = ConfigTag.CLIENT) private int streamBufferSize = 4 * 1024 * 1024; + @Config(key = "datastream.buffer.flush.size", + defaultValue = "16MB", + type = ConfigType.SIZE, + description = "The boundary at which putBlock is executed", + tags = ConfigTag.CLIENT) + private long dataStreamBufferFlushSize = 16 * 1024 * 1024; + + @Config(key = "datastream.min.packet.size", + defaultValue = "1MB", + type = ConfigType.SIZE, + description = "The maximum size of the ByteBuffer " + + "(used via ratis streaming)", + tags = ConfigTag.CLIENT) + private int dataStreamMinPacketSize = 1024 * 1024; + + @Config(key = "datastream.window.size", + defaultValue = "64MB", + type = ConfigType.SIZE, + description = "Maximum size of BufferList(used for retry) size per " + + "BlockDataStreamOutput instance", + tags = ConfigTag.CLIENT) + private long streamWindowSize = 64 * 1024 * 1024; + + @Config(key = "datastream.pipeline.mode", + defaultValue = "true", + description = "Streaming write support both pipeline mode(datanode1->" + + "datanode2->datanode3) and star mode(datanode1->datanode2, " + + "datanode1->datanode3). By default we use pipeline mode.", + tags = ConfigTag.CLIENT) + private boolean datastreamPipelineMode = true; + @Config(key = "stream.buffer.increment", defaultValue = "0B", type = ConfigType.SIZE, @@ -210,6 +241,22 @@ public void setStreamBufferMaxSize(long streamBufferMaxSize) { this.streamBufferMaxSize = streamBufferMaxSize; } + public int getDataStreamMinPacketSize() { + return dataStreamMinPacketSize; + } + + public void setDataStreamMinPacketSize(int dataStreamMinPacketSize) { + this.dataStreamMinPacketSize = dataStreamMinPacketSize; + } + + public long getStreamWindowSize() { + return streamWindowSize; + } + + public void setStreamWindowSize(long streamWindowSize) { + this.streamWindowSize = streamWindowSize; + } + public int getMaxRetryCount() { return maxRetryCount; } @@ -254,6 +301,14 @@ public int getBufferIncrement() { return bufferIncrement; } + public long getDataStreamBufferFlushSize() { + return dataStreamBufferFlushSize; + } + + public void setDataStreamBufferFlushSize(long dataStreamBufferFlushSize) { + this.dataStreamBufferFlushSize = dataStreamBufferFlushSize; + } + public ChecksumCombineMode getChecksumCombineMode() { try { return ChecksumCombineMode.valueOf(checksumCombineMode); @@ -265,4 +320,12 @@ public ChecksumCombineMode getChecksumCombineMode() { ChecksumCombineMode.COMPOSITE_CRC.name()); } } + + public boolean isDatastreamPipelineMode() { + return datastreamPipelineMode; + } + + public void setDatastreamPipelineMode(boolean datastreamPipelineMode) { + this.datastreamPipelineMode = datastreamPipelineMode; + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java index 6982d41fbce5..2ad6b690644e 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java @@ -53,6 +53,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.ratis.client.RaftClient; +import org.apache.ratis.client.api.DataStreamApi; import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.exceptions.GroupMismatchException; @@ -121,7 +122,7 @@ private XceiverClientRatis(Pipeline pipeline, RpcType rpcType, this.ozoneConfiguration = configuration; } - private void updateCommitInfosMap( + public void updateCommitInfosMap( Collection commitInfoProtos) { // if the commitInfo map is empty, just update the commit indexes for each // of the servers @@ -359,4 +360,8 @@ public XceiverClientReply sendCommandAsync( throw new UnsupportedOperationException( "Operation Not supported for ratis client"); } + + public DataStreamApi getDataStreamApi() { + return this.getClient().getDataStreamApi(); + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java index 07fd0a8c2d48..b5c27c2e79b4 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hdds.scm.client; +import java.io.IOException; +import java.io.InterruptedIOException; import java.text.ParseException; import java.time.Instant; import java.time.ZoneId; @@ -28,10 +30,12 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; 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.function.ConsumerWithIOException; import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.io.retry.RetryPolicies; @@ -271,6 +275,22 @@ public static Throwable checkForException(Exception e) { return t; } + /** + * Checks if the provided exception signifies retry failure in ratis client. + * In case of retry failure, ratis client throws RaftRetryFailureException + * and all succeeding operations are failed with AlreadyClosedException. + */ + public static boolean checkForRetryFailure(Throwable t) { + return t instanceof RaftRetryFailureException + || t instanceof AlreadyClosedException; + } + + // Every container specific exception from datatnode will be seen as + // StorageContainerException + public static boolean checkIfContainerToExclude(Throwable t) { + return t instanceof StorageContainerException; + } + public static RetryPolicy createRetryPolicy(int maxRetryCount, long retryInterval) { // retry with fixed sleep between retries @@ -298,6 +318,54 @@ RetryPolicy> getRetryPolicyByException(int maxRetryCount, return policyMap; } + public static void streamRetryHandle( + IOException exception, + Map, RetryPolicy> retryPolicyMap, + AtomicInteger retryCount, + ConsumerWithIOException setExceptionAndThrow) + throws IOException { + RetryPolicy retryPolicy = retryPolicyMap + .get(HddsClientUtils.checkForException(exception).getClass()); + if (retryPolicy == null) { + retryPolicy = retryPolicyMap.get(Exception.class); + } + RetryPolicy.RetryAction action = null; + try { + action = retryPolicy.shouldRetry(exception, retryCount.get(), 0, true); + } catch (Exception e) { + setExceptionAndThrow.accept(new IOException(e)); + } + if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) { + String msg = ""; + if (action.reason != null) { + msg = "Retry request failed. " + action.reason; + //LOG.error(msg, exception); + } + setExceptionAndThrow.accept(new IOException(msg, exception)); + } + + // Throw the exception if the thread is interrupted + if (Thread.currentThread().isInterrupted()) { + //LOG.warn("Interrupted while trying for retry"); + setExceptionAndThrow.accept(exception); + } + Preconditions.checkArgument( + action.action == RetryPolicy.RetryAction.RetryDecision.RETRY); + if (action.delayMillis > 0) { + try { + Thread.sleep(action.delayMillis); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + IOException ioe = (IOException) new InterruptedIOException( + "Interrupted: action=" + action + ", retry policy=" + retryPolicy) + .initCause(e); + setExceptionAndThrow.accept(ioe); + } + } + retryCount.incrementAndGet(); + } + + public static List> getExceptionList() { return EXCEPTION_LIST; } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java new file mode 100644 index 000000000000..d5b9dd9d81c2 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java @@ -0,0 +1,691 @@ +/* + * 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.hdds.scm.storage; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +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.ContainerCommandResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue; +import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage; +import org.apache.hadoop.hdds.ratis.RatisHelper; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientMetrics; +import org.apache.hadoop.hdds.scm.XceiverClientRatis; +import org.apache.hadoop.hdds.scm.XceiverClientReply; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.ozone.common.Checksum; +import org.apache.hadoop.ozone.common.ChecksumData; +import org.apache.hadoop.ozone.common.OzoneChecksumException; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.ratis.client.api.DataStreamOutput; +import org.apache.ratis.io.StandardWriteOption; +import org.apache.ratis.protocol.DataStreamReply; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync; + +/** + * An {@link ByteBufferStreamOutput} used by the REST service in combination + * with the SCMClient to write the value of a key to a sequence + * of container chunks. Writes are buffered locally and periodically written to + * the container as a new chunk. In order to preserve the semantics that + * replacement of a pre-existing key is atomic, each instance of the stream has + * an internal unique identifier. This unique identifier and a monotonically + * increasing chunk index form a composite key that is used as the chunk name. + * After all data is written, a putKey call creates or updates the corresponding + * container key, and this call includes the full list of chunks that make up + * the key data. The list of chunks is updated all at once. Therefore, a + * concurrent reader never can see an intermediate state in which different + * chunks of data from different versions of the key data are interleaved. + * This class encapsulates all state management for buffering and writing + * through to the container. + */ +public class BlockDataStreamOutput implements ByteBufferStreamOutput { + public static final Logger LOG = + LoggerFactory.getLogger(BlockDataStreamOutput.class); + public static final String EXCEPTION_MSG = + "Unexpected Storage Container Exception: "; + private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {}; + + private AtomicReference blockID; + + private final BlockData.Builder containerBlockData; + private XceiverClientFactory xceiverClientFactory; + private XceiverClientRatis xceiverClient; + private OzoneClientConfig config; + + private int chunkIndex; + private final AtomicLong chunkOffset = new AtomicLong(); + + // Similar to 'BufferPool' but this list maintains only references + // to the ByteBuffers. + private List bufferList; + + // The IOException will be set by response handling thread in case there is an + // exception received in the response. If the exception is set, the next + // request will fail upfront. + private final AtomicReference ioException; + private final ExecutorService responseExecutor; + + // the effective length of data flushed so far + private long totalDataFlushedLength; + + // effective data write attempted so far for the block + private long writtenDataLength; + + // This object will maintain the commitIndexes and byteBufferList in order + // Also, corresponding to the logIndex, the corresponding list of buffers will + // be released from the buffer pool. + private final StreamCommitWatcher commitWatcher; + + private Queue> + putBlockFutures = new LinkedList<>(); + + private final List failedServers; + private final Checksum checksum; + + //number of buffers used before doing a flush/putBlock. + private int flushPeriod; + private final Token token; + private final DataStreamOutput out; + private CompletableFuture dataStreamCloseReply; + private List> futures = new ArrayList<>(); + private final long syncSize = 0; // TODO: disk sync is disabled for now + private long syncPosition = 0; + private StreamBuffer currentBuffer; + private XceiverClientMetrics metrics; + // buffers for which putBlock is yet to be executed + private List buffersForPutBlock; + private boolean isDatastreamPipelineMode; + /** + * Creates a new BlockDataStreamOutput. + * + * @param blockID block ID + * @param xceiverClientManager client manager that controls client + * @param pipeline pipeline where block will be written + */ + public BlockDataStreamOutput( + BlockID blockID, + XceiverClientFactory xceiverClientManager, + Pipeline pipeline, + OzoneClientConfig config, + Token token, + List bufferList + ) throws IOException { + this.xceiverClientFactory = xceiverClientManager; + this.config = config; + this.isDatastreamPipelineMode = config.isDatastreamPipelineMode(); + this.blockID = new AtomicReference<>(blockID); + KeyValue keyValue = + KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build(); + this.containerBlockData = + BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf()) + .addMetadata(keyValue); + this.xceiverClient = + (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline); + // Alternatively, stream setup can be delayed till the first chunk write. + this.out = setupStream(pipeline); + this.token = token; + this.bufferList = bufferList; + flushPeriod = (int) (config.getStreamBufferFlushSize() / config + .getStreamBufferSize()); + + Preconditions + .checkArgument( + (long) flushPeriod * config.getStreamBufferSize() == config + .getStreamBufferFlushSize()); + + // A single thread executor handle the responses of async requests + responseExecutor = Executors.newSingleThreadExecutor(); + commitWatcher = new StreamCommitWatcher(xceiverClient, bufferList); + totalDataFlushedLength = 0; + writtenDataLength = 0; + failedServers = new ArrayList<>(0); + ioException = new AtomicReference<>(null); + checksum = new Checksum(config.getChecksumType(), + config.getBytesPerChecksum()); + metrics = XceiverClientManager.getXceiverClientMetrics(); + } + + private DataStreamOutput setupStream(Pipeline pipeline) throws IOException { + // Execute a dummy WriteChunk request to get the path of the target file, + // but does NOT write any data to it. + ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest = + ContainerProtos.WriteChunkRequestProto.newBuilder() + .setBlockID(blockID.get().getDatanodeBlockIDProtobuf()); + + String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); + ContainerProtos.ContainerCommandRequestProto.Builder builder = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.StreamInit) + .setContainerID(blockID.get().getContainerID()) + .setDatanodeUuid(id).setWriteChunk(writeChunkRequest); + + ContainerCommandRequestMessage message = + ContainerCommandRequestMessage.toMessage(builder.build(), null); + + if (isDatastreamPipelineMode) { + return Preconditions.checkNotNull(xceiverClient.getDataStreamApi()) + .stream(message.getContent().asReadOnlyByteBuffer(), + RatisHelper.getRoutingTable(pipeline)); + } else { + return Preconditions.checkNotNull(xceiverClient.getDataStreamApi()) + .stream(message.getContent().asReadOnlyByteBuffer()); + } + } + + public BlockID getBlockID() { + return blockID.get(); + } + + public long getWrittenDataLength() { + return writtenDataLength; + } + + public List getFailedServers() { + return failedServers; + } + + @VisibleForTesting + public XceiverClientRatis getXceiverClient() { + return xceiverClient; + } + + public IOException getIoException() { + return ioException.get(); + } + + @Override + public void write(ByteBuffer b, int off, int len) throws IOException { + checkOpen(); + if (b == null) { + throw new NullPointerException(); + } + if (len == 0) { + return; + } + while (len > 0) { + allocateNewBufferIfNeeded(); + int writeLen = Math.min(len, currentBuffer.length()); + final StreamBuffer buf = new StreamBuffer(b, off, writeLen); + currentBuffer.put(buf); + writeChunkIfNeeded(); + off += writeLen; + writtenDataLength += writeLen; + len -= writeLen; + doFlushIfNeeded(); + } + } + + private void writeChunkIfNeeded() throws IOException { + if (currentBuffer.length() == 0) { + writeChunk(currentBuffer); + currentBuffer = null; + } + } + + private void writeChunk(StreamBuffer sb) throws IOException { + bufferList.add(sb); + if (buffersForPutBlock == null) { + buffersForPutBlock = new ArrayList<>(); + } + buffersForPutBlock.add(sb); + ByteBuffer dup = sb.duplicate(); + dup.position(0); + dup.limit(sb.position()); + writeChunkToContainer(dup); + } + + private void allocateNewBufferIfNeeded() { + if (currentBuffer == null) { + currentBuffer = + StreamBuffer.allocate(config.getDataStreamMinPacketSize()); + } + } + + private void doFlushIfNeeded() throws IOException { + long boundary = config.getDataStreamBufferFlushSize() / config + .getDataStreamMinPacketSize(); + // streamWindow is the maximum number of buffers that + // are allowed to exist in the bufferList. If buffers in + // the list exceed this limit , client will till it gets + // one putBlockResponse (first index) . This is similar to + // the bufferFull condition in async write path. + long streamWindow = config.getStreamWindowSize() / config + .getDataStreamMinPacketSize(); + if (!bufferList.isEmpty() && bufferList.size() % boundary == 0 && + buffersForPutBlock != null && !buffersForPutBlock.isEmpty()) { + updateFlushLength(); + executePutBlock(false, false); + } + if (bufferList.size() == streamWindow) { + try { + checkOpen(); + if (!putBlockFutures.isEmpty()) { + putBlockFutures.remove().get(); + } + } catch (ExecutionException e) { + handleExecutionException(e); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + handleInterruptedException(ex, true); + } + watchForCommit(true); + } + } + + private void updateFlushLength() { + totalDataFlushedLength = writtenDataLength; + } + + @VisibleForTesting + public long getTotalDataFlushedLength() { + return totalDataFlushedLength; + } + /** + * Will be called on the retryPath in case closedContainerException/ + * TimeoutException. + * @param len length of data to write + * @throws IOException if error occurred + */ + + public void writeOnRetry(long len) throws IOException { + if (len == 0) { + return; + } + if (LOG.isDebugEnabled()) { + LOG.debug("Retrying write length {} for blockID {}", len, blockID); + } + int count = 0; + while (len > 0) { + final StreamBuffer buf = bufferList.get(count); + final long writeLen = Math.min(buf.position(), len); + if (buffersForPutBlock == null) { + buffersForPutBlock = new ArrayList<>(); + } + buffersForPutBlock.add(buf); + final ByteBuffer duplicated = buf.duplicate(); + duplicated.position(0); + duplicated.limit(buf.position()); + writeChunkToContainer(duplicated); + len -= writeLen; + count++; + writtenDataLength += writeLen; + } + + + } + + /** + * calls watchForCommit API of the Ratis Client. For Standalone client, + * it is a no op. + * @param bufferFull flag indicating whether bufferFull condition is hit or + * its called as part flush/close + * @return minimum commit index replicated to all nodes + * @throws IOException IOException in case watch gets timed out + */ + private void watchForCommit(boolean bufferFull) throws IOException { + checkOpen(); + try { + XceiverClientReply reply = bufferFull ? + commitWatcher.streamWatchOnFirstIndex() : + commitWatcher.streamWatchOnLastIndex(); + if (reply != null) { + List dnList = reply.getDatanodes(); + if (!dnList.isEmpty()) { + Pipeline pipe = xceiverClient.getPipeline(); + + LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}", + blockID, pipe, dnList); + failedServers.addAll(dnList); + } + } + } catch (IOException ioe) { + setIoException(ioe); + throw getIoException(); + } + + } + + /** + * @param close whether putBlock is happening as part of closing the stream + * @param force true if no data was written since most recent putBlock and + * stream is being closed + */ + private void executePutBlock(boolean close, + boolean force) throws IOException { + checkOpen(); + long flushPos = totalDataFlushedLength; + final List byteBufferList; + if (!force) { + Preconditions.checkNotNull(bufferList); + byteBufferList = buffersForPutBlock; + buffersForPutBlock = null; + Preconditions.checkNotNull(byteBufferList); + } else { + byteBufferList = null; + } + waitFuturesComplete(); + if (close) { + dataStreamCloseReply = out.closeAsync(); + } + + try { + BlockData blockData = containerBlockData.build(); + XceiverClientReply asyncReply = + putBlockAsync(xceiverClient, blockData, close, token); + final CompletableFuture flushFuture + = asyncReply.getResponse().thenApplyAsync(e -> { + try { + validateResponse(e); + } catch (IOException sce) { + throw new CompletionException(sce); + } + // 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() + + " commitMap size " + + commitWatcher.getCommitInfoMapSize() + " flushLength " + + flushPos + " blockID " + blockID); + } + // for standalone protocol, logIndex will always be 0. + commitWatcher + .updateCommitInfoMap(asyncReply.getLogIndex(), + byteBufferList); + } + return e; + }, responseExecutor).exceptionally(e -> { + if (LOG.isDebugEnabled()) { + LOG.debug("putBlock failed for blockID {} with exception {}", + blockID, e.getLocalizedMessage()); + } + CompletionException ce = new CompletionException(e); + setIoException(ce); + throw ce; + }); + putBlockFutures.add(flushFuture); + } catch (IOException | ExecutionException e) { + throw new IOException(EXCEPTION_MSG + e.toString(), e); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + handleInterruptedException(ex, false); + } + } + + @Override + public void flush() throws IOException { + if (xceiverClientFactory != null && xceiverClient != null + && !config.isStreamBufferFlushDelay()) { + waitFuturesComplete(); + } + } + + public void waitFuturesComplete() throws IOException { + try { + CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get(); + futures.clear(); + } catch (Exception e) { + LOG.warn("Failed to write all chunks through stream: " + e); + throw new IOException(e); + } + } + + /** + * @param close whether the flush is happening as part of closing the stream + */ + private void handleFlush(boolean close) + throws IOException, InterruptedException, ExecutionException { + checkOpen(); + // flush the last chunk data residing on the currentBuffer + if (totalDataFlushedLength < writtenDataLength) { + // 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 + + if (currentBuffer != null) { + writeChunk(currentBuffer); + currentBuffer = null; + } + 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 + executePutBlock(true, true); + } + CompletableFuture.allOf(putBlockFutures.toArray(EMPTY_FUTURE_ARRAY)).get(); + watchForCommit(false); + // just check again if the exception is hit while waiting for the + // futures to ensure flush has indeed succeeded + + // irrespective of whether the commitIndex2flushedDataMap is empty + // or not, ensure there is no exception set + checkOpen(); + } + + @Override + public void close() throws IOException { + if (xceiverClientFactory != null && xceiverClient != null) { + try { + handleFlush(true); + dataStreamCloseReply.get(); + } catch (ExecutionException e) { + handleExecutionException(e); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + handleInterruptedException(ex, true); + } finally { + cleanup(false); + } + + } + } + + private void validateResponse( + ContainerProtos.ContainerCommandResponseProto responseProto) + throws IOException { + try { + // if the ioException is already set, it means a prev request has failed + // just throw the exception. The current operation will fail with the + // original error + IOException exception = getIoException(); + if (exception != null) { + throw exception; + } + ContainerProtocolCalls.validateContainerResponse(responseProto); + } catch (StorageContainerException sce) { + setIoException(sce); + throw sce; + } + } + + + private void setIoException(Exception e) { + IOException ioe = getIoException(); + if (ioe == null) { + IOException exception = new IOException(EXCEPTION_MSG + e.toString(), e); + ioException.compareAndSet(null, exception); + } else { + LOG.debug("Previous request had already failed with " + ioe.toString() + + " so subsequent request also encounters" + + " Storage Container Exception ", e); + } + } + + public void cleanup(boolean invalidateClient) { + if (xceiverClientFactory != null) { + xceiverClientFactory.releaseClient(xceiverClient, invalidateClient); + } + xceiverClientFactory = null; + xceiverClient = null; + commitWatcher.cleanup(); + responseExecutor.shutdown(); + } + + /** + * Checks if the stream is open or exception has occurred. + * If not, throws an exception. + * + * @throws IOException if stream is closed + */ + private void checkOpen() throws IOException { + if (isClosed()) { + throw new IOException("BlockDataStreamOutput has been closed."); + } else if (getIoException() != null) { + throw getIoException(); + } + } + + public boolean isClosed() { + return xceiverClient == null; + } + + private boolean needSync(long position) { + if (syncSize > 0) { + // TODO: or position >= fileLength + if (position - syncPosition >= syncSize) { + syncPosition = position; + return true; + } + } + return false; + } + + /** + * Writes buffered data as a new chunk to the container and saves chunk + * information to be used later in putKey call. + * + * @param buf chunk data to write, from position to limit + * @throws IOException if there is an I/O error while performing the call + * @throws OzoneChecksumException if there is an error while computing + * checksum + */ + private void writeChunkToContainer(ByteBuffer buf) + throws IOException { + final int effectiveChunkSize = buf.remaining(); + final long offset = chunkOffset.getAndAdd(effectiveChunkSize); + ChecksumData checksumData = checksum.computeChecksum( + buf.asReadOnlyBuffer()); + ChunkInfo chunkInfo = ChunkInfo.newBuilder() + .setChunkName(blockID.get().getLocalID() + "_chunk_" + ++chunkIndex) + .setOffset(offset) + .setLen(effectiveChunkSize) + .setChecksumData(checksumData.getProtoBufMessage()) + .build(); + metrics.incrPendingContainerOpsMetrics(ContainerProtos.Type.WriteChunk); + + if (LOG.isDebugEnabled()) { + LOG.debug("Writing chunk {} length {} at offset {}", + chunkInfo.getChunkName(), effectiveChunkSize, offset); + } + + CompletableFuture future = + (needSync(offset + effectiveChunkSize) ? + out.writeAsync(buf, StandardWriteOption.SYNC) : + out.writeAsync(buf)) + .whenCompleteAsync((r, e) -> { + if (e != null || !r.isSuccess()) { + if (e == null) { + e = new IOException("result is not success"); + } + 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; + } else if (r.isSuccess()) { + xceiverClient.updateCommitInfosMap(r.getCommitInfos()); + } + }, responseExecutor); + + futures.add(future); + containerBlockData.addChunks(chunkInfo); + } + + @VisibleForTesting + public void setXceiverClient(XceiverClientRatis xceiverClient) { + this.xceiverClient = xceiverClient; + } + + /** + * Handles InterruptedExecution. + * + * @param ex + * @param processExecutionException is optional, if passed as TRUE, then + * handle ExecutionException else skip it. + * @throws IOException + */ + private void handleInterruptedException(Exception ex, + boolean processExecutionException) + throws IOException { + LOG.error("Command execution was interrupted."); + if (processExecutionException) { + handleExecutionException(ex); + } else { + throw new IOException(EXCEPTION_MSG + ex.toString(), ex); + } + } + + /** + * Handles ExecutionException by adjusting buffers. + * @param ex + * @throws IOException + */ + private void handleExecutionException(Exception ex) throws IOException { + setIoException(ex); + throw getIoException(); + } + + public long getTotalAckDataLength() { + return commitWatcher.getTotalAckDataLength(); + } +} diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStreamOutput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStreamOutput.java new file mode 100644 index 000000000000..0650a685b634 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStreamOutput.java @@ -0,0 +1,57 @@ +/* + * 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.hdds.scm.storage; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** +* This interface is for writing an output stream of ByteBuffers. +* An ByteBufferStreamOutput accepts nio ByteBuffer and sends them to some sink. +*/ +public interface ByteBufferStreamOutput extends Closeable { + /** + * Try to write all the bytes in ByteBuf b to DataStream. + * + * @param b the data. + * @exception IOException if an I/O error occurs. + */ + default void write(ByteBuffer b) throws IOException { + write(b, b.position(), b.remaining()); + } + + /** + * Try to write the [off:off + len) slice in ByteBuf b to DataStream. + * + * @param b the data. + * @param off the start offset in the data. + * @param len the number of bytes to write. + * @exception IOException if an I/O error occurs. + */ + void write(ByteBuffer b, int off, int len) throws IOException; + + /** + * Flushes this DataStream output and forces any buffered output bytes + * to be written out. + * + * @exception IOException if an I/O error occurs. + */ + void flush() throws IOException; +} diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBuffer.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBuffer.java new file mode 100644 index 000000000000..d34e4dca9483 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBuffer.java @@ -0,0 +1,59 @@ +/* + * 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.hdds.scm.storage; + +import java.nio.ByteBuffer; + +/** + * Used for streaming write. + */ +public class StreamBuffer { + private final ByteBuffer buffer; + + public StreamBuffer(ByteBuffer buffer) { + this.buffer = buffer; + } + + public StreamBuffer(ByteBuffer buffer, int offset, int length) { + this((ByteBuffer) buffer.asReadOnlyBuffer().position(offset) + .limit(offset + length)); + } + + public ByteBuffer duplicate() { + return buffer.duplicate(); + } + + public int length() { + return buffer.limit() - buffer.position(); + } + + public int position() { + return buffer.position(); + } + + + public void put(StreamBuffer sb) { + buffer.put(sb.buffer); + } + + public static StreamBuffer allocate(int size) { + return new StreamBuffer(ByteBuffer.allocate(size)); + } + +} \ No newline at end of file diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamCommitWatcher.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamCommitWatcher.java new file mode 100644 index 000000000000..1820416d32fa --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamCommitWatcher.java @@ -0,0 +1,199 @@ +/* + * 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. + */ + +/** + * This class maintains the map of the commitIndexes to be watched for + * successful replication in the datanodes in a given pipeline. It also releases + * the buffers associated with the user data back to {@Link BufferPool} once + * minimum replication criteria is achieved during an ozone key write. + */ +package org.apache.hadoop.hdds.scm.storage; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.scm.XceiverClientReply; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * This class executes watchForCommit on ratis pipeline and releases + * buffers once data successfully gets replicated. + */ +public class StreamCommitWatcher { + + private static final Logger LOG = + LoggerFactory.getLogger(StreamCommitWatcher.class); + + private Map> commitIndexMap; + private List bufferList; + + // total data which has been successfully flushed and acknowledged + // by all servers + private long totalAckDataLength; + + private XceiverClientSpi xceiverClient; + + public StreamCommitWatcher(XceiverClientSpi xceiverClient, + List bufferList) { + this.xceiverClient = xceiverClient; + commitIndexMap = new ConcurrentSkipListMap<>(); + this.bufferList = bufferList; + totalAckDataLength = 0; + } + + public void updateCommitInfoMap(long index, List buffers) { + commitIndexMap.computeIfAbsent(index, k -> new LinkedList<>()) + .addAll(buffers); + } + + int getCommitInfoMapSize() { + return commitIndexMap.size(); + } + + /** + * Calls watch for commit for the first index in commitIndex2flushedDataMap to + * the Ratis client. + * @return {@link XceiverClientReply} reply from raft client + * @throws IOException in case watchForCommit fails + */ + public XceiverClientReply streamWatchOnFirstIndex() throws IOException { + if (!commitIndexMap.isEmpty()) { + // wait for the first commit index in the commitIndex2flushedDataMap + // to get committed to all or majority of nodes in case timeout + // happens. + long index = + commitIndexMap.keySet().stream().mapToLong(v -> v).min() + .getAsLong(); + if (LOG.isDebugEnabled()) { + LOG.debug("waiting for first index {} to catch up", index); + } + return streamWatchForCommit(index); + } else { + return null; + } + } + + /** + * Calls watch for commit for the last index in commitIndex2flushedDataMap to + * the Ratis client. + * @return {@link XceiverClientReply} reply from raft client + * @throws IOException in case watchForCommit fails + */ + public XceiverClientReply streamWatchOnLastIndex() + throws IOException { + if (!commitIndexMap.isEmpty()) { + // wait for the commit index in the commitIndex2flushedDataMap + // to get committed to all or majority of nodes in case timeout + // happens. + long index = + commitIndexMap.keySet().stream().mapToLong(v -> v).max() + .getAsLong(); + if (LOG.isDebugEnabled()) { + LOG.debug("waiting for last flush Index {} to catch up", index); + } + return streamWatchForCommit(index); + } else { + return null; + } + } + + /** + * calls watchForCommit API of the Ratis Client. This method is for streaming + * and no longer requires releaseBuffers + * @param commitIndex log index to watch for + * @return minimum commit index replicated to all nodes + * @throws IOException IOException in case watch gets timed out + */ + public XceiverClientReply streamWatchForCommit(long commitIndex) + throws IOException { + final long index; + try { + XceiverClientReply reply = + xceiverClient.watchForCommit(commitIndex); + if (reply == null) { + index = 0; + } else { + index = reply.getLogIndex(); + } + adjustBuffers(index); + return reply; + } catch (InterruptedException e) { + // Re-interrupt the thread while catching InterruptedException + Thread.currentThread().interrupt(); + throw getIOExceptionForWatchForCommit(commitIndex, e); + } catch (TimeoutException | ExecutionException e) { + throw getIOExceptionForWatchForCommit(commitIndex, e); + } + } + + void releaseBuffersOnException() { + adjustBuffers(xceiverClient.getReplicatedMinCommitIndex()); + } + + private void adjustBuffers(long commitIndex) { + List keyList = commitIndexMap.keySet().stream() + .filter(p -> p <= commitIndex).collect(Collectors.toList()); + if (!keyList.isEmpty()) { + releaseBuffers(keyList); + } + } + + private long releaseBuffers(List indexes) { + Preconditions.checkArgument(!commitIndexMap.isEmpty()); + for (long index : indexes) { + Preconditions.checkState(commitIndexMap.containsKey(index)); + final List buffers = commitIndexMap.remove(index); + final long length = + buffers.stream().mapToLong(StreamBuffer::position).sum(); + totalAckDataLength += length; + for (StreamBuffer byteBuffer : buffers) { + bufferList.remove(byteBuffer); + } + } + return totalAckDataLength; + } + + public long getTotalAckDataLength() { + return totalAckDataLength; + } + + private IOException getIOExceptionForWatchForCommit(long commitIndex, + Exception e) { + LOG.warn("watchForCommit failed for index {}", commitIndex, e); + IOException ioException = new IOException( + "Unexpected Storage Container Exception: " + e.toString(), e); + releaseBuffersOnException(); + return ioException; + } + + public void cleanup() { + if (commitIndexMap != null) { + commitIndexMap.clear(); + } + commitIndexMap = null; + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/function/ConsumerWithIOException.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/function/ConsumerWithIOException.java new file mode 100644 index 000000000000..b56a2e8c387c --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/function/ConsumerWithIOException.java @@ -0,0 +1,35 @@ +/** + * 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.hdds.function; + +import java.io.IOException; + +/** + * Functional interface like java.util.function.Consumer but with + * checked exception. + */ +@FunctionalInterface +public interface ConsumerWithIOException { + + /** + * Performs this operation on the given argument. + * + * @param t the input argument + */ + void accept(T t) throws IOException; +} 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 25826f3e23d7..78a0eeb7c5ee 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 @@ -273,8 +273,10 @@ public synchronized Port getPort(Port.Name name) { return port; } } - // if no separate admin/server port, return single Ratis one for compat - if (name == Name.RATIS_ADMIN || name == Name.RATIS_SERVER) { + // if no separate admin/server/datastream port, return single Ratis one for + // compat + if (name == Name.RATIS_ADMIN || name == Name.RATIS_SERVER || + name == Name.RATIS_DATASTREAM) { return getPort(Name.RATIS); } return null; @@ -784,7 +786,8 @@ public static final class Port { * Ports that are supported in DataNode. */ public enum Name { - STANDALONE, RATIS, REST, REPLICATION, RATIS_ADMIN, RATIS_SERVER; + STANDALONE, RATIS, REST, REPLICATION, RATIS_ADMIN, RATIS_SERVER, + RATIS_DATASTREAM; public static final Set ALL_PORTS = ImmutableSet.copyOf( Name.values()); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java index c1cd865036fb..6e62ccbfee02 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java @@ -43,13 +43,16 @@ import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.datastream.SupportedDataStreamType; import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.grpc.GrpcTlsConfig; +import org.apache.ratis.netty.NettyConfigKeys; import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.RoutingTable; import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.rpc.RpcType; import org.apache.ratis.rpc.SupportedRpcType; @@ -118,7 +121,9 @@ private static RaftPeer.Builder raftPeerBuilderFor(DatanodeDetails dn) { .setId(toRaftPeerId(dn)) .setAddress(toRaftPeerAddress(dn, Port.Name.RATIS_SERVER)) .setAdminAddress(toRaftPeerAddress(dn, Port.Name.RATIS_ADMIN)) - .setClientAddress(toRaftPeerAddress(dn, Port.Name.RATIS)); + .setClientAddress(toRaftPeerAddress(dn, Port.Name.RATIS)) + .setDataStreamAddress( + toRaftPeerAddress(dn, Port.Name.RATIS_DATASTREAM)); } private static List toRaftPeers(Pipeline pipeline) { @@ -172,6 +177,7 @@ public static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline, ConfigurationSource ozoneConfiguration) throws IOException { return newRaftClient(rpcType, toRaftPeerId(pipeline.getLeaderNode()), + toRaftPeer(pipeline.getFirstNode()), newRaftGroup(RaftGroupId.valueOf(pipeline.getId().getId()), pipeline.getNodes()), retryPolicy, tlsConfig, ozoneConfiguration); } @@ -191,7 +197,7 @@ public static RaftClient newRaftClient(RaftPeer leader, public static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader, RetryPolicy retryPolicy, GrpcTlsConfig tlsConfig, ConfigurationSource configuration) { - return newRaftClient(rpcType, leader.getId(), + return newRaftClient(rpcType, leader.getId(), leader, newRaftGroup(Collections.singletonList(leader)), retryPolicy, tlsConfig, configuration); } @@ -199,14 +205,14 @@ public static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader, public static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader, RetryPolicy retryPolicy, ConfigurationSource ozoneConfiguration) { - return newRaftClient(rpcType, leader.getId(), + return newRaftClient(rpcType, leader.getId(), leader, newRaftGroup(Collections.singletonList(leader)), retryPolicy, null, ozoneConfiguration); } @SuppressWarnings("checkstyle:ParameterNumber") private static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader, - RaftGroup group, RetryPolicy retryPolicy, + RaftPeer primary, RaftGroup group, RetryPolicy retryPolicy, GrpcTlsConfig tlsConfig, ConfigurationSource ozoneConfiguration) { if (LOG.isTraceEnabled()) { LOG.trace("newRaftClient: {}, leader={}, group={}", @@ -214,6 +220,9 @@ private static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader, } final RaftProperties properties = new RaftProperties(); + RaftConfigKeys.DataStream.setType(properties, + SupportedDataStreamType.NETTY); + RaftConfigKeys.Rpc.setType(properties, rpcType); // Set the ratis client headers which are matching with regex. @@ -223,6 +232,7 @@ private static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader, .setRaftGroup(group) .setLeaderId(leader) .setProperties(properties) + .setPrimaryDataStreamServer(primary) .setRetryPolicy(retryPolicy); // TODO: GRPC TLS only for now, netty/hadoop RPC TLS support later. @@ -249,7 +259,8 @@ public static void createRaftClientProperties(ConfigurationSource ozoneConf, Map ratisClientConf = getDatanodeRatisPrefixProps(ozoneConf); ratisClientConf.forEach((key, val) -> { - if (isClientConfig(key) || isGrpcClientConfig(key)) { + if (isClientConfig(key) || isGrpcClientConfig(key) + || isNettyStreamConfig(key)) { raftProperties.set(key, val); } }); @@ -265,6 +276,15 @@ private static boolean isGrpcClientConfig(String key) { !key.startsWith(GrpcConfigKeys.Admin.PREFIX) && !key.startsWith(GrpcConfigKeys.Server.PREFIX); } + + private static boolean isNettyStreamConfig(String key) { + return key.startsWith(NettyConfigKeys.DataStream.PREFIX); + } + + private static boolean isStreamClientConfig(String key) { + return key.startsWith(RaftClientConfigKeys.DataStream.PREFIX); + } + /** * Set all server properties matching with prefix * {@link RatisHelper#HDDS_DATANODE_RATIS_PREFIX_KEY} in @@ -279,7 +299,8 @@ public static void createRaftServerProperties(ConfigurationSource ozoneConf, getDatanodeRatisPrefixProps(ozoneConf); ratisServerConf.forEach((key, val) -> { // Exclude ratis client configuration. - if (!isClientConfig(key)) { + if (isNettyStreamConfig(key) || isStreamClientConfig(key) || + !isClientConfig(key)) { raftProperties.set(key, val); } }); @@ -323,6 +344,37 @@ public static Long getMinReplicatedIndex( .min(Long::compareTo).orElse(null); } + public static RoutingTable getRoutingTable(Pipeline pipeline) { + RaftPeerId primaryId = null; + List raftPeers = new ArrayList<>(); + + for (DatanodeDetails dn : pipeline.getNodes()) { + final RaftPeerId raftPeerId = RaftPeerId.valueOf(dn.getUuidString()); + try { + if (dn == pipeline.getFirstNode()) { + primaryId = raftPeerId; + } + } catch (IOException e) { + LOG.error("Can not get FirstNode from the pipeline: {} with " + + "exception: {}", pipeline.toString(), e.getLocalizedMessage()); + return null; + } + raftPeers.add(raftPeerId); + } + + RoutingTable.Builder builder = RoutingTable.newBuilder(); + RaftPeerId previousId = primaryId; + for (RaftPeerId peerId : raftPeers) { + if (peerId.equals(primaryId)) { + continue; + } + builder.addSuccessor(previousId, peerId); + previousId = peerId; + } + + return builder.build(); + } + private static Class getClass(String name, Class xface) { try { 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 627c432d3c5d..e81b18abcb0a 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 @@ -57,6 +57,12 @@ public final class OzoneConfigKeys { public static final boolean DFS_CONTAINER_IPC_RANDOM_PORT_DEFAULT = false; + public static final String DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT = + "dfs.container.ratis.datastream.random.port"; + public static final boolean + DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT_DEFAULT = + false; + public static final String DFS_CONTAINER_CHUNK_WRITE_SYNC_KEY = "dfs.container.chunk.write.sync"; public static final boolean DFS_CONTAINER_CHUNK_WRITE_SYNC_DEFAULT = false; @@ -79,6 +85,25 @@ public final class OzoneConfigKeys { "dfs.container.ratis.server.port"; public static final int DFS_CONTAINER_RATIS_SERVER_PORT_DEFAULT = 9856; + /** + * Ratis Port where containers listen to datastream requests. + */ + public static final String DFS_CONTAINER_RATIS_DATASTREAM_ENABLE + = "dfs.container.ratis.datastream.enable"; + public static final boolean DFS_CONTAINER_RATIS_DATASTREAM_ENABLE_DEFAULT + = true; + public static final String DFS_CONTAINER_RATIS_DATASTREAM_PORT + = "dfs.container.ratis.datastream.port"; + public static final int DFS_CONTAINER_RATIS_DATASTREAM_PORT_DEFAULT + = 9855; + + /** + * Flag to enable ratis streaming on filesystem writes. + */ + public static final String OZONE_FS_DATASTREAM_ENABLE = + "ozone.fs.datastream.enable"; + public static final boolean OZONE_FS_DATASTREAM_ENABLE_DEFAULT = false; + /** * When set to true, allocate a random free port for ozone container, so that * a mini cluster is able to launch multiple containers on a node. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java index 1c87f2bdebad..73aff9ac830c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java @@ -38,7 +38,8 @@ public enum DNAction implements AuditAction { PUT_SMALL_FILE, GET_SMALL_FILE, CLOSE_CONTAINER, - GET_COMMITTED_BLOCK_LENGTH; + GET_COMMITTED_BLOCK_LENGTH, + STREAM_INIT; @Override public String getAction() { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java index 76f84c46ab5e..d300b9ef0e50 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java @@ -139,6 +139,11 @@ public ChecksumData computeChecksum(byte[] data) */ public ChecksumData computeChecksum(ByteBuffer data) throws OzoneChecksumException { + // If type is set to NONE, we do not need to compute the checksums. We also + // need to avoid unnecessary conversions. + if (checksumType == ChecksumType.NONE) { + return new ChecksumData(checksumType, bytesPerChecksum); + } if (!data.isReadOnly()) { data = data.asReadOnlyBuffer(); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java index a13f164eec62..4d7f0f37c4eb 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerCommandRequestPBHelper.java @@ -187,6 +187,7 @@ public static DNAction getAuditAction(Type cmdType) { case GetSmallFile : return DNAction.GET_SMALL_FILE; case CloseContainer : return DNAction.CLOSE_CONTAINER; case GetCommittedBlockLength : return DNAction.GET_COMMITTED_BLOCK_LENGTH; + case StreamInit : return DNAction.STREAM_INIT; default : LOG.debug("Invalid command type - {}", cmdType); return null; diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 24f0c454c2ff..121564dd6e81 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -53,6 +53,26 @@ OZONE, CONTAINER, MANAGEMENT The ipc port number of container. + + dfs.container.ratis.datastream.enable + true + OZONE, CONTAINER, RATIS, DATASTREAM + If enable datastream ipc of container. + + + dfs.container.ratis.datastream.port + 9855 + OZONE, CONTAINER, RATIS, DATASTREAM + The datastream port number of container. + + + dfs.container.ratis.datastream.random.port + false + OZONE, CONTAINER, RATIS, DATASTREAM + Allocates a random free port for ozone container datastream. + This is used only while running unit tests. + + dfs.container.ipc.random.port false @@ -3049,4 +3069,11 @@ will create intermediate directories. + + ozone.fs.datastream.enable + false + OZONE, DATANODE + To enable/disable filesystem write via ratis streaming. + + 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 86198cd4cba7..59cba8215085 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 @@ -566,6 +566,18 @@ public static String getFixedLengthString(String string, int length) { return String.format("%1$" + length + "s", string); } + public static byte[] generateData(int length, boolean random) { + final byte[] data = new byte[length]; + if (random) { + ThreadLocalRandom.current().nextBytes(data); + } else { + for (int i = 0; i < length; i++) { + data[i] = (byte) i; + } + } + return data; + } + /** * Construct fake protobuf messages for various types of requests. * This is tedious, however necessary to test. Protobuf classes are final diff --git a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java index 8cf584d75f61..3728a0b1f590 100644 --- a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java +++ b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java @@ -46,5 +46,6 @@ public enum ConfigTag { DELETION, HA, BALANCER, - UPGRADE + UPGRADE, + DATASTREAM } 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 1edd046f09a6..bc6c804d54d2 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 @@ -66,6 +66,7 @@ import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.malformedRequest; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; +import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.thirdparty.com.google.protobuf.ProtocolMessageEnum; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -199,7 +200,8 @@ private ContainerCommandResponseProto dispatchRequest( boolean isWriteStage = (cmdType == Type.WriteChunk && dispatcherContext != null && dispatcherContext.getStage() - == DispatcherContext.WriteChunkStage.WRITE_DATA); + == DispatcherContext.WriteChunkStage.WRITE_DATA) + || (cmdType == Type.StreamInit); boolean isWriteCommitStage = (cmdType == Type.WriteChunk && dispatcherContext != null && dispatcherContext.getStage() @@ -669,4 +671,21 @@ private boolean isAllowed(String action) { default: return false; } } + + @Override + public StateMachine.DataChannel getStreamDataChannel( + ContainerCommandRequestProto msg) + throws StorageContainerException { + long containerID = msg.getContainerID(); + Container container = getContainer(containerID); + if (container != null) { + Handler handler = getHandler(getContainerType(container)); + return handler.getStreamDataChannel(container, msg); + } else { + throw new StorageContainerException( + "ContainerID " + containerID + " does not exist", + ContainerProtos.Result.CONTAINER_NOT_FOUND); + } + } + } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java index a2e397d54615..d02bae0a35ad 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java @@ -25,6 +25,7 @@ .ContainerCommandResponseProto; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; +import org.apache.ratis.statemachine.StateMachine; import java.util.Map; @@ -84,4 +85,13 @@ void validateContainerCommand( * @param clusterId */ void setClusterId(String clusterId); + + /** + * When uploading using stream, get StreamDataChannel. + */ + default StateMachine.DataChannel getStreamDataChannel( + ContainerCommandRequestProto msg) throws StorageContainerException { + throw new UnsupportedOperationException( + "getStreamDataChannel not supported."); + } } 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 1dbd588d33cb..7401aa545df1 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 @@ -36,6 +36,7 @@ import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker; +import org.apache.ratis.statemachine.StateMachine; /** * Dispatcher sends ContainerCommandRequests to Handler. Each Container Type @@ -80,6 +81,10 @@ public static Handler getHandlerForContainerType( } } + public abstract StateMachine.DataChannel getStreamDataChannel( + Container container, ContainerCommandRequestProto msg) + throws StorageContainerException; + /** * Returns the Id of this datanode. * 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 494cfe1b136b..7cdca22f75c0 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 @@ -29,6 +29,7 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -61,6 +62,8 @@ import org.apache.hadoop.ozone.common.utils.BufferUtils; import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; +import org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel; +import org.apache.hadoop.ozone.container.keyvalue.impl.SmallFileStreamDataChannel; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; import org.apache.hadoop.util.Time; @@ -80,6 +83,7 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.storage.RaftStorage; +import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.StateMachineStorage; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.BaseStateMachine; @@ -90,6 +94,7 @@ import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat; import org.apache.ratis.util.TaskQueue; import org.apache.ratis.util.function.CheckedSupplier; +import org.apache.ratis.util.JavaUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -510,6 +515,84 @@ private CompletableFuture handleWriteChunk( return raftFuture; } + private StateMachine.DataChannel getStreamDataChannel( + ContainerCommandRequestProto requestProto, + DispatcherContext context) throws StorageContainerException { + if (LOG.isDebugEnabled()) { + LOG.debug("{}: getStreamDataChannel {} containerID={} pipelineID={} " + + "traceID={}", gid, requestProto.getCmdType(), + requestProto.getContainerID(), requestProto.getPipelineID(), + requestProto.getTraceID()); + } + runCommand(requestProto, context); // stream init + return dispatcher.getStreamDataChannel(requestProto); + } + + @Override + public CompletableFuture stream(RaftClientRequest request) { + return CompletableFuture.supplyAsync(() -> { + try { + ContainerCommandRequestProto requestProto = + message2ContainerCommandRequestProto(request.getMessage()); + DispatcherContext context = + new DispatcherContext.Builder() + .setStage(DispatcherContext.WriteChunkStage.WRITE_DATA) + .setContainer2BCSIDMap(container2BCSIDMap) + .build(); + DataChannel channel = getStreamDataChannel(requestProto, context); + final ExecutorService chunkExecutor = requestProto.hasWriteChunk() ? + getChunkExecutor(requestProto.getWriteChunk()) : null; + return new LocalStream(channel, chunkExecutor); + } catch (IOException e) { + throw new CompletionException("Failed to create data stream", e); + } + }, executor); + } + + @Override + public CompletableFuture link(DataStream stream, LogEntryProto entry) { + if (stream == null) { + return JavaUtils.completeExceptionally(new IllegalStateException( + "DataStream is null")); + } + final DataChannel dataChannel = stream.getDataChannel(); + if (dataChannel.isOpen()) { + return JavaUtils.completeExceptionally(new IllegalStateException( + "DataStream: " + stream + " is not closed properly")); + } + + final CompletableFuture f; + if (dataChannel instanceof SmallFileStreamDataChannel) { + f = runCommandAsync(((SmallFileStreamDataChannel) dataChannel) + .getPutBlockRequest(), entry); + } else if (dataChannel instanceof KeyValueStreamDataChannel) { + return CompletableFuture.completedFuture(null); + } else { + return JavaUtils.completeExceptionally(new IllegalStateException( + "Unexpected DataChannel " + dataChannel.getClass())); + } + return f.whenComplete((res, e) -> { + if (LOG.isDebugEnabled()) { + LOG.debug("PutBlock {} Term: {} Index: {}", + res.getResult(), entry.getTerm(), entry.getIndex()); + } + }); + } + + private CompletableFuture runCommandAsync( + ContainerCommandRequestProto requestProto, LogEntryProto entry) { + return CompletableFuture.supplyAsync(() -> { + final DispatcherContext context = new DispatcherContext.Builder() + .setTerm(entry.getTerm()) + .setLogIndex(entry.getIndex()) + .setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA) + .setContainer2BCSIDMap(container2BCSIDMap) + .build(); + + return runCommand(requestProto, context); + }, executor); + } + private ExecutorService getChunkExecutor(WriteChunkRequestProto req) { int i = (int)(req.getBlockID().getLocalID() % chunkExecutors.size()); return chunkExecutors.get(i); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java new file mode 100644 index 000000000000..780f8743988a --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/LocalStream.java @@ -0,0 +1,58 @@ +/* + * 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.transport.server.ratis; + +import org.apache.ratis.statemachine.StateMachine; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +class LocalStream implements StateMachine.DataStream { + private final StateMachine.DataChannel dataChannel; + private final Executor executor; + + LocalStream(StateMachine.DataChannel dataChannel, Executor executor) { + this.dataChannel = dataChannel; + this.executor = executor; + } + + @Override + public StateMachine.DataChannel getDataChannel() { + return dataChannel; + } + + @Override + public CompletableFuture cleanUp() { + return CompletableFuture.supplyAsync(() -> { + try { + dataChannel.close(); + return true; + } catch (IOException e) { + throw new CompletionException("Failed to close data channel", e); + } + }); + } + + @Override + public Executor getExecutor() { + return executor; + } +} \ No newline at end of file diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java index aad93422d8cf..d754fefe3b2f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java @@ -80,6 +80,7 @@ import org.apache.ratis.RaftConfigKeys; import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.datastream.SupportedDataStreamType; import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.netty.NettyConfigKeys; @@ -99,6 +100,7 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.rpc.RpcType; import org.apache.ratis.rpc.SupportedRpcType; +import org.apache.ratis.server.DataStreamServerRpc; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.RaftServerRpc; @@ -130,6 +132,7 @@ private static long nextCallId() { private int serverPort; private int adminPort; private int clientPort; + private int dataStreamPort; private final RaftServer server; private final List chunkExecutors; private final ContainerDispatcher dispatcher; @@ -149,6 +152,7 @@ private static long nextCallId() { // Timeout used while calling submitRequest directly. private long requestTimeout; private boolean shouldDeleteRatisLogDirectory; + private boolean streamEnable; private XceiverServerRatis(DatanodeDetails dd, ContainerDispatcher dispatcher, ContainerController containerController, @@ -158,6 +162,9 @@ private XceiverServerRatis(DatanodeDetails dd, Objects.requireNonNull(dd, "id == null"); datanodeDetails = dd; assignPorts(); + this.streamEnable = conf.getBoolean( + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_ENABLE, + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_ENABLE_DEFAULT); RaftProperties serverProperties = newRaftProperties(); this.context = context; this.dispatcher = dispatcher; @@ -214,6 +221,34 @@ private ContainerStateMachine getStateMachine(RaftGroupId gid) { chunkExecutors, this, conf); } + private void setUpRatisStream(RaftProperties properties) { + // set the datastream config + if (conf.getBoolean( + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT, + OzoneConfigKeys. + DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT_DEFAULT)) { + dataStreamPort = 0; + } else { + dataStreamPort = conf.getInt( + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_PORT, + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_PORT_DEFAULT); + } + NettyConfigKeys.DataStream.setPort(properties, dataStreamPort); + RaftConfigKeys.DataStream.setType(properties, + SupportedDataStreamType.NETTY); + int dataStreamAsyncRequestThreadPoolSize = + conf.getObject(DatanodeRatisServerConfig.class) + .getStreamRequestThreads(); + RaftServerConfigKeys.DataStream.setAsyncRequestThreadPoolSize(properties, + dataStreamAsyncRequestThreadPoolSize); + int dataStreamClientPoolSize = + conf.getObject(DatanodeRatisServerConfig.class) + .getClientPoolSize(); + RaftServerConfigKeys.DataStream.setClientPoolSize(properties, + dataStreamClientPoolSize); + } + + @SuppressWarnings("checkstyle:methodlength") private RaftProperties newRaftProperties() { final RaftProperties properties = new RaftProperties(); @@ -232,6 +267,10 @@ private RaftProperties newRaftProperties() { // set the configs enable and set the stateMachineData sync timeout RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true); + if (streamEnable) { + setUpRatisStream(properties); + } + timeUnit = OzoneConfigKeys. DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT.getUnit(); duration = conf.getTimeDuration( @@ -497,7 +536,12 @@ public void start() throws IOException { Port.Name.RATIS_ADMIN); serverPort = getRealPort(serverRpc.getInetSocketAddress(), Port.Name.RATIS_SERVER); - + if (streamEnable) { + DataStreamServerRpc dataStreamServerRpc = + server.getDataStreamServerRpc(); + dataStreamPort = getRealPort(dataStreamServerRpc.getInetSocketAddress(), + Port.Name.RATIS_DATASTREAM); + } isStarted = true; } } 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 73df5e4d95c0..19fe2309495d 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 @@ -97,11 +97,13 @@ import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getReadChunkResponse; 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.malformedRequest; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.putBlockResponseSuccess; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion; +import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -175,6 +177,37 @@ public VolumeChoosingPolicy getVolumeChoosingPolicyForTesting() { return volumeChoosingPolicy; } + @Override + public StateMachine.DataChannel getStreamDataChannel( + Container container, ContainerCommandRequestProto msg) + throws StorageContainerException { + KeyValueContainer kvContainer = (KeyValueContainer) container; + checkContainerOpen(kvContainer); + + if (msg.hasWriteChunk()) { + BlockID blockID = + BlockID.getFromProtobuf(msg.getWriteChunk().getBlockID()); + + return chunkManager.getStreamDataChannel(kvContainer, + blockID, false, 0, metrics); + } else if (msg.hasPutSmallFile()) { + PutSmallFileRequestProto putSmallFile = msg.getPutSmallFile(); + BlockID blockID = BlockID.getFromProtobuf( + putSmallFile.getBlock().getBlockData().getBlockID()); + + return chunkManager.getStreamDataChannel(kvContainer, + blockID, true, putSmallFile.getBlock().getBlockData().getSize(), + metrics); + } else { + LOG.error( + "Get stream data channel error Malformed request " + + "containerID: {} msg: {}", + container.getContainerData().getContainerID(), msg); + throw new StorageContainerException("Malformed request.", + ContainerProtos.Result.IO_EXCEPTION); + } + } + @Override public void stop() { } @@ -222,6 +255,8 @@ static ContainerCommandResponseProto dispatchRequest(KeyValueHandler handler, return handler.handleDeleteChunk(request, kvContainer); case WriteChunk: return handler.handleWriteChunk(request, kvContainer, dispatcherContext); + case StreamInit: + return handler.handleStreamInit(request, kvContainer, dispatcherContext); case ListChunk: return handler.handleUnsupportedOp(request); case CompactChunk: @@ -248,6 +283,40 @@ public BlockManager getBlockManager() { return this.blockManager; } + ContainerCommandResponseProto handleStreamInit( + ContainerCommandRequestProto request, KeyValueContainer kvContainer, + DispatcherContext dispatcherContext) { + + BlockID blockID; + if (request.hasWriteChunk()) { + WriteChunkRequestProto writeChunk = request.getWriteChunk(); + blockID = BlockID.getFromProtobuf(writeChunk.getBlockID()); + } else if (request.hasPutSmallFile()) { + PutSmallFileRequestProto putSmallFile = request.getPutSmallFile(); + blockID = BlockID + .getFromProtobuf(putSmallFile.getBlock().getBlockData().getBlockID()); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Malformed {} request. trace ID: {}", + request.getCmdType(), request.getTraceID()); + } + return malformedRequest(request); + } + + String path = null; + try { + checkContainerOpen(kvContainer); + path = chunkManager + .streamInit(kvContainer, blockID); + } catch (StorageContainerException ex) { + return ContainerUtils.logAndReturnError(LOG, ex, request); + } + + return getSuccessResponseBuilder(request) + .setMessage(path) + .build(); + } + /** * Handles Create Container Request. If successful, adds the container to * ContainerSet and sends an ICR to the SCM. 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 763647313b8c..63b2c2c50aa3 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 @@ -25,6 +25,7 @@ import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.common.volume.VolumeSet; @@ -33,6 +34,7 @@ import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.ratis.statemachine.StateMachine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,6 +75,21 @@ public void writeChunk(Container container, BlockID blockID, ChunkInfo info, .writeChunk(container, blockID, info, data, dispatcherContext); } + public String streamInit(Container container, BlockID blockID) + throws StorageContainerException { + return selectHandler(container) + .streamInit(container, blockID); + } + + @Override + public StateMachine.DataChannel getStreamDataChannel( + Container container, BlockID blockID, boolean isSmallFile, long len, + ContainerMetrics metrics) + throws StorageContainerException { + return selectHandler(container) + .getStreamDataChannel(container, blockID, isSmallFile, len, metrics); + } + @Override public void finishWriteChunks(KeyValueContainer kvContainer, BlockData blockData) throws IOException { 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 18c6b9d28d5f..ff6035d6e1a8 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 @@ -32,6 +32,7 @@ import org.apache.hadoop.ozone.common.utils.BufferUtils; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; @@ -42,6 +43,7 @@ import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.ratis.statemachine.StateMachine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,12 +78,15 @@ public class FilePerBlockStrategy implements ChunkManager { private final long defaultReadBufferCapacity; private final VolumeSet volumeSet; + private final BlockManager blockManager; + public FilePerBlockStrategy(boolean sync, BlockManager manager, VolumeSet volSet) { doSyncWrite = sync; this.defaultReadBufferCapacity = manager == null ? 0 : manager.getDefaultReadBufferCapacity(); this.volumeSet = volSet; + this.blockManager = manager; } private static void checkLayoutVersion(Container container) { @@ -89,6 +94,30 @@ private static void checkLayoutVersion(Container container) { container.getContainerData().getLayoutVersion() == FILE_PER_BLOCK); } + @Override + public String streamInit(Container container, BlockID blockID) + throws StorageContainerException { + checkLayoutVersion(container); + File chunkFile = getChunkFile(container, blockID, null); + return chunkFile.getAbsolutePath(); + } + + @Override + public StateMachine.DataChannel getStreamDataChannel( + Container container, BlockID blockID, boolean isSmallFile, long len, + ContainerMetrics metrics) + throws StorageContainerException { + checkLayoutVersion(container); + File chunkFile = getChunkFile(container, blockID, null); + + if (isSmallFile) { + return new SmallFileStreamDataChannel(chunkFile, container, len, metrics); + } + + return new KeyValueStreamDataChannel(chunkFile, + container.getContainerData(), metrics); + } + @Override public void writeChunk(Container container, BlockID blockID, ChunkInfo info, ChunkBuffer data, DispatcherContext dispatcherContext) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyValueStreamDataChannel.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyValueStreamDataChannel.java new file mode 100644 index 000000000000..66723031f069 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyValueStreamDataChannel.java @@ -0,0 +1,42 @@ +/* + * 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.keyvalue.impl; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.impl.ContainerData; + +import java.io.File; + +/** + * This class is used to get the DataChannel for streaming. + */ +public class KeyValueStreamDataChannel extends StreamDataChannelBase { + KeyValueStreamDataChannel(File file, ContainerData containerData, + ContainerMetrics metrics) + throws StorageContainerException { + super(file, containerData, metrics); + } + + @Override + ContainerProtos.Type getType() { + return ContainerProtos.Type.StreamWrite; + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/SmallFileStreamDataChannel.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/SmallFileStreamDataChannel.java new file mode 100644 index 000000000000..9d08145b2926 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/SmallFileStreamDataChannel.java @@ -0,0 +1,147 @@ +/* + * 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.keyvalue.impl; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +/** + * This class is used to get the DataChannel for streaming. + */ +public class SmallFileStreamDataChannel extends StreamDataChannelBase { + public static final Logger LOG = + LoggerFactory.getLogger(SmallFileStreamDataChannel.class); + + private final Container kvContainer; + + private final int realLen; + private int writeLen = 0; + private final List metadata = new ArrayList<>(); + private int metadataLen = 0; + private ContainerProtos.ContainerCommandRequestProto putBlockRequest; + + SmallFileStreamDataChannel(File file, Container container, long dataLen, + ContainerMetrics metrics) + throws StorageContainerException { + super(file, container.getContainerData(), metrics); + this.kvContainer = container; + this.realLen = (int) dataLen; + } + + @Override + ContainerProtos.Type getType() { + return ContainerProtos.Type.PutSmallFile; + } + + @Override + public int write(ByteBuffer src) throws IOException { + int srcLen = src.capacity(); + + if (srcLen == 0) { + return 0; + } else if (writeLen + srcLen > realLen) { + + if (metadataLen > 0) { + metadataLen += srcLen; + metadata.add(src); + } else { + metadataLen += (writeLen + srcLen - realLen); + + int dataLen = srcLen - metadataLen; + byte[] data = new byte[dataLen]; + src.get(data, 0, dataLen); + super.write(ByteBuffer.wrap(data)); + + byte[] meta = new byte[metadataLen]; + src.get(meta, dataLen, metadataLen); + metadata.add(ByteBuffer.wrap(meta)); + } + } else { + super.write(src); + } + writeLen += srcLen; + return srcLen; + } + + private ByteString asByteString() { + ByteBuffer buffer = ByteBuffer.allocate(metadataLen); + for (ByteBuffer b : metadata) { + buffer.put(b); + } + buffer.flip(); + return ByteString.copyFrom(buffer); + } + + @Override + public void close() throws IOException { + super.close(); + + if (writeLen <= realLen || metadataLen <= 0) { + String msg = "Put small file write length mismatch realLen: " + + realLen + " writeLen: " + writeLen + " metadataLen: " + metadataLen; + throw new StorageContainerException(msg, + ContainerProtos.Result.PUT_SMALL_FILE_ERROR); + } + + ContainerProtos.ContainerCommandRequestProto request = + ContainerCommandRequestMessage.toProto(asByteString(), null); + + if (!request.hasPutSmallFile()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Malformed Put Small File request. trace ID: {}", + request.getTraceID()); + } + throw new StorageContainerException("Malformed Put Small File request.", + ContainerProtos.Result.PUT_SMALL_FILE_ERROR); + } + + ContainerProtos.PutSmallFileRequestProto putSmallFileReq = + request.getPutSmallFile(); + + ContainerProtos.PutBlockRequestProto.Builder createBlockRequest = + ContainerProtos.PutBlockRequestProto.newBuilder() + .setBlockData(putSmallFileReq.getBlock().getBlockData()) + .setEof(true); + + ContainerProtos.ContainerCommandRequestProto.Builder builder = + ContainerProtos.ContainerCommandRequestProto + .newBuilder().setCmdType(ContainerProtos.Type.PutBlock) + .setContainerID(kvContainer.getContainerData().getContainerID()) + .setDatanodeUuid(kvContainer.getContainerData().getOriginNodeId()) + .setPutBlock(createBlockRequest); + + putBlockRequest = builder.build(); + } + + public ContainerProtos.ContainerCommandRequestProto getPutBlockRequest() { + return putBlockRequest; + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/StreamDataChannelBase.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/StreamDataChannelBase.java new file mode 100644 index 000000000000..b31e2ccbf413 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/StreamDataChannelBase.java @@ -0,0 +1,97 @@ +/* + * 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.keyvalue.impl; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.impl.ContainerData; +import org.apache.ratis.statemachine.StateMachine; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; + +/** + * For write state machine data. + */ +abstract class StreamDataChannelBase implements StateMachine.DataChannel { + private final RandomAccessFile randomAccessFile; + + private final File file; + + private final ContainerData containerData; + private final ContainerMetrics metrics; + + StreamDataChannelBase(File file, ContainerData containerData, + ContainerMetrics metrics) + throws StorageContainerException { + try { + this.file = file; + this.randomAccessFile = new RandomAccessFile(file, "rw"); + } catch (FileNotFoundException e) { + throw new StorageContainerException("BlockFile not exists with " + + "container Id " + containerData.getContainerID() + + " file " + file.getAbsolutePath(), + ContainerProtos.Result.IO_EXCEPTION); + } + this.containerData = containerData; + this.metrics = metrics; + } + + abstract ContainerProtos.Type getType(); + + private FileChannel getChannel() { + return randomAccessFile.getChannel(); + } + + @Override + public final void force(boolean metadata) throws IOException { + getChannel().force(metadata); + } + + @Override + public final boolean isOpen() { + return getChannel().isOpen(); + } + + @Override + public void close() throws IOException { + randomAccessFile.close(); + } + + @Override + public int write(ByteBuffer src) throws IOException { + final int writeBytes = getChannel().write(src); + metrics.incContainerBytesStats(getType(), writeBytes); + containerData.updateWriteStats(writeBytes, false); + return writeBytes; + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + + "File=" + file.getAbsolutePath() + + ", containerID=" + containerData.getContainerID() + + '}'; + } +} 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 15ff9d6b9d61..e90c99336052 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 @@ -25,9 +25,11 @@ import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; +import org.apache.ratis.statemachine.StateMachine; import java.io.IOException; import java.nio.ByteBuffer; @@ -104,6 +106,18 @@ default void finishWriteChunks(KeyValueContainer kvContainer, // no-op } + default String streamInit(Container container, BlockID blockID) + throws StorageContainerException { + return null; + } + + default StateMachine.DataChannel getStreamDataChannel( + Container container, BlockID blockID, boolean isSmallFile, long len, + ContainerMetrics metrics) + throws StorageContainerException { + return null; + } + static long getBufferCapacityForChunkRead(ChunkInfo chunkInfo, long defaultReadBufferCapacity) { long bufferCapacity = 0; diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java index 1337f28ad945..bb1145bb2b2d 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java @@ -81,6 +81,8 @@ public void setUp() throws Exception { TimeUnit.MILLISECONDS); conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, true); conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, true); + conf.setBoolean( + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT, true); serverAddresses = new ArrayList<>(); scmServers = new ArrayList<>(); mockServers = new ArrayList<>(); @@ -215,7 +217,6 @@ public void testDatanodeStateContext() throws IOException, OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT); datanodeDetails.setPort(port); ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath); - try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(datanodeDetails, conf, null, null, null)) { @@ -424,6 +425,8 @@ private DatanodeDetails getNewDatanodeDetails() { DatanodeDetails.Port.Name.RATIS, 0); DatanodeDetails.Port restPort = DatanodeDetails.newPort( DatanodeDetails.Port.Name.REST, 0); + DatanodeDetails.Port streamPort = DatanodeDetails.newPort( + DatanodeDetails.Port.Name.RATIS_DATASTREAM, 0); return DatanodeDetails.newBuilder() .setUuid(UUID.randomUUID()) .setHostName("localhost") @@ -431,6 +434,7 @@ private DatanodeDetails getNewDatanodeDetails() { .addPort(containerPort) .addPort(ratisPort) .addPort(restPort) + .addPort(streamPort) .build(); } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCreatePipelineCommandHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCreatePipelineCommandHandler.java index d23f1c49bf13..ce6264056e8d 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCreatePipelineCommandHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCreatePipelineCommandHandler.java @@ -38,6 +38,7 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.protocol.ClientId; import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.retry.RetryPolicy; import org.junit.Before; @@ -98,6 +99,8 @@ private RaftClient.Builder mockRaftClientBuilder() { .thenReturn(builder); Mockito.when(builder.setRetryPolicy(Mockito.any(RetryPolicy.class))) .thenReturn(builder); + Mockito.when(builder.setPrimaryDataStreamServer( + Mockito.any(RaftPeer.class))).thenReturn(builder); return builder; } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/conf/DatanodeRatisServerConfig.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/conf/DatanodeRatisServerConfig.java index 25ed4776b7d8..058932e76902 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/conf/DatanodeRatisServerConfig.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/conf/DatanodeRatisServerConfig.java @@ -23,6 +23,7 @@ import java.time.Duration; import static org.apache.hadoop.hdds.conf.ConfigTag.DATANODE; +import static org.apache.hadoop.hdds.conf.ConfigTag.DATASTREAM; import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE; import static org.apache.hadoop.hdds.conf.ConfigTag.PERFORMANCE; import static org.apache.hadoop.hdds.conf.ConfigTag.RATIS; @@ -123,6 +124,40 @@ public void setLeaderNumPendingRequests(int leaderNumPendingRequests) { this.leaderNumPendingRequests = leaderNumPendingRequests; } + @Config(key = "datastream.request.threads", + defaultValue = "20", + type = ConfigType.INT, + tags = {OZONE, DATANODE, RATIS, DATASTREAM}, + description = "Maximum number of threads in the thread pool for " + + "datastream request." + ) + private int streamRequestThreads; + + public int getStreamRequestThreads() { + return streamRequestThreads; + } + + public void setStreamRequestThreads(int streamRequestThreads) { + this.streamRequestThreads = streamRequestThreads; + } + + @Config(key = "datastream.client.pool.size", + defaultValue = "10", + type = ConfigType.INT, + tags = {OZONE, DATANODE, RATIS, DATASTREAM}, + description = "Maximum number of client proxy in NettyServerStreamRpc " + + "for datastream write." + ) + private int clientPoolSize; + + public int getClientPoolSize() { + return clientPoolSize; + } + + public void setClientPoolSize(int clientPoolSize) { + this.clientPoolSize = clientPoolSize; + } + @Config(key = "delete.ratis.log.directory", defaultValue = "true", type = ConfigType.BOOLEAN, diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 01c9a1e0b4a8..7bdd3cc45d0c 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -100,6 +100,9 @@ enum Type { GetSmallFile = 16; CloseContainer = 17; GetCommittedBlockLength = 18; + + StreamInit = 19; + StreamWrite = 20; } @@ -392,7 +395,7 @@ enum ChecksumType { message WriteChunkRequestProto { required DatanodeBlockID blockID = 1; - required ChunkInfo chunkData = 2; + optional ChunkInfo chunkData = 2; optional bytes data = 3; } @@ -448,7 +451,7 @@ RPC */ message PutSmallFileRequestProto { required PutBlockRequestProto block = 1; required ChunkInfo chunkInfo = 2; - required bytes data = 3; + optional bytes data = 3; } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java index c0ff64669360..a5fff4797f51 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java @@ -145,6 +145,8 @@ public void testGetVersionTask() throws Exception { try (EndpointStateMachine rpcEndPoint = createEndpoint(conf, serverAddress, 1000)) { DatanodeDetails datanodeDetails = randomDatanodeDetails(); + conf.setBoolean( + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT, true); OzoneContainer ozoneContainer = new OzoneContainer( datanodeDetails, conf, getContext(datanodeDetails), null); rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION); @@ -169,6 +171,8 @@ public void testCheckVersionResponse() throws Exception { true); conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, true); + conf.setBoolean( + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT, true); conf.setFromObject(new ReplicationConfig().setPort(0)); try (EndpointStateMachine rpcEndPoint = createEndpoint(conf, serverAddress, 1000)) { diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java index a292ae263f75..5c3dae546613 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.ozone.OmUtils; +import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.client.protocol.ClientProtocol; @@ -557,6 +558,24 @@ public OzoneOutputStream createKey(String key, long size, .createKey(volumeName, name, key, size, replicationConfig, keyMetadata); } + /** + * Creates a new key in the bucket. + * + * @param key Name of the key to be created. + * @param size Size of the data the key will point to. + * @param replicationConfig Replication configuration. + * @return OzoneDataStreamOutput to which the data has to be written. + * @throws IOException + */ + public OzoneDataStreamOutput createStreamKey(String key, long size, + ReplicationConfig replicationConfig, + Map keyMetadata) + throws IOException { + return proxy + .createStreamKey(volumeName, name, key, size, replicationConfig, + keyMetadata); + } + /** * Reads an existing key from the bucket. * @@ -748,6 +767,21 @@ public OzoneOutputStream createMultipartKey(String key, long size, uploadID); } + /** + * Create a part key for a multipart upload key. + * @param key + * @param size + * @param partNumber + * @param uploadID + * @return OzoneDataStreamOutput + * @throws IOException + */ + public OzoneDataStreamOutput createMultipartStreamKey(String key, + long size, int partNumber, String uploadID) throws IOException { + return proxy.createMultipartStreamKey(volumeName, name, + key, size, partNumber, uploadID); + } + /** * Complete Multipart upload. This will combine all the parts and make the * key visible in ozone. @@ -878,6 +912,14 @@ public OzoneOutputStream createFile(String keyName, long size, overWrite, recursive); } + public OzoneDataStreamOutput createStreamFile(String keyName, long size, + ReplicationConfig replicationConfig, boolean overWrite, + boolean recursive) throws IOException { + return proxy + .createStreamFile(volumeName, name, keyName, size, replicationConfig, + overWrite, recursive); + } + /** * List the status for a file or a directory and its contents. * diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockDataStreamOutputEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockDataStreamOutputEntry.java new file mode 100644 index 000000000000..4e5a35a539ce --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockDataStreamOutputEntry.java @@ -0,0 +1,287 @@ +/* + * 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.io; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput; +import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput; +import org.apache.hadoop.hdds.scm.storage.StreamBuffer; +import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; +import org.apache.hadoop.security.token.Token; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * Helper class used inside {@link BlockDataStreamOutput}. + * */ +public final class BlockDataStreamOutputEntry + implements ByteBufferStreamOutput { + + private final OzoneClientConfig config; + private ByteBufferStreamOutput byteBufferStreamOutput; + private BlockID blockID; + private final String key; + private final XceiverClientFactory xceiverClientManager; + private final Pipeline pipeline; + // total number of bytes that should be written to this stream + private final long length; + // the current position of this stream 0 <= currentPosition < length + private long currentPosition; + private final Token token; + private List bufferList; + + @SuppressWarnings({"parameternumber", "squid:S00107"}) + private BlockDataStreamOutputEntry( + BlockID blockID, String key, + XceiverClientFactory xceiverClientManager, + Pipeline pipeline, + long length, + Token token, + OzoneClientConfig config, + List bufferList + ) { + this.config = config; + this.byteBufferStreamOutput = null; + this.blockID = blockID; + this.key = key; + this.xceiverClientManager = xceiverClientManager; + this.pipeline = pipeline; + this.token = token; + this.length = length; + this.currentPosition = 0; + this.bufferList = bufferList; + } + + long getLength() { + return length; + } + + Token getToken() { + return token; + } + + long getRemaining() { + return length - currentPosition; + } + + /** + * BlockDataStreamOutput is initialized in this function. This makes sure that + * xceiverClient initialization is not done during preallocation and only + * done when data is written. + * @throws IOException if xceiverClient initialization fails + */ + private void checkStream() throws IOException { + if (this.byteBufferStreamOutput == null) { + this.byteBufferStreamOutput = + new BlockDataStreamOutput(blockID, xceiverClientManager, pipeline, + config, token, bufferList); + } + } + + @Override + public void write(ByteBuffer b, int off, int len) throws IOException { + checkStream(); + byteBufferStreamOutput.write(b, off, len); + this.currentPosition += len; + } + + @Override + public void flush() throws IOException { + if (this.byteBufferStreamOutput != null) { + this.byteBufferStreamOutput.flush(); + } + } + + @Override + public void close() throws IOException { + if (this.byteBufferStreamOutput != null) { + this.byteBufferStreamOutput.close(); + // after closing the chunkOutPutStream, blockId would have been + // reconstructed with updated bcsId + this.blockID = + ((BlockDataStreamOutput) byteBufferStreamOutput).getBlockID(); + } + } + + boolean isClosed() { + if (byteBufferStreamOutput != null) { + return ((BlockDataStreamOutput) byteBufferStreamOutput).isClosed(); + } + return false; + } + + Collection getFailedServers() { + if (byteBufferStreamOutput != null) { + BlockDataStreamOutput out = + (BlockDataStreamOutput) this.byteBufferStreamOutput; + return out.getFailedServers(); + } + return Collections.emptyList(); + } + + long getWrittenDataLength() { + if (byteBufferStreamOutput != null) { + BlockDataStreamOutput out = + (BlockDataStreamOutput) this.byteBufferStreamOutput; + return out.getWrittenDataLength(); + } else { + // For a pre allocated block for which no write has been initiated, + // the ByteBufferStreamOutput will be null here. + // In such cases, the default blockCommitSequenceId will be 0 + return 0; + } + } + + public long getTotalAckDataLength() { + if (byteBufferStreamOutput != null) { + BlockDataStreamOutput out = + (BlockDataStreamOutput) this.byteBufferStreamOutput; + blockID = out.getBlockID(); + return out.getTotalAckDataLength(); + } else { + // For a pre allocated block for which no write has been initiated, + // the OutputStream will be null here. + // In such cases, the default blockCommitSequenceId will be 0 + return 0; + } + } + + void cleanup(boolean invalidateClient) throws IOException { + checkStream(); + BlockDataStreamOutput out = + (BlockDataStreamOutput) this.byteBufferStreamOutput; + out.cleanup(invalidateClient); + + } + + void writeOnRetry(long len) throws IOException { + checkStream(); + BlockDataStreamOutput out = + (BlockDataStreamOutput) this.byteBufferStreamOutput; + out.writeOnRetry(len); + this.currentPosition += len; + + } + + /** + * Builder class for BlockDataStreamOutputEntry. + * */ + public static class Builder { + + private BlockID blockID; + private String key; + private XceiverClientFactory xceiverClientManager; + private Pipeline pipeline; + private long length; + private Token token; + private OzoneClientConfig config; + private List bufferList; + + public Builder setBlockID(BlockID bID) { + this.blockID = bID; + return this; + } + + public Builder setKey(String keys) { + this.key = keys; + return this; + } + + public Builder setXceiverClientManager( + XceiverClientFactory + xClientManager) { + this.xceiverClientManager = xClientManager; + return this; + } + + public Builder setPipeline(Pipeline ppln) { + this.pipeline = ppln; + return this; + } + + + public Builder setLength(long len) { + this.length = len; + return this; + } + + public Builder setConfig(OzoneClientConfig clientConfig) { + this.config = clientConfig; + return this; + } + + public Builder setToken(Token bToken) { + this.token = bToken; + return this; + } + + public Builder setBufferList(List bList) { + this.bufferList = bList; + return this; + } + + public BlockDataStreamOutputEntry build() { + return new BlockDataStreamOutputEntry(blockID, + key, + xceiverClientManager, + pipeline, + length, + token, config, bufferList); + } + } + + @VisibleForTesting + public ByteBufferStreamOutput getByteBufStreamOutput() { + return byteBufferStreamOutput; + } + + public BlockID getBlockID() { + return blockID; + } + + public String getKey() { + return key; + } + + public XceiverClientFactory getXceiverClientManager() { + return xceiverClientManager; + } + + public Pipeline getPipeline() { + return pipeline; + } + + public long getCurrentPosition() { + return currentPosition; + } + + public void setCurrentPosition(long curPosition) { + this.currentPosition = curPosition; + } +} + + diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockDataStreamOutputEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockDataStreamOutputEntryPool.java new file mode 100644 index 000000000000..00cda7844ae9 --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockDataStreamOutputEntryPool.java @@ -0,0 +1,316 @@ + +/* + * 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.io; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; +import org.apache.hadoop.hdds.scm.pipeline.PipelineID; +import org.apache.hadoop.hdds.scm.storage.StreamBuffer; +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.OmMultipartCommitUploadPartInfo; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.ListIterator; + +/** + * This class manages the stream entries list and handles block allocation + * from OzoneManager. + */ +public class BlockDataStreamOutputEntryPool { + + public static final Logger LOG = + LoggerFactory.getLogger(BlockDataStreamOutputEntryPool.class); + + private final List streamEntries; + private final OzoneClientConfig config; + private int currentStreamIndex; + private final OzoneManagerProtocol omClient; + private final OmKeyArgs keyArgs; + private final XceiverClientFactory xceiverClientFactory; + private final String requestID; + private OmMultipartCommitUploadPartInfo commitUploadPartInfo; + private final long openID; + private final ExcludeList excludeList; + private List bufferList; + + @SuppressWarnings({"parameternumber", "squid:S00107"}) + public BlockDataStreamOutputEntryPool( + OzoneClientConfig config, + OzoneManagerProtocol omClient, + String requestId, ReplicationConfig replicationConfig, + String uploadID, int partNumber, + boolean isMultipart, OmKeyInfo info, + boolean unsafeByteBufferConversion, + XceiverClientFactory xceiverClientFactory, long openID + ) { + this.config = config; + this.xceiverClientFactory = xceiverClientFactory; + streamEntries = new ArrayList<>(); + currentStreamIndex = 0; + this.omClient = omClient; + this.keyArgs = new OmKeyArgs.Builder().setVolumeName(info.getVolumeName()) + .setBucketName(info.getBucketName()).setKeyName(info.getKeyName()) + .setReplicationConfig(replicationConfig).setDataSize(info.getDataSize()) + .setIsMultipartKey(isMultipart).setMultipartUploadID(uploadID) + .setMultipartUploadPartNumber(partNumber).build(); + this.requestID = requestId; + this.openID = openID; + this.excludeList = new ExcludeList(); + this.bufferList = new ArrayList<>(); + } + + /** + * A constructor for testing purpose only. + * + * @see KeyDataStreamOutput#KeyDataStreamOutput() + */ + @VisibleForTesting + BlockDataStreamOutputEntryPool() { + streamEntries = new ArrayList<>(); + omClient = null; + keyArgs = null; + xceiverClientFactory = null; + config = + new OzoneConfiguration().getObject(OzoneClientConfig.class); + config.setStreamBufferSize(0); + config.setStreamBufferMaxSize(0); + config.setStreamBufferFlushSize(0); + config.setStreamBufferFlushDelay(false); + requestID = null; + int chunkSize = 0; + currentStreamIndex = 0; + openID = -1; + excludeList = new ExcludeList(); + } + + /** + * When a key is opened, it is possible that there are some blocks already + * allocated to it for this open session. In this case, to make use of these + * blocks, we need to add these blocks to stream entries. But, a key's version + * also includes blocks from previous versions, we need to avoid adding these + * old blocks to stream entries, because these old blocks should not be picked + * for write. To do this, the following method checks that, only those + * blocks created in this particular open version are added to stream entries. + * + * @param version the set of blocks that are pre-allocated. + * @param openVersion the version corresponding to the pre-allocation. + * @throws IOException + */ + public void addPreallocateBlocks(OmKeyLocationInfoGroup version, + long openVersion) throws IOException { + // server may return any number of blocks, (0 to any) + // only the blocks allocated in this open session (block createVersion + // equals to open session version) + for (OmKeyLocationInfo subKeyInfo : version.getLocationList(openVersion)) { + addKeyLocationInfo(subKeyInfo); + } + } + + private void addKeyLocationInfo(OmKeyLocationInfo subKeyInfo) { + Preconditions.checkNotNull(subKeyInfo.getPipeline()); + BlockDataStreamOutputEntry.Builder builder = + new BlockDataStreamOutputEntry.Builder() + .setBlockID(subKeyInfo.getBlockID()) + .setKey(keyArgs.getKeyName()) + .setXceiverClientManager(xceiverClientFactory) + .setPipeline(subKeyInfo.getPipeline()) + .setConfig(config) + .setLength(subKeyInfo.getLength()) + .setToken(subKeyInfo.getToken()) + .setBufferList(bufferList); + streamEntries.add(builder.build()); + } + + public List getLocationInfoList() { + List locationInfoList = new ArrayList<>(); + for (BlockDataStreamOutputEntry streamEntry : streamEntries) { + long length = streamEntry.getCurrentPosition(); + + // Commit only those blocks to OzoneManager which are not empty + if (length != 0) { + OmKeyLocationInfo info = + new OmKeyLocationInfo.Builder().setBlockID(streamEntry.getBlockID()) + .setLength(streamEntry.getCurrentPosition()).setOffset(0) + .setToken(streamEntry.getToken()) + .setPipeline(streamEntry.getPipeline()).build(); + locationInfoList.add(info); + } + if (LOG.isDebugEnabled()) { + LOG.debug( + "block written " + streamEntry.getBlockID() + ", length " + length + + " bcsID " + streamEntry.getBlockID() + .getBlockCommitSequenceId()); + } + } + return locationInfoList; + } + + /** + * Discards the subsequent pre allocated blocks and removes the streamEntries + * from the streamEntries list for the container which is closed. + * @param containerID id of the closed container + * @param pipelineId id of the associated pipeline + */ + void discardPreallocatedBlocks(long containerID, PipelineID pipelineId) { + // currentStreamIndex < streamEntries.size() signifies that, there are still + // pre allocated blocks available. + + // This will be called only to discard the next subsequent unused blocks + // in the streamEntryList. + if (currentStreamIndex + 1 < streamEntries.size()) { + ListIterator streamEntryIterator = + streamEntries.listIterator(currentStreamIndex + 1); + while (streamEntryIterator.hasNext()) { + BlockDataStreamOutputEntry streamEntry = streamEntryIterator.next(); + Preconditions.checkArgument(streamEntry.getCurrentPosition() == 0); + if ((streamEntry.getPipeline().getId().equals(pipelineId)) || + (containerID != -1 && + streamEntry.getBlockID().getContainerID() == containerID)) { + streamEntryIterator.remove(); + } + } + } + } + + List getStreamEntries() { + return streamEntries; + } + + XceiverClientFactory getXceiverClientFactory() { + return xceiverClientFactory; + } + + String getKeyName() { + return keyArgs.getKeyName(); + } + + long getKeyLength() { + return streamEntries.stream().mapToLong( + BlockDataStreamOutputEntry::getCurrentPosition).sum(); + } + /** + * Contact OM to get a new block. Set the new block with the index (e.g. + * first block has index = 0, second has index = 1 etc.) + * + * The returned block is made to new BlockDataStreamOutputEntry to write. + * + * @throws IOException + */ + private void allocateNewBlock() throws IOException { + if (!excludeList.isEmpty()) { + LOG.debug("Allocating block with {}", excludeList); + } + OmKeyLocationInfo subKeyInfo = + omClient.allocateBlock(keyArgs, openID, excludeList); + addKeyLocationInfo(subKeyInfo); + } + + + void commitKey(long offset) throws IOException { + if (keyArgs != null) { + // in test, this could be null + long length = getKeyLength(); + Preconditions.checkArgument(offset == length); + keyArgs.setDataSize(length); + keyArgs.setLocationInfoList(getLocationInfoList()); + // When the key is multipart upload part file upload, we should not + // commit the key, as this is not an actual key, this is a just a + // partial key of a large file. + if (keyArgs.getIsMultipartKey()) { + commitUploadPartInfo = + omClient.commitMultipartUploadPart(keyArgs, openID); + } else { + omClient.commitKey(keyArgs, openID); + } + } else { + LOG.warn("Closing KeyDataStreamOutput, but key args is null"); + } + } + + public BlockDataStreamOutputEntry getCurrentStreamEntry() { + if (streamEntries.isEmpty() || streamEntries.size() <= currentStreamIndex) { + return null; + } else { + return streamEntries.get(currentStreamIndex); + } + } + + BlockDataStreamOutputEntry allocateBlockIfNeeded() throws IOException { + BlockDataStreamOutputEntry streamEntry = getCurrentStreamEntry(); + if (streamEntry != null && streamEntry.isClosed()) { + // a stream entry gets closed either by : + // a. If the stream gets full + // b. it has encountered an exception + currentStreamIndex++; + } + if (streamEntries.size() <= currentStreamIndex) { + Preconditions.checkNotNull(omClient); + // allocate a new block, if a exception happens, log an error and + // throw exception to the caller directly, and the write fails. + allocateNewBlock(); + } + // in theory, this condition should never violate due the check above + // still do a sanity check. + Preconditions.checkArgument(currentStreamIndex < streamEntries.size()); + return streamEntries.get(currentStreamIndex); + } + + void cleanup() { + if (excludeList != null) { + excludeList.clear(); + } + + if (streamEntries != null) { + streamEntries.clear(); + } + } + + public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() { + return commitUploadPartInfo; + } + + public ExcludeList getExcludeList() { + return excludeList; + } + + boolean isEmpty() { + return streamEntries.isEmpty(); + } + + long computeBufferData() { + long totalDataLen = 0; + for (StreamBuffer b : bufferList) { + totalDataLen += b.position(); + } + return totalDataLen; + } +} diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyDataStreamOutput.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyDataStreamOutput.java new file mode 100644 index 000000000000..759384f93e51 --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyDataStreamOutput.java @@ -0,0 +1,549 @@ +/* + * 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.io; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FileEncryptionInfo; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.client.HddsClientUtils; +import org.apache.hadoop.hdds.scm.container.ContainerID; +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.scm.storage.ByteBufferStreamOutput; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +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.OmMultipartCommitUploadPartInfo; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Maintaining a list of BlockInputStream. Write based on offset. + * + * Note that this may write to multiple containers in one write call. In case + * that first container succeeded but later ones failed, the succeeded writes + * are not rolled back. + * + * TODO : currently not support multi-thread access. + */ +public class KeyDataStreamOutput implements ByteBufferStreamOutput { + + private OzoneClientConfig config; + + /** + * Defines stream action while calling handleFlushOrClose. + */ + enum StreamAction { + FLUSH, CLOSE, FULL + } + + public static final Logger LOG = + LoggerFactory.getLogger(KeyDataStreamOutput.class); + + private boolean closed; + private FileEncryptionInfo feInfo; + private final Map, RetryPolicy> retryPolicyMap; + private AtomicInteger retryCount; + // how much of data is actually written yet to underlying stream + private long offset; + // how much data has been ingested into the stream + private long writeOffset; + // whether an exception is encountered while write and whole write could + // not succeed + private boolean isException; + private final BlockDataStreamOutputEntryPool blockDataStreamOutputEntryPool; + + private long clientID; + + /** + * A constructor for testing purpose only. + */ + @VisibleForTesting + public KeyDataStreamOutput() { + closed = false; + this.retryPolicyMap = HddsClientUtils.getExceptionList() + .stream() + .collect(Collectors.toMap(Function.identity(), + e -> RetryPolicies.TRY_ONCE_THEN_FAIL)); + retryCount = new AtomicInteger(0); + offset = 0; + blockDataStreamOutputEntryPool = new BlockDataStreamOutputEntryPool(); + } + + @VisibleForTesting + public List getStreamEntries() { + return blockDataStreamOutputEntryPool.getStreamEntries(); + } + + @VisibleForTesting + public XceiverClientFactory getXceiverClientFactory() { + return blockDataStreamOutputEntryPool.getXceiverClientFactory(); + } + + @VisibleForTesting + public List getLocationInfoList() { + return blockDataStreamOutputEntryPool.getLocationInfoList(); + } + + @VisibleForTesting + public int getRetryCount() { + return retryCount.get(); + } + + @VisibleForTesting + public long getClientID() { + return clientID; + } + + @SuppressWarnings({"parameternumber", "squid:S00107"}) + public KeyDataStreamOutput( + OzoneClientConfig config, + OpenKeySession handler, + XceiverClientFactory xceiverClientManager, + OzoneManagerProtocol omClient, int chunkSize, + String requestId, ReplicationConfig replicationConfig, + String uploadID, int partNumber, boolean isMultipart, + boolean unsafeByteBufferConversion + ) { + this.config = config; + OmKeyInfo info = handler.getKeyInfo(); + blockDataStreamOutputEntryPool = + new BlockDataStreamOutputEntryPool( + config, + omClient, + requestId, replicationConfig, + uploadID, partNumber, + isMultipart, info, + unsafeByteBufferConversion, + xceiverClientManager, + handler.getId()); + + // Retrieve the file encryption key info, null if file is not in + // encrypted bucket. + this.feInfo = info.getFileEncryptionInfo(); + this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException( + config.getMaxRetryCount(), config.getRetryInterval()); + this.retryCount = new AtomicInteger(0); + this.isException = false; + this.writeOffset = 0; + this.clientID = handler.getId(); + } + + /** + * When a key is opened, it is possible that there are some blocks already + * allocated to it for this open session. In this case, to make use of these + * blocks, we need to add these blocks to stream entries. But, a key's version + * also includes blocks from previous versions, we need to avoid adding these + * old blocks to stream entries, because these old blocks should not be picked + * for write. To do this, the following method checks that, only those + * blocks created in this particular open version are added to stream entries. + * + * @param version the set of blocks that are pre-allocated. + * @param openVersion the version corresponding to the pre-allocation. + * @throws IOException + */ + public void addPreallocateBlocks(OmKeyLocationInfoGroup version, + long openVersion) throws IOException { + blockDataStreamOutputEntryPool.addPreallocateBlocks(version, openVersion); + } + + @Override + public void write(ByteBuffer b, int off, int len) throws IOException { + checkNotClosed(); + if (b == null) { + throw new NullPointerException(); + } + handleWrite(b, off, len, false); + writeOffset += len; + } + + private void handleWrite(ByteBuffer b, int off, long len, boolean retry) + throws IOException { + while (len > 0) { + try { + BlockDataStreamOutputEntry current = + blockDataStreamOutputEntryPool.allocateBlockIfNeeded(); + // length(len) will be in int range if the call is happening through + // write API of blockDataStreamOutput. Length can be in long range + // if it comes via Exception path. + int expectedWriteLen = Math.min((int) len, + (int) current.getRemaining()); + long currentPos = current.getWrittenDataLength(); + // writeLen will be updated based on whether the write was succeeded + // or if it sees an exception, how much the actual write was + // acknowledged. + int writtenLength = + writeToDataStreamOutput(current, retry, len, b, + expectedWriteLen, off, currentPos); + if (current.getRemaining() <= 0) { + // since the current block is already written close the stream. + handleFlushOrClose(StreamAction.FULL); + } + len -= writtenLength; + off += writtenLength; + } catch (Exception e) { + markStreamClosed(); + throw new IOException(e); + } + } + } + + private int writeToDataStreamOutput(BlockDataStreamOutputEntry current, + boolean retry, long len, ByteBuffer b, int writeLen, int off, + long currentPos) throws IOException { + try { + if (retry) { + current.writeOnRetry(len); + } else { + current.write(b, off, writeLen); + offset += writeLen; + } + } catch (IOException ioe) { + // for the current iteration, totalDataWritten - currentPos gives the + // amount of data already written to the buffer + + // In the retryPath, the total data to be written will always be equal + // to or less than the max length of the buffer allocated. + // The len specified here is the combined sum of the data length of + // the buffers + Preconditions.checkState(!retry || len <= config + .getStreamBufferMaxSize()); + int dataWritten = (int) (current.getWrittenDataLength() - currentPos); + writeLen = retry ? (int) len : dataWritten; + // In retry path, the data written is already accounted in offset. + if (!retry) { + offset += writeLen; + } + LOG.debug("writeLen {}, total len {}", writeLen, len); + handleException(current, ioe); + } + return writeLen; + } + + /** + * It performs following actions : + * a. Updates the committed length at datanode for the current stream in + * datanode. + * b. Reads the data from the underlying buffer and writes it the next stream. + * + * @param streamEntry StreamEntry + * @param exception actual exception that occurred + * @throws IOException Throws IOException if Write fails + */ + private void handleException(BlockDataStreamOutputEntry streamEntry, + IOException exception) throws IOException { + Throwable t = HddsClientUtils.checkForException(exception); + Preconditions.checkNotNull(t); + boolean retryFailure = HddsClientUtils.checkForRetryFailure(t); + boolean containerExclusionException = false; + if (!retryFailure) { + containerExclusionException = + HddsClientUtils.checkIfContainerToExclude(t); + } + Pipeline pipeline = streamEntry.getPipeline(); + PipelineID pipelineId = pipeline.getId(); + long totalSuccessfulFlushedData = streamEntry.getTotalAckDataLength(); + //set the correct length for the current stream + streamEntry.setCurrentPosition(totalSuccessfulFlushedData); + long containerId = streamEntry.getBlockID().getContainerID(); + Collection failedServers = streamEntry.getFailedServers(); + Preconditions.checkNotNull(failedServers); + ExcludeList excludeList = blockDataStreamOutputEntryPool.getExcludeList(); + long bufferedDataLen = blockDataStreamOutputEntryPool.computeBufferData(); + if (!failedServers.isEmpty()) { + excludeList.addDatanodes(failedServers); + } + + // if the container needs to be excluded , add the container to the + // exclusion list , otherwise add the pipeline to the exclusion list + if (containerExclusionException) { + excludeList.addConatinerId(ContainerID.valueOf(containerId)); + } else { + excludeList.addPipeline(pipelineId); + } + // just clean up the current stream. + streamEntry.cleanup(retryFailure); + + // discard all subsequent blocks the containers and pipelines which + // are in the exclude list so that, the very next retry should never + // write data on the closed container/pipeline + if (containerExclusionException) { + // discard subsequent pre allocated blocks from the streamEntries list + // from the closed container + blockDataStreamOutputEntryPool + .discardPreallocatedBlocks(streamEntry.getBlockID().getContainerID(), + null); + } else { + // In case there is timeoutException or Watch for commit happening over + // majority or the client connection failure to the leader in the + // pipeline, just discard all the pre allocated blocks on this pipeline. + // Next block allocation will happen with excluding this specific pipeline + // This will ensure if 2 way commit happens , it cannot span over multiple + // blocks + blockDataStreamOutputEntryPool + .discardPreallocatedBlocks(-1, pipelineId); + } + if (bufferedDataLen > 0) { + // If the data is still cached in the underlying stream, we need to + // allocate new block and write this data in the datanode. + HddsClientUtils.streamRetryHandle(exception, retryPolicyMap, retryCount, + this::setExceptionAndThrow); + + handleWrite(null, 0, bufferedDataLen, true); + // reset the retryCount after handling the exception + retryCount.set(0); + } + } + + private void markStreamClosed() { + blockDataStreamOutputEntryPool.cleanup(); + closed = true; + } + + private void setExceptionAndThrow(IOException ioe) throws IOException { + isException = true; + throw ioe; + } + + @Override + public void flush() throws IOException { + checkNotClosed(); + handleFlushOrClose(StreamAction.FLUSH); + } + + /** + * Close or Flush the latest outputStream depending upon the action. + * This function gets called when while write is going on, the current stream + * gets full or explicit flush or close request is made by client. when the + * stream gets full and we try to close the stream , we might end up hitting + * an exception in the exception handling path, we write the data residing in + * in the buffer pool to a new Block. In cases, as such, when the data gets + * written to new stream , it will be at max half full. In such cases, we + * should just write the data and not close the stream as the block won't be + * completely full. + * + * @param op Flag which decides whether to call close or flush on the + * outputStream. + * @throws IOException In case, flush or close fails with exception. + */ + @SuppressWarnings("squid:S1141") + private void handleFlushOrClose(StreamAction op) throws IOException { + if (!blockDataStreamOutputEntryPool.isEmpty()) { + while (true) { + try { + BlockDataStreamOutputEntry entry = + blockDataStreamOutputEntryPool.getCurrentStreamEntry(); + if (entry != null) { + try { + handleStreamAction(entry, op); + } catch (IOException ioe) { + handleException(entry, ioe); + continue; + } + } + return; + } catch (Exception e) { + markStreamClosed(); + throw e; + } + } + } + } + + private void handleStreamAction(BlockDataStreamOutputEntry entry, + StreamAction op) throws IOException { + Collection failedServers = entry.getFailedServers(); + // failed servers can be null in case there is no data written in + // the stream + if (!failedServers.isEmpty()) { + blockDataStreamOutputEntryPool.getExcludeList().addDatanodes( + failedServers); + } + switch (op) { + case CLOSE: + entry.close(); + break; + case FULL: + if (entry.getRemaining() == 0) { + entry.close(); + } + break; + case FLUSH: + entry.flush(); + break; + default: + throw new IOException("Invalid Operation"); + } + } + + /** + * Commit the key to OM, this will add the blocks as the new key blocks. + * + * @throws IOException + */ + @Override + public void close() throws IOException { + if (closed) { + return; + } + closed = true; + try { + handleFlushOrClose(StreamAction.CLOSE); + if (!isException) { + Preconditions.checkArgument(writeOffset == offset); + } + blockDataStreamOutputEntryPool.commitKey(offset); + } finally { + blockDataStreamOutputEntryPool.cleanup(); + } + } + + public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() { + return blockDataStreamOutputEntryPool.getCommitUploadPartInfo(); + } + + public FileEncryptionInfo getFileEncryptionInfo() { + return feInfo; + } + + @VisibleForTesting + public ExcludeList getExcludeList() { + return blockDataStreamOutputEntryPool.getExcludeList(); + } + + /** + * Builder class of KeyDataStreamOutput. + */ + public static class Builder { + private OpenKeySession openHandler; + private XceiverClientFactory xceiverManager; + private OzoneManagerProtocol omClient; + private int chunkSize; + private String requestID; + private String multipartUploadID; + private int multipartNumber; + private boolean isMultipartKey; + private boolean unsafeByteBufferConversion; + private OzoneClientConfig clientConfig; + private ReplicationConfig replicationConfig; + + public Builder setMultipartUploadID(String uploadID) { + this.multipartUploadID = uploadID; + return this; + } + + public Builder setMultipartNumber(int partNumber) { + this.multipartNumber = partNumber; + return this; + } + + public Builder setHandler(OpenKeySession handler) { + this.openHandler = handler; + return this; + } + + public Builder setXceiverClientManager(XceiverClientFactory manager) { + this.xceiverManager = manager; + return this; + } + + public Builder setOmClient(OzoneManagerProtocol client) { + this.omClient = client; + return this; + } + + public Builder setChunkSize(int size) { + this.chunkSize = size; + return this; + } + + public Builder setRequestID(String id) { + this.requestID = id; + return this; + } + + public Builder setIsMultipartKey(boolean isMultipart) { + this.isMultipartKey = isMultipart; + return this; + } + + public Builder setConfig(OzoneClientConfig config) { + this.clientConfig = config; + return this; + } + + public Builder enableUnsafeByteBufferConversion(boolean enabled) { + this.unsafeByteBufferConversion = enabled; + return this; + } + + + public Builder setReplicationConfig(ReplicationConfig replConfig) { + this.replicationConfig = replConfig; + return this; + } + + public KeyDataStreamOutput build() { + return new KeyDataStreamOutput( + clientConfig, + openHandler, + xceiverManager, + omClient, + chunkSize, + requestID, + replicationConfig, + multipartUploadID, + multipartNumber, + isMultipartKey, + unsafeByteBufferConversion); + } + + } + + /** + * Verify that the output stream is open. Non blocking; this gives + * the last state of the volatile {@link #closed} field. + * @throws IOException if the connection is closed. + */ + private void checkNotClosed() throws IOException { + if (closed) { + throw new IOException( + ": " + FSExceptionMessages.STREAM_IS_CLOSED + " Key: " + + blockDataStreamOutputEntryPool.getKeyName()); + } + } +} diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneDataStreamOutput.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneDataStreamOutput.java new file mode 100644 index 000000000000..d40ac2b332ef --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneDataStreamOutput.java @@ -0,0 +1,70 @@ +/** + * 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.io; + +import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput; +import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * OzoneDataStreamOutput is used to write data into Ozone. + * It uses SCM's {@link KeyDataStreamOutput} for writing the data. + */ +public class OzoneDataStreamOutput implements ByteBufferStreamOutput { + + private final ByteBufferStreamOutput byteBufferStreamOutput; + + /** + * Constructs OzoneDataStreamOutput with KeyDataStreamOutput. + * + * @param byteBufferStreamOutput the underlying ByteBufferStreamOutput + */ + public OzoneDataStreamOutput(ByteBufferStreamOutput byteBufferStreamOutput) { + this.byteBufferStreamOutput = byteBufferStreamOutput; + } + + @Override + public void write(ByteBuffer b, int off, int len) throws IOException { + byteBufferStreamOutput.write(b, off, len); + } + + @Override + public synchronized void flush() throws IOException { + byteBufferStreamOutput.flush(); + } + + @Override + public synchronized void close() throws IOException { + //commitKey can be done here, if needed. + byteBufferStreamOutput.close(); + } + + public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() { + if (byteBufferStreamOutput instanceof KeyDataStreamOutput) { + return ((KeyDataStreamOutput) + byteBufferStreamOutput).getCommitUploadPartInfo(); + } + // Otherwise return null. + return null; + } + + public ByteBufferStreamOutput getByteBufStreamOutput() { + return byteBufferStreamOutput; + } +} diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/SmallFileDataStreamOutput.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/SmallFileDataStreamOutput.java new file mode 100644 index 000000000000..9acb2630d6b8 --- /dev/null +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/SmallFileDataStreamOutput.java @@ -0,0 +1,479 @@ +/* + * 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.io; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage; +import org.apache.hadoop.hdds.ratis.RatisHelper; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientRatis; +import org.apache.hadoop.hdds.scm.client.HddsClientUtils; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput; +import org.apache.hadoop.hdds.scm.storage.StreamBuffer; +import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.ozone.common.Checksum; +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.OpenKeySession; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.apache.hadoop.security.token.Token; +import org.apache.ratis.client.api.DataStreamOutput; +import org.apache.ratis.io.StandardWriteOption; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +/** + * SmallFileDataStreamOutput, only used to write requests smaller than ChunkSize + *

+ * TODO : currently not support multi-thread access. + */ +public class SmallFileDataStreamOutput implements ByteBufferStreamOutput { + + public static final Logger LOG = + LoggerFactory.getLogger(SmallFileDataStreamOutput.class); + + private final AtomicReference blockID; + + private final XceiverClientFactory xceiverClientFactory; + private XceiverClientRatis xceiverClient; + private final OzoneClientConfig config; + + private final OzoneManagerProtocol omClient; + + private final OpenKeySession openKeySession; + private OmKeyLocationInfo keyLocationInfo; + private final OmKeyArgs keyArgs; + private final int realFileLen; + + private DataStreamOutput dataStreamOutput; + + private final boolean unsafeByteBufferConversion; + + private final List retryBuffers = new ArrayList<>(); + private long writtenDataLength = 0; + private long versionID; + private final Token token; + + private static final ByteString EMPTY_DATA = ByteString.copyFrom(new byte[0]); + + // error handler + private final ExcludeList excludeList; + private final Map, RetryPolicy> retryPolicyMap; + private AtomicInteger retryCount; + + private final AtomicReference> responseFuture = + new AtomicReference<>(); + + private final AtomicBoolean isDataStreamClose = new AtomicBoolean(false); + + private final AtomicReference responseExecutor = + new AtomicReference<>(); + + private boolean isDatastreamPipelineMode; + + public SmallFileDataStreamOutput( + OpenKeySession handler, + XceiverClientFactory xceiverClientManager, + OzoneManagerProtocol omClient, + OzoneClientConfig config, + boolean unsafeByteBufferConversion + ) { + this.xceiverClientFactory = xceiverClientManager; + this.omClient = omClient; + this.config = config; + this.openKeySession = handler; + + this.keyLocationInfo = handler.getKeyInfo().getLatestVersionLocations() + .getBlocksLatestVersionOnly().get(0); + + this.blockID = new AtomicReference<>(keyLocationInfo.getBlockID()); + this.versionID = keyLocationInfo.getCreateVersion(); + + this.unsafeByteBufferConversion = unsafeByteBufferConversion; + + OmKeyInfo info = handler.getKeyInfo(); + + this.keyArgs = new OmKeyArgs.Builder().setVolumeName(info.getVolumeName()) + .setBucketName(info.getBucketName()).setKeyName(info.getKeyName()) + .setReplicationConfig(info.getReplicationConfig()) + .setDataSize(info.getDataSize()) + .setIsMultipartKey(false).build(); + this.realFileLen = (int) info.getDataSize(); + + this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException( + config.getMaxRetryCount(), config.getRetryInterval()); + this.retryCount = new AtomicInteger(0); + + this.excludeList = new ExcludeList(); + + this.token = null; + + this.responseFuture.set(new CompletableFuture<>()); + this.responseExecutor.set(Executors.newSingleThreadExecutor()); + + this.isDatastreamPipelineMode = config.isDatastreamPipelineMode(); + } + + @VisibleForTesting + public BlockID getBlockID() { + return blockID.get(); + } + + @VisibleForTesting + public OmKeyLocationInfo getKeyLocationInfo() { + return keyLocationInfo; + } + + private void allocateNewBlock() throws IOException { + if (!excludeList.isEmpty()) { + LOG.info("Allocating block with {}", excludeList); + } + Pipeline oldPipeline = this.keyLocationInfo.getPipeline(); + this.keyLocationInfo = + omClient.allocateBlock(keyArgs, openKeySession.getId(), excludeList); + BlockID oldBlockID = this.blockID.getAndSet(keyLocationInfo.getBlockID()); + LOG.info("Replace Block {} ({}) to {} ({})", oldBlockID, oldPipeline, + keyLocationInfo.getBlockID(), keyLocationInfo.getPipeline()); + this.versionID = keyLocationInfo.getCreateVersion(); + } + + @Override + public void write(ByteBuffer bb) throws IOException { + if (bb == null) { + throw new NullPointerException(); + } + assert writtenDataLength + 1 <= realFileLen; + retryBuffers.add(new StreamBuffer(bb.duplicate())); + writtenDataLength++; + } + + @Override + public void write(ByteBuffer bb, int off, int len) throws IOException { + if (bb == null) { + throw new NullPointerException(); + } + + if ((off < 0) || (off > bb.remaining()) || (len < 0) || + ((off + len) > bb.remaining()) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException(); + } + if (len == 0) { + return; + } + assert writtenDataLength + len <= realFileLen; + retryBuffers.add(new StreamBuffer(bb.duplicate(), off, len)); + writtenDataLength += len; + } + + @Override + public void flush() { + } + + @Override + public void close() throws IOException { + boolean retry = false; + ByteString blockData = null; + while (true) { + try { + checkOpen(); + DataStreamOutput out = maybeInitStream(); + if (!retry) { + blockData = getBlockData(retryBuffers); + } + + ContainerProtos.ContainerCommandRequestProto putSmallFileRequest = + getPutSmallFileRequest(blockData); + putSmallFileToContainer(putSmallFileRequest, out); + + handleWriteMetaData(out); + } catch (IOException ee) { + handleException(ee); + retry = true; + continue; + } + break; + } + cleanup(false); + } + + private void handleWriteMetaData(DataStreamOutput out) throws IOException { + if (out != null) { + keyArgs.setDataSize(writtenDataLength); + keyLocationInfo.setLength(writtenDataLength); + + Map metadata = keyArgs.getMetadata(); + keyArgs.setMetadata(metadata); + + keyArgs.setLocationInfoList(Collections.singletonList(keyLocationInfo)); + omClient.commitKey(keyArgs, openKeySession.getId()); + } else if (writtenDataLength == 0) { + keyArgs.setDataSize(0); + keyLocationInfo.setLength(0); + + Map metadata = keyArgs.getMetadata(); + keyArgs.setMetadata(metadata); + + keyArgs.setLocationInfoList(Collections.emptyList()); + + omClient.commitKey(keyArgs, openKeySession.getId()); + } + } + + private void setExceptionAndThrow(IOException ioe) throws IOException { + throw ioe; + } + + /** + * It performs following actions : + * a. Updates the committed length at datanode for the current stream in + * datanode. + * b. Reads the data from the underlying buffer and writes it the next stream. + * + * @param exception actual exception that occurred + * @throws IOException Throws IOException if Write fails + */ + private void handleException(IOException exception) throws IOException { + Throwable t = HddsClientUtils.checkForException(exception); + Preconditions.checkNotNull(t); + boolean retryFailure = HddsClientUtils.checkForRetryFailure(t); + boolean containerExclusionException = false; + if (!retryFailure) { + containerExclusionException = + HddsClientUtils.checkIfContainerToExclude(t); + } + + long totalSuccessfulFlushedData = 0L; + long bufferedDataLen = writtenDataLength; + + if (containerExclusionException) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "Encountered exception {}. The last committed block length is {}, " + + "uncommitted data length is {} retry count {}", exception, + totalSuccessfulFlushedData, bufferedDataLen, retryCount.get()); + } + excludeList + .addConatinerId(ContainerID.valueOf(getBlockID().getContainerID())); + } else if (xceiverClient != null) { + LOG.warn( + "Encountered exception {} on the pipeline {}. " + + "The last committed block length is {}, " + + "uncommitted data length is {} retry count {}", exception, + xceiverClient.getPipeline(), totalSuccessfulFlushedData, + bufferedDataLen, retryCount.get()); + excludeList.addPipeline(xceiverClient.getPipeline().getId()); + } + allocateNewBlock(); + + // just clean up the current stream. + cleanup(retryFailure); + + if (bufferedDataLen > 0) { + // If the data is still cached in the underlying stream, we need to + // allocate new block and write this data in the datanode. + HddsClientUtils.streamRetryHandle(exception, retryPolicyMap, retryCount, + this::setExceptionAndThrow); + } + } + + private void cleanup(boolean invalidateClient) { + try { + if (dataStreamOutput != null && !isDataStreamClose.get()) { + dataStreamOutput.close(); + } + if (xceiverClientFactory != null && xceiverClient != null) { + xceiverClientFactory.releaseClient(xceiverClient, invalidateClient); + } + + responseExecutor.get().shutdownNow(); + } catch (Throwable e) { + LOG.warn("cleanup error", e); + } finally { + dataStreamOutput = null; + isDataStreamClose.set(false); + xceiverClient = null; + + responseFuture.set(new CompletableFuture<>()); + responseExecutor.set(Executors.newSingleThreadExecutor()); + } + } + + private static ByteString getBlockData(List buffers) { + List byteStrings = new ArrayList<>(); + buffers.forEach(c -> byteStrings.add(ByteString.copyFrom(c.duplicate()))); + return ByteString.copyFrom(byteStrings); + } + + private ContainerProtos.ContainerCommandRequestProto getPutSmallFileRequest( + ByteString blockData) throws IOException { + // new checksum + ByteBuffer checksumBuffer = ByteBuffer.allocate((int) writtenDataLength); + retryBuffers.forEach(c -> checksumBuffer.put(c.duplicate())); + checksumBuffer.flip(); + ContainerProtos.ChecksumData checksumData = + (new Checksum(config.getChecksumType(), config.getBytesPerChecksum())) + .computeChecksum(checksumBuffer).getProtoBufMessage(); + + return generatePutSmallFileRequest(writtenDataLength, checksumData, + ContainerProtos.Type.PutSmallFile, blockData); + } + + private ContainerProtos.ContainerCommandRequestProto + generatePutSmallFileRequest(long len, + ContainerProtos.ChecksumData checksumData, + ContainerProtos.Type type, ByteString blockData) + throws IOException { + ContainerProtos.ChunkInfo chunk = + ContainerProtos.ChunkInfo.newBuilder() + .setChunkName(getBlockID().getLocalID() + "_chunk_0") + .setOffset(0) + .setLen(len) + .setChecksumData(checksumData) + .build(); + + ContainerProtos.BlockData containerBlockData = + ContainerProtos.BlockData.newBuilder() + .setBlockID(getBlockID().getDatanodeBlockIDProtobuf()) + .setSize(len) + .build(); + ContainerProtos.PutBlockRequestProto.Builder createBlockRequest = + ContainerProtos.PutBlockRequestProto.newBuilder() + .setBlockData(containerBlockData); + + ContainerProtos.PutSmallFileRequestProto putSmallFileRequest = + ContainerProtos.PutSmallFileRequestProto.newBuilder() + .setChunkInfo(chunk) + .setBlock(createBlockRequest) + .setData(blockData) + .build(); + + String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); + ContainerProtos.ContainerCommandRequestProto.Builder builder = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(type) + .setContainerID(getBlockID().getContainerID()) + .setDatanodeUuid(id) + .setPutSmallFile(putSmallFileRequest); + if (token != null) { + builder.setEncodedToken(token.encodeToUrlString()); + } + return builder.build(); + } + + private void putSmallFileToContainer( + ContainerProtos.ContainerCommandRequestProto requestProto, + DataStreamOutput out) + throws IOException { + ByteBuffer buf = + ContainerCommandRequestMessage.toMessage(requestProto, null) + .getContent().asReadOnlyByteBuffer(); + + out.writeAsync(buf, StandardWriteOption.CLOSE).whenCompleteAsync((r, e) -> { + isDataStreamClose.set(true); + if (e != null || !r.isSuccess()) { + String msg = + "close stream is not success, Failed to putSmallFile into block " + + getBlockID(); + if (!responseFuture.get().isDone()) { + responseFuture.get().completeExceptionally(new IOException(msg, e)); + } + LOG.warn(msg); + } else { + if (!responseFuture.get().isDone()) { + responseFuture.get().complete(true); + } + if (LOG.isDebugEnabled()) { + LOG.debug("close stream success, block id: {} metadata len: {}", + getBlockID(), buf.capacity()); + } + + } + }, this.responseExecutor.get()); + + try { + responseFuture.get().get(); + } catch (Exception e) { + throw new IOException(e); + } + } + + private DataStreamOutput maybeInitStream() throws IOException { + if (xceiverClientFactory != null && xceiverClient != null && + dataStreamOutput == null) { + // fake checksum + Checksum checksum = + new Checksum(config.getChecksumType(), config.getBytesPerChecksum()); + ContainerProtos.ChecksumData checksumData = + checksum.computeChecksum(new byte[0]).getProtoBufMessage(); + + ContainerProtos.ContainerCommandRequestProto streamInitRequest = + generatePutSmallFileRequest(keyArgs.getDataSize(), checksumData, + ContainerProtos.Type.StreamInit, EMPTY_DATA); + dataStreamOutput = sendStreamHeader(streamInitRequest); + } + return dataStreamOutput; + } + + private DataStreamOutput sendStreamHeader( + ContainerProtos.ContainerCommandRequestProto request) { + ContainerCommandRequestMessage message = + ContainerCommandRequestMessage.toMessage(request, null); + + if (isDatastreamPipelineMode) { + return Preconditions.checkNotNull(xceiverClient.getDataStreamApi()) + .stream(message.getContent().asReadOnlyByteBuffer(), + RatisHelper.getRoutingTable(xceiverClient.getPipeline())); + } else { + return Preconditions.checkNotNull(xceiverClient.getDataStreamApi()) + .stream(message.getContent().asReadOnlyByteBuffer()); + } + } + + private void checkOpen() throws IOException { + if (xceiverClient == null) { + this.xceiverClient = + (XceiverClientRatis) xceiverClientFactory.acquireClient( + keyLocationInfo.getPipeline()); + } + } + +} diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java index 7da2108038c7..000832a07520 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java @@ -39,6 +39,7 @@ import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts; import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.VolumeArgs; +import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.om.OMConfigKeys; @@ -294,6 +295,20 @@ OzoneOutputStream createKey(String volumeName, String bucketName, Map metadata) throws IOException; + /** + * Writes a key in an existing bucket. + * @param volumeName Name of the Volume + * @param bucketName Name of the Bucket + * @param keyName Name of the Key + * @param size Size of the data + * @param metadata custom key value metadata + * @return {@link OzoneDataStreamOutput} + * + */ + OzoneDataStreamOutput createStreamKey(String volumeName, String bucketName, + String keyName, long size, ReplicationConfig replicationConfig, + Map metadata) + throws IOException; /** * Reads a key from an existing bucket. @@ -462,6 +477,24 @@ OzoneOutputStream createMultipartKey(String volumeName, String bucketName, int partNumber, String uploadID) throws IOException; + /** + * Create a part key for a multipart upload key. + * @param volumeName + * @param bucketName + * @param keyName + * @param size + * @param partNumber + * @param uploadID + * @return OzoneDataStreamOutput + * @throws IOException + */ + OzoneDataStreamOutput createMultipartStreamKey(String volumeName, + String bucketName, + String keyName, long size, + int partNumber, + String uploadID) + throws IOException; + /** * Complete Multipart upload. This will combine all the parts and make the * key visible in ozone. @@ -668,6 +701,11 @@ OzoneOutputStream createFile(String volumeName, String bucketName, String keyName, long size, ReplicationConfig replicationConfig, boolean overWrite, boolean recursive) throws IOException; + @SuppressWarnings("checkstyle:parameternumber") + OzoneDataStreamOutput createStreamFile(String volumeName, String bucketName, + String keyName, long size, ReplicationConfig replicationConfig, + boolean overWrite, boolean recursive) throws IOException; + /** * List the status for a file or a directory and its contents. diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 28efa8c91245..40057f792ff0 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -77,13 +77,16 @@ import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts; import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.VolumeArgs; +import org.apache.hadoop.ozone.client.io.KeyDataStreamOutput; import org.apache.hadoop.ozone.client.io.KeyInputStream; import org.apache.hadoop.ozone.client.io.KeyOutputStream; import org.apache.hadoop.ozone.client.io.LengthInputStream; import org.apache.hadoop.ozone.client.io.MultipartCryptoKeyInputStream; import org.apache.hadoop.ozone.client.io.OzoneCryptoInputStream; +import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.client.io.SmallFileDataStreamOutput; import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.exceptions.OMException; @@ -874,6 +877,48 @@ public OzoneOutputStream createKey( return createOutputStream(openKey, requestId, replicationConfig); } + @Override + public OzoneDataStreamOutput createStreamKey( + String volumeName, String bucketName, String keyName, long size, + ReplicationConfig replicationConfig, + Map metadata) + throws IOException { + verifyVolumeName(volumeName); + verifyBucketName(bucketName); + if (checkKeyNameEnabled) { + HddsClientUtils.verifyKeyName(keyName); + } + HddsClientUtils.checkNotNull(keyName, replicationConfig); + String requestId = UUID.randomUUID().toString(); + + OmKeyArgs.Builder builder = new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .setDataSize(size) + .setReplicationConfig(replicationConfig) + .addAllMetadata(metadata) + .setAcls(getAclList()); + + if (Boolean.parseBoolean(metadata.get(OzoneConsts.GDPR_FLAG))) { + try { + GDPRSymmetricKey gKey = new GDPRSymmetricKey(new SecureRandom()); + builder.addAllMetadata(gKey.getKeyDetails()); + } catch (Exception e) { + if (e instanceof InvalidKeyException && + e.getMessage().contains("Illegal key size or default parameters")) { + LOG.error("Missing Unlimited Strength Policy jars. Please install " + + "Java Cryptography Extension (JCE) Unlimited Strength " + + "Jurisdiction Policy Files"); + } + throw new IOException(e); + } + } + + OpenKeySession openKey = ozoneManagerClient.openKey(builder.build()); + return createDataStreamOutput(openKey, requestId, replicationConfig, size); + } + private KeyProvider.KeyVersion getDEK(FileEncryptionInfo feInfo) throws IOException { // check crypto protocol version @@ -1207,6 +1252,70 @@ public OzoneOutputStream createMultipartKey(String volumeName, } } + @Override + public OzoneDataStreamOutput createMultipartStreamKey( + String volumeName, + String bucketName, + String keyName, + long size, + int partNumber, + String uploadID) + throws IOException { + verifyVolumeName(volumeName); + verifyBucketName(bucketName); + if (checkKeyNameEnabled) { + HddsClientUtils.verifyKeyName(keyName); + } + HddsClientUtils.checkNotNull(keyName, uploadID); + Preconditions.checkArgument(partNumber > 0 && partNumber <= 10000, "Part " + + "number should be greater than zero and less than or equal to 10000"); + Preconditions.checkArgument(size >= 0, "size should be greater than or " + + "equal to zero"); + String requestId = UUID.randomUUID().toString(); + + OmKeyArgs keyArgs = new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .setDataSize(size) + .setIsMultipartKey(true) + .setMultipartUploadID(uploadID) + .setMultipartUploadPartNumber(partNumber) + .setAcls(getAclList()) + .build(); + + OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs); + + KeyDataStreamOutput keyOutputStream = + new KeyDataStreamOutput.Builder() + .setHandler(openKey) + .setXceiverClientManager(xceiverClientManager) + .setOmClient(ozoneManagerClient) + .setRequestID(requestId) + .setReplicationConfig(openKey.getKeyInfo().getReplicationConfig()) + .setMultipartNumber(partNumber) + .setMultipartUploadID(uploadID) + .setIsMultipartKey(true) + .enableUnsafeByteBufferConversion(unsafeByteBufferConversion) + .setConfig(clientConfig) + .build(); + keyOutputStream + .addPreallocateBlocks( + openKey.getKeyInfo().getLatestVersionLocations(), + openKey.getOpenVersion()); + + FileEncryptionInfo feInfo = openKey.getKeyInfo().getFileEncryptionInfo(); + if (feInfo != null) { + // todo: need to support file encrypt, + // https://issues.apache.org/jira/browse/HDDS-5892 + throw new UnsupportedOperationException( + "FileEncryptionInfo is not yet supported in " + + "createMultipartStreamKey"); + } else { + return new OzoneDataStreamOutput(keyOutputStream); + } + } + @Override public OmMultipartUploadCompleteInfo completeMultipartUpload( String volumeName, String bucketName, String keyName, String uploadID, @@ -1397,6 +1506,26 @@ public OzoneOutputStream createFile(String volumeName, String bucketName, replicationConfig); } + @Override + public OzoneDataStreamOutput createStreamFile(String volumeName, + String bucketName, String keyName, long size, + ReplicationConfig replicationConfig, boolean overWrite, boolean recursive) + throws IOException { + OmKeyArgs keyArgs = new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .setDataSize(size) + .setReplicationConfig(replicationConfig) + .setAcls(getAclList()) + .setLatestVersionLocation(getLatestVersionLocation) + .build(); + OpenKeySession keySession = + ozoneManagerClient.createFile(keyArgs, overWrite, recursive); + return createDataStreamOutput(keySession, UUID.randomUUID().toString(), + replicationConfig, size); + } + @Override public List listStatus(String volumeName, String bucketName, String keyName, boolean recursive, String startKey, long numEntries) @@ -1520,6 +1649,33 @@ private OzoneInputStream createInputStream( } } + private OzoneDataStreamOutput createDataStreamOutput(OpenKeySession openKey, + String requestId, ReplicationConfig replicationConfig, long size) + throws IOException { + // size == 0, can be represented as borderless + if (size <= chunkSize && size != 0) { + SmallFileDataStreamOutput smallFileDataStreamOutput = + new SmallFileDataStreamOutput(openKey, xceiverClientManager, + ozoneManagerClient, clientConfig, unsafeByteBufferConversion); + return new OzoneDataStreamOutput(smallFileDataStreamOutput); + } + + KeyDataStreamOutput keyOutputStream = + new KeyDataStreamOutput.Builder() + .setHandler(openKey) + .setXceiverClientManager(xceiverClientManager) + .setOmClient(ozoneManagerClient) + .setRequestID(requestId) + .setReplicationConfig(replicationConfig) + .enableUnsafeByteBufferConversion(unsafeByteBufferConversion) + .setConfig(clientConfig) + .build(); + keyOutputStream + .addPreallocateBlocks(openKey.getKeyInfo().getLatestVersionLocations(), + openKey.getOpenVersion()); + return new OzoneDataStreamOutput(keyOutputStream); + } + private OzoneOutputStream createOutputStream(OpenKeySession openKey, String requestId, ReplicationConfig replicationConfig) throws IOException { diff --git a/hadoop-ozone/dev-support/intellij/runConfigurations/Datanode2.xml b/hadoop-ozone/dev-support/intellij/runConfigurations/Datanode2.xml index 3d3302030d18..040b515b9fac 100644 --- a/hadoop-ozone/dev-support/intellij/runConfigurations/Datanode2.xml +++ b/hadoop-ozone/dev-support/intellij/runConfigurations/Datanode2.xml @@ -18,7 +18,7 @@

+ * 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.conf.StorageUnit; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientMetrics; +import org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput; +import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput; +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.ContainerTestHelper; +import org.apache.hadoop.ozone.container.TestHelper; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; + +/** + * Tests BlockDataStreamOutput class. + */ +public class TestBlockDataStreamOutput { + + /** + * Set a timeout for each test. + */ + @Rule + public Timeout timeout = Timeout.seconds(300); + 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 String keyString; + + /** + * Create a MiniDFSCluster for testing. + *

+ * Ozone is made active by setting OZONE_ENABLED = true + * + * @throws IOException + */ + @BeforeClass + 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.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS); + conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); + conf.setQuietMode(false); + conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4, + StorageUnit.MB); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(7) + .setTotalPipelineNumLimit(10) + .setBlockSize(blockSize) + .setChunkSize(chunkSize) + .setStreamBufferFlushSize(flushSize) + .setStreamBufferMaxSize(maxFlushSize) + .setDataStreamBufferFlushize(maxFlushSize) + .setStreamBufferSizeUnit(StorageUnit.BYTES) + .setDataStreamMinPacketSize(chunkSize) + .setDataStreamStreamWindowSize(5 * chunkSize) + .build(); + cluster.waitForClusterToBeReady(); + //the easiest way to create an open container is creating a key + client = OzoneClientFactory.getRpcClient(conf); + objectStore = client.getObjectStore(); + keyString = UUID.randomUUID().toString(); + volumeName = "testblockoutputstream"; + bucketName = volumeName; + objectStore.createVolume(volumeName); + objectStore.getVolume(volumeName).createBucket(bucketName); + } + + private String getKeyName() { + return UUID.randomUUID().toString(); + } + + /** + * Shutdown MiniDFSCluster. + */ + @AfterClass + public static void shutdown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testMultiChunkWrite() throws Exception { + testWrite(chunkSize + 50); + testWriteWithFailure(chunkSize + 50); + } + + @Test + public void testMultiBlockWrite() throws Exception { + testWrite(blockSize + 50); + testWriteWithFailure(blockSize + 50); + } + + private void testWrite(int dataLength) throws Exception { + String keyName = getKeyName(); + OzoneDataStreamOutput key = createKey( + keyName, ReplicationType.RATIS, dataLength); + byte[] data = + ContainerTestHelper.getFixedLengthString(keyString, dataLength) + .getBytes(UTF_8); + key.write(ByteBuffer.wrap(data)); + // now close the stream, It will update the key length. + key.close(); + validateData(keyName, data); + } + + private void testWriteWithFailure(int dataLength) throws Exception { + String keyName = getKeyName(); + OzoneDataStreamOutput key = createKey( + keyName, ReplicationType.RATIS, dataLength); + byte[] data = + ContainerTestHelper.getFixedLengthString(keyString, dataLength) + .getBytes(UTF_8); + ByteBuffer b = ByteBuffer.wrap(data); + key.write(b); + KeyDataStreamOutput keyDataStreamOutput = + (KeyDataStreamOutput) key.getByteBufStreamOutput(); + ByteBufferStreamOutput stream = + keyDataStreamOutput.getStreamEntries().get(0).getByteBufStreamOutput(); + Assert.assertTrue(stream instanceof BlockDataStreamOutput); + TestHelper.waitForContainerClose(key, cluster); + key.write(b); + key.close(); + String dataString = new String(data, UTF_8); + validateData(keyName, dataString.concat(dataString).getBytes(UTF_8)); + } + + @Test + public void testPutBlockAtBoundary() throws Exception { + int dataLength = 500; + XceiverClientMetrics metrics = + XceiverClientManager.getXceiverClientMetrics(); + long putBlockCount = metrics.getContainerOpCountMetrics( + ContainerProtos.Type.PutBlock); + long pendingPutBlockCount = metrics.getPendingContainerOpCountMetrics( + ContainerProtos.Type.PutBlock); + String keyName = getKeyName(); + OzoneDataStreamOutput key = createKey( + keyName, ReplicationType.RATIS, 0); + byte[] data = + ContainerTestHelper.getFixedLengthString(keyString, dataLength) + .getBytes(UTF_8); + key.write(ByteBuffer.wrap(data)); + Assert.assertTrue( + metrics.getPendingContainerOpCountMetrics(ContainerProtos.Type.PutBlock) + <= pendingPutBlockCount + 1); + key.close(); + // Since data length is 500 , first putBlock will be at 400(flush boundary) + // and the other at 500 + Assert.assertTrue( + metrics.getContainerOpCountMetrics(ContainerProtos.Type.PutBlock) + == putBlockCount + 2); + validateData(keyName, data); + } + + + private OzoneDataStreamOutput createKey(String keyName, ReplicationType type, + long size) throws Exception { + return TestHelper.createStreamKey( + keyName, type, size, objectStore, volumeName, bucketName); + } + private void validateData(String keyName, byte[] data) throws Exception { + TestHelper + .validateData(keyName, data, objectStore, volumeName, bucketName); + } + + + @Test + public void testMinPacketSize() throws Exception { + String keyName = getKeyName(); + XceiverClientMetrics metrics = + XceiverClientManager.getXceiverClientMetrics(); + OzoneDataStreamOutput key = createKey(keyName, ReplicationType.RATIS, 0); + long writeChunkCount = + metrics.getContainerOpCountMetrics(ContainerProtos.Type.WriteChunk); + byte[] data = + ContainerTestHelper.getFixedLengthString(keyString, chunkSize / 2) + .getBytes(UTF_8); + key.write(ByteBuffer.wrap(data)); + // minPacketSize= 100, so first write of 50 wont trigger a writeChunk + Assert.assertEquals(writeChunkCount, + metrics.getContainerOpCountMetrics(ContainerProtos.Type.WriteChunk)); + key.write(ByteBuffer.wrap(data)); + Assert.assertEquals(writeChunkCount + 1, + metrics.getContainerOpCountMetrics(ContainerProtos.Type.WriteChunk)); + // now close the stream, It will update the key length. + key.close(); + String dataString = new String(data, UTF_8); + validateData(keyName, dataString.concat(dataString).getBytes(UTF_8)); + } + + @Test + public void testTotalAckDataLength() throws Exception { + int dataLength = 400; + String keyName = getKeyName(); + OzoneDataStreamOutput key = createKey( + keyName, ReplicationType.RATIS, 0); + byte[] data = + ContainerTestHelper.getFixedLengthString(keyString, dataLength) + .getBytes(UTF_8); + KeyDataStreamOutput keyDataStreamOutput = + (KeyDataStreamOutput) key.getByteBufStreamOutput(); + BlockDataStreamOutputEntry stream = + keyDataStreamOutput.getStreamEntries().get(0); + key.write(ByteBuffer.wrap(data)); + key.close(); + Assert.assertEquals(dataLength, stream.getTotalAckDataLength()); + } + +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineStream.java new file mode 100644 index 000000000000..07233f4dc9b5 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineStream.java @@ -0,0 +1,219 @@ +/** + * 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.conf.StorageUnit; +import org.apache.hadoop.hdds.client.ReplicationType; +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.OzoneClientConfig; +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.KeyDataStreamOutput; +import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; +import org.apache.hadoop.ozone.client.io.SmallFileDataStreamOutput; +import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.TestHelper; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; + +/** + * Tests the containerStateMachine stream handling. + */ +public class TestContainerStateMachineStream { + + /** + * Set a timeout for each test. + */ + @Rule + public Timeout timeout = Timeout.seconds(300); + + private MiniOzoneCluster cluster; + private OzoneConfiguration conf = new OzoneConfiguration(); + private OzoneClient client; + private ObjectStore objectStore; + private String volumeName; + private String bucketName; + + private static final int CHUNK_SIZE = 100; + 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; + + /** + * Create a MiniDFSCluster for testing. + * + * @throws IOException + */ + @Before + public void setup() throws Exception { + conf = new OzoneConfiguration(); + + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setStreamBufferFlushDelay(false); + conf.setFromObject(clientConfig); + + conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 200, + TimeUnit.MILLISECONDS); + conf.setTimeDuration(HDDS_COMMAND_STATUS_REPORT_INTERVAL, 200, + TimeUnit.MILLISECONDS); + conf.setTimeDuration(HDDS_PIPELINE_REPORT_INTERVAL, 200, + TimeUnit.MILLISECONDS); + conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 30, TimeUnit.SECONDS); + conf.setTimeDuration(OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, 1, + TimeUnit.SECONDS); + + RatisClientConfig ratisClientConfig = + conf.getObject(RatisClientConfig.class); + ratisClientConfig.setWriteRequestTimeout(Duration.ofSeconds(10)); + ratisClientConfig.setWatchRequestTimeout(Duration.ofSeconds(10)); + conf.setFromObject(ratisClientConfig); + + 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); + + conf.setLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY, 1); + conf.setQuietMode(false); + cluster = + MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(3) + .setHbInterval(200) + .setDataStreamMinPacketSize(1024) + .setBlockSize(BLOCK_SIZE) + .setChunkSize(CHUNK_SIZE) + .setStreamBufferFlushSize(FLUSH_SIZE) + .setStreamBufferMaxSize(MAX_FLUSH_SIZE) + .setStreamBufferSizeUnit(StorageUnit.BYTES) + .build(); + cluster.waitForClusterToBeReady(); + cluster.waitForPipelineTobeReady(HddsProtos.ReplicationFactor.ONE, 60000); + //the easiest way to create an open container is creating a key + client = OzoneClientFactory.getRpcClient(conf); + objectStore = client.getObjectStore(); + + volumeName = "testcontainerstatemachinestream"; + bucketName = "teststreambucket"; + objectStore.createVolume(volumeName); + objectStore.getVolume(volumeName).createBucket(bucketName); + + } + + /** + * Shutdown MiniDFSCluster. + */ + @After + public void shutdown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testContainerStateMachineForStreaming() throws Exception { + long size = CHUNK_SIZE + 1; + + OzoneDataStreamOutput key = TestHelper.createStreamKey( + "ozone-stream-test.txt", ReplicationType.RATIS, size, objectStore, + volumeName, bucketName); + + byte[] data = ContainerTestHelper.generateData((int) size, true); + key.write(ByteBuffer.wrap(data)); + key.flush(); + + KeyDataStreamOutput streamOutput = + (KeyDataStreamOutput) key.getByteBufStreamOutput(); + List locationInfoList = + streamOutput.getLocationInfoList(); + + key.close(); + + OmKeyLocationInfo omKeyLocationInfo = locationInfoList.get(0); + HddsDatanodeService dn = TestHelper.getDatanodeService(omKeyLocationInfo, + cluster); + + long bytesUsed = dn.getDatanodeStateMachine() + .getContainer().getContainerSet() + .getContainer(omKeyLocationInfo.getContainerID()). + getContainerData().getBytesUsed(); + + Assert.assertTrue(bytesUsed == size); + } + + + @Test + public void testContainerStateMachineForStreamingSmallFile() + throws Exception { + long size = CHUNK_SIZE - 1; + + OzoneDataStreamOutput key = TestHelper.createStreamKey( + "ozone-stream-test-small-file.txt", ReplicationType.RATIS, size, + objectStore, volumeName, bucketName); + + byte[] data = ContainerTestHelper.generateData((int) size, true); + key.write(ByteBuffer.wrap(data)); + key.flush(); + + SmallFileDataStreamOutput streamOutput = + (SmallFileDataStreamOutput) key.getByteBufStreamOutput(); + + OmKeyLocationInfo omKeyLocationInfo = streamOutput.getKeyLocationInfo(); + key.close(); + HddsDatanodeService dn = TestHelper.getDatanodeService(omKeyLocationInfo, + cluster); + + long bytesUsed = dn.getDatanodeStateMachine() + .getContainer().getContainerSet() + .getContainer(omKeyLocationInfo.getContainerID()). + getContainerData().getBytesUsed(); + + Assert.assertTrue(bytesUsed == size); + } + +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java index 92147cfde9f1..8dfdaba3d4c0 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java @@ -190,6 +190,7 @@ static void startCluster(OzoneConfiguration conf) throws Exception { .setTotalPipelineNumLimit(10) .setScmId(scmId) .setClusterId(clusterId) + .setDataStreamMinPacketSize(1024) .build(); cluster.waitForClusterToBeReady(); ozClient = OzoneClientFactory.getRpcClient(conf); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java index d44c4d0a86b9..109b32c0f51d 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java @@ -19,10 +19,12 @@ package org.apache.hadoop.ozone.client.rpc; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.HashMap; import java.util.UUID; +import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.ScmConfigKeys; @@ -31,12 +33,15 @@ import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClientFactory; +import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts; import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.common.OzoneChecksumException; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -44,6 +49,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.hadoop.hdds.client.ReplicationFactor.THREE; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; /** @@ -155,4 +161,51 @@ public void testGetKeyAndFileWithNetworkTopology() throws IOException { } } } + + @Test(timeout = 3000000) + public void testMultiPartUploadWithStream() throws IOException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + String keyName = UUID.randomUUID().toString(); + + byte[] sampleData = new byte[1024 * 8]; + + int valueLength = sampleData.length; + + getStore().createVolume(volumeName); + OzoneVolume volume = getStore().getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + ReplicationConfig replicationConfig = + ReplicationConfig.fromTypeAndFactor( + ReplicationType.RATIS, + THREE); + + OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName, + replicationConfig); + + assertNotNull(multipartInfo); + String uploadID = multipartInfo.getUploadID(); + Assert.assertEquals(volumeName, multipartInfo.getVolumeName()); + Assert.assertEquals(bucketName, multipartInfo.getBucketName()); + Assert.assertEquals(keyName, multipartInfo.getKeyName()); + assertNotNull(multipartInfo.getUploadID()); + + OzoneDataStreamOutput ozoneStreamOutput = bucket.createMultipartStreamKey( + keyName, valueLength, 1, uploadID); + ozoneStreamOutput.write(ByteBuffer.wrap(sampleData), 0, + valueLength); + ozoneStreamOutput.close(); + + OzoneMultipartUploadPartListParts parts = + bucket.listParts(keyName, uploadID, 0, 1); + + Assert.assertEquals(parts.getPartInfoList().size(), 1); + + OzoneMultipartUploadPartListParts.PartInfo partInfo = + parts.getPartInfoList().get(0); + Assert.assertEquals(valueLength, partInfo.getSize()); + + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSmallFileDataStreamOutput.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSmallFileDataStreamOutput.java new file mode 100644 index 000000000000..3370f5eb2e4d --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestSmallFileDataStreamOutput.java @@ -0,0 +1,197 @@ +/** + * 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.conf.StorageUnit; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +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.OzoneDataStreamOutput; +import org.apache.hadoop.ozone.client.io.SmallFileDataStreamOutput; +import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.TestHelper; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; + +/** + * Tests TestSmallFileDataStreamOutput class. + */ +public class TestSmallFileDataStreamOutput { + + /** + * Set a timeout for each test. + */ + @Rule + public Timeout timeout = Timeout.seconds(1000); + 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 String keyString; + + /** + * Create a MiniDFSCluster for testing. + *

+ * Ozone is made active by setting OZONE_ENABLED = true + * + * @throws IOException + */ + @BeforeClass + public static void init() throws Exception { + chunkSize = 100; + flushSize = 2 * chunkSize; + maxFlushSize = 2 * flushSize; + blockSize = 2 * maxFlushSize; + + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setStreamBufferFlushDelay(false); + conf.setFromObject(clientConfig); + + conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS); + conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); + conf.setQuietMode(false); + conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4, + StorageUnit.MB); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(7) + .setTotalPipelineNumLimit(10) + .setBlockSize(blockSize) + .setChunkSize(chunkSize) + .setStreamBufferFlushSize(flushSize) + .setStreamBufferMaxSize(maxFlushSize) + .setStreamBufferSizeUnit(StorageUnit.BYTES) + .build(); + cluster.waitForClusterToBeReady(); + //the easiest way to create an open container is creating a key + client = OzoneClientFactory.getRpcClient(conf); + objectStore = client.getObjectStore(); + keyString = UUID.randomUUID().toString(); + volumeName = "testsmallfiledatastreamoutput"; + bucketName = "testbucket1"; + objectStore.createVolume(volumeName); + objectStore.getVolume(volumeName).createBucket(bucketName); + } + + private String getKeyName() { + return UUID.randomUUID().toString(); + } + + /** + * Shutdown MiniDFSCluster. + */ + @AfterClass + public static void shutdown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void test2byteWrite() throws Exception { + testWrite(2); + } + + @Test + public void test2byteWriteWithFailure() throws Exception { + testWriteWithFailure(2); + } + + @Test + public void testChunkSizeWrite() throws Exception { + testWrite(chunkSize); + } + + @Test + public void testChunkSizeWriteWithFailure() throws Exception { + testWriteWithFailure(chunkSize); + } + + private void testWrite(int dataLength) throws Exception { + String keyName = getKeyName(); + OzoneDataStreamOutput key = createKey( + keyName, ReplicationType.RATIS, dataLength); + byte[] data = ContainerTestHelper.generateData(dataLength, true); + key.write(ByteBuffer.wrap(data)); + // now close the stream, It will update the key length. + key.close(); + validateData(keyName, data); + } + + private void testWriteWithFailure(int dataLength) throws Exception { + String keyName = getKeyName(); + OzoneDataStreamOutput key = createKey( + keyName, ReplicationType.RATIS, dataLength); + byte[] data = + ContainerTestHelper.generateData(dataLength / 2, true); + ByteBuffer b = ByteBuffer.wrap(data); + key.write(b); + + Assert.assertTrue( + key.getByteBufStreamOutput() instanceof SmallFileDataStreamOutput); + + waitForContainerClose( + ((SmallFileDataStreamOutput) key.getByteBufStreamOutput()).getBlockID() + .getContainerID()); + + key.write(b); + key.close(); + + validateData(keyName, data); + } + + private void waitForContainerClose(Long cid) + throws Exception { + TestHelper + .waitForContainerClose(cluster, cid); + } + + private OzoneDataStreamOutput createKey(String keyName, ReplicationType type, + long size) throws Exception { + return TestHelper.createStreamKey( + keyName, type, size, objectStore, volumeName, bucketName); + } + + private void validateData(String keyName, byte[] data) throws Exception { + TestHelper + .validateData(keyName, data, objectStore, volumeName, bucketName); + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java index 7a4e1c3def12..280852501020 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java @@ -28,6 +28,8 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeoutException; + +import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -44,8 +46,11 @@ import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.client.ObjectStore; +import org.apache.hadoop.ozone.client.io.BlockDataStreamOutputEntry; import org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry; +import org.apache.hadoop.ozone.client.io.KeyDataStreamOutput; import org.apache.hadoop.ozone.client.io.KeyOutputStream; +import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.container.common.impl.ContainerData; @@ -132,8 +137,23 @@ public static OzoneOutputStream createKey(String keyName, } org.apache.hadoop.hdds.client.ReplicationFactor factor = org.apache.hadoop.hdds.client.ReplicationFactor.THREE; + ReplicationConfig config = + ReplicationConfig.fromTypeAndFactor(type, factor); return objectStore.getVolume(volumeName).getBucket(bucketName) - .createKey(keyName, size, type, factor, new HashMap<>()); + .createKey(keyName, size, config, new HashMap<>()); + } + + public static OzoneDataStreamOutput createStreamKey(String keyName, + ReplicationType type, long size, ObjectStore objectStore, + String volumeName, String bucketName) throws Exception { + org.apache.hadoop.hdds.client.ReplicationFactor factor = + type == ReplicationType.STAND_ALONE ? + org.apache.hadoop.hdds.client.ReplicationFactor.ONE : + org.apache.hadoop.hdds.client.ReplicationFactor.THREE; + ReplicationConfig config = + ReplicationConfig.fromTypeAndFactor(type, factor); + return objectStore.getVolume(volumeName).getBucket(bucketName) + .createStreamKey(keyName, size, config, new HashMap<>()); } public static OzoneOutputStream createKey(String keyName, @@ -141,8 +161,10 @@ public static OzoneOutputStream createKey(String keyName, org.apache.hadoop.hdds.client.ReplicationFactor factor, long size, ObjectStore objectStore, String volumeName, String bucketName) throws Exception { + ReplicationConfig config = + ReplicationConfig.fromTypeAndFactor(type, factor); return objectStore.getVolume(volumeName).getBucket(bucketName) - .createKey(keyName, size, type, factor, new HashMap<>()); + .createKey(keyName, size, config, new HashMap<>()); } public static void validateData(String keyName, byte[] data, @@ -178,6 +200,24 @@ public static void waitForContainerClose(OzoneOutputStream outputStream, waitForContainerClose(cluster, containerIdList.toArray(new Long[0])); } + + public static void waitForContainerClose(OzoneDataStreamOutput outputStream, + MiniOzoneCluster cluster) throws Exception { + KeyDataStreamOutput keyOutputStream = + (KeyDataStreamOutput) outputStream.getByteBufStreamOutput(); + List streamEntryList = + keyOutputStream.getStreamEntries(); + List containerIdList = new ArrayList<>(); + for (BlockDataStreamOutputEntry entry : streamEntryList) { + long id = entry.getBlockID().getContainerID(); + if (!containerIdList.contains(id)) { + containerIdList.add(id); + } + } + Assert.assertTrue(!containerIdList.isEmpty()); + waitForContainerClose(cluster, containerIdList.toArray(new Long[0])); + } + public static void waitForPipelineClose(OzoneOutputStream outputStream, MiniOzoneCluster cluster, boolean waitForContainerCreation) throws Exception { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java index c2720d9c15f2..8c2322b6dbc8 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java @@ -64,6 +64,7 @@ import org.apache.ratis.rpc.RpcType; import static org.apache.ratis.rpc.SupportedRpcType.GRPC; import static org.apache.ratis.rpc.SupportedRpcType.NETTY; + import org.apache.ratis.util.function.CheckedBiConsumer; import org.junit.Assert; import org.junit.BeforeClass; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestSecureContainerServer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestSecureContainerServer.java index cd7c99554438..cb2db30c1023 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestSecureContainerServer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestSecureContainerServer.java @@ -216,6 +216,8 @@ static XceiverServerRatis newXceiverServerRatis( DatanodeDetails dn, OzoneConfiguration conf) throws IOException { conf.setInt(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT, dn.getPort(DatanodeDetails.Port.Name.RATIS).getValue()); + conf.setBoolean( + OzoneConfigKeys.DFS_CONTAINER_RATIS_DATASTREAM_RANDOM_PORT, true); final String dir = TEST_DIR + dn.getUuid(); conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir); final ContainerDispatcher dispatcher = createDispatcher(dn, 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 55bce591fdc1..85abca5820de 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 @@ -50,6 +50,7 @@ import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.OzoneKey; 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.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; @@ -234,6 +235,38 @@ public OzoneFSOutputStream createFile(String key, short replication, } } + @Override + public OzoneFSDataStreamOutput createStreamFile(String key, short replication, + boolean overWrite, boolean recursive) throws IOException { + incrementCounter(Statistic.OBJECTS_CREATED, 1); + try { + OzoneDataStreamOutput ozoneDataStreamOutput = null; + if (replication == ReplicationFactor.ONE.getValue() + || replication == ReplicationFactor.THREE.getValue()) { + + ReplicationConfig customReplicationConfig = + ReplicationConfig.adjustReplication(replicationConfig, replication, + config); + ozoneDataStreamOutput = bucket + .createStreamFile(key, 0, customReplicationConfig, overWrite, + recursive); + } else { + ozoneDataStreamOutput = bucket + .createStreamFile(key, 0, replicationConfig, overWrite, recursive); + } + return new OzoneFSDataStreamOutput( + ozoneDataStreamOutput.getByteBufStreamOutput()); + } catch (OMException ex) { + if (ex.getResult() == OMException.ResultCodes.FILE_ALREADY_EXISTS + || ex.getResult() == OMException.ResultCodes.NOT_A_FILE) { + throw new FileAlreadyExistsException( + ex.getResult().name() + ": " + ex.getMessage()); + } else { + throw ex; + } + } + } + @Override public void renameKey(String key, String newKeyName) throws IOException { incrementCounter(Statistic.OBJECTS_RENAMED, 1); diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java index 34961050ad91..3ac013a837e3 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.utils.LegacyHadoopConfigurationSource; +import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils; import org.apache.hadoop.security.UserGroupInformation; @@ -259,6 +260,13 @@ public FSDataOutputStream createNonRecursive(Path path, private FSDataOutputStream createOutputStream(String key, short replication, boolean overwrite, boolean recursive) throws IOException { + boolean isRatisStreamingEnabled = getConf().getBoolean( + OzoneConfigKeys.OZONE_FS_DATASTREAM_ENABLE, + OzoneConfigKeys.OZONE_FS_DATASTREAM_ENABLE_DEFAULT); + if (isRatisStreamingEnabled) { + return new FSDataOutputStream(adapter.createStreamFile(key, + replication, overwrite, recursive), statistics); + } return new FSDataOutputStream(adapter.createFile(key, replication, overwrite, recursive), statistics); } 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 35a027da901c..08a1ed35a1b0 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 @@ -61,6 +61,7 @@ import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.OzoneKey; 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.protocol.ClientProtocol; import org.apache.hadoop.ozone.om.exceptions.OMException; @@ -357,6 +358,44 @@ public OzoneFSOutputStream createFile(String pathStr, short replication, } } + @Override + public OzoneFSDataStreamOutput createStreamFile(String pathStr, + short replication, boolean overWrite, boolean recursive) + throws IOException { + incrementCounter(Statistic.OBJECTS_CREATED, 1); + OFSPath ofsPath = new OFSPath(pathStr); + if (ofsPath.isRoot() || ofsPath.isVolume() || ofsPath.isBucket()) { + throw new IOException("Cannot create file under root or volume."); + } + String key = ofsPath.getKeyName(); + try { + // Hadoop CopyCommands class always sets recursive to true + OzoneBucket bucket = getBucket(ofsPath, recursive); + OzoneDataStreamOutput ozoneDataStreamOutput = null; + if (replication == ReplicationFactor.ONE.getValue() + || replication == ReplicationFactor.THREE.getValue()) { + + ozoneDataStreamOutput = bucket.createStreamFile(key, 0, + ReplicationConfig.adjustReplication(replicationConfig, replication, + config), + overWrite, recursive); + } else { + ozoneDataStreamOutput = bucket + .createStreamFile(key, 0, replicationConfig, overWrite, recursive); + } + return new OzoneFSDataStreamOutput( + ozoneDataStreamOutput.getByteBufStreamOutput()); + } catch (OMException ex) { + if (ex.getResult() == OMException.ResultCodes.FILE_ALREADY_EXISTS + || ex.getResult() == OMException.ResultCodes.NOT_A_FILE) { + throw new FileAlreadyExistsException( + ex.getResult().name() + ": " + ex.getMessage()); + } else { + throw ex; + } + } + } + @Override public void renameKey(String key, String newKeyName) throws IOException { throw new IOException("OFS doesn't support renameKey, use rename instead."); diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java index 09383a62c0e7..a55aee4a0d1f 100644 --- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.utils.LegacyHadoopConfigurationSource; import org.apache.hadoop.ozone.OFSPath; +import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.om.exceptions.OMException; @@ -235,6 +236,13 @@ public FSDataOutputStream createNonRecursive(Path path, private FSDataOutputStream createOutputStream(String key, short replication, boolean overwrite, boolean recursive) throws IOException { + boolean isRatisStreamingEnabled = getConf().getBoolean( + OzoneConfigKeys.OZONE_FS_DATASTREAM_ENABLE, + OzoneConfigKeys.OZONE_FS_DATASTREAM_ENABLE_DEFAULT); + if (isRatisStreamingEnabled) { + return new FSDataOutputStream(adapter.createStreamFile(key, + replication, overwrite, recursive), statistics); + } return new FSDataOutputStream(adapter.createFile(key, replication, overwrite, recursive), statistics); } 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 31bf351f01a6..24566cb83f8b 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 @@ -45,6 +45,9 @@ public interface OzoneClientAdapter { OzoneFSOutputStream createFile(String key, short replication, boolean overWrite, boolean recursive) throws IOException; + OzoneFSDataStreamOutput createStreamFile(String key, short replication, + boolean overWrite, boolean recursive) throws IOException; + void renameKey(String key, String newKeyName) throws IOException; // Users should use rename instead of renameKey in OFS. diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSDataStreamOutput.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSDataStreamOutput.java new file mode 100644 index 000000000000..515dbca92b42 --- /dev/null +++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSDataStreamOutput.java @@ -0,0 +1,103 @@ +/* + * 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.ozone; + +import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +/** + * The ByteBuffer output stream for Ozone file system. + */ +public class OzoneFSDataStreamOutput extends OutputStream + implements ByteBufferStreamOutput { + + private final ByteBufferStreamOutput byteBufferStreamOutput; + + public OzoneFSDataStreamOutput( + ByteBufferStreamOutput byteBufferStreamOutput) { + this.byteBufferStreamOutput = byteBufferStreamOutput; + } + + /** + * Try to write the [off:off + len) slice in ByteBuf b to DataStream. + * + * @param b the data. + * @param off the start offset in the data. + * @param len the number of bytes to write. + * @throws IOException if an I/O error occurs. + */ + @Override + public void write(ByteBuffer b, int off, int len) + throws IOException { + byteBufferStreamOutput.write(b, off, len); + } + + /** + * Writes the specified byte to this output stream. The general + * contract for write is that one byte is written + * to the output stream. The byte to be written is the eight + * low-order bits of the argument b. The 24 + * high-order bits of b are ignored. + *

+ * Subclasses of OutputStream must provide an + * implementation for this method. + * + * @param b the byte. + * @throws IOException if an I/O error occurs. In particular, + * an IOException may be thrown if the + * output stream has been closed. + */ + @Override + public void write(int b) throws IOException { + byte[] singleBytes = new byte[1]; + singleBytes[0] = (byte) b; + byteBufferStreamOutput.write(ByteBuffer.wrap(singleBytes)); + } + + /** + * Flushes this DataStream output and forces any buffered output bytes + * to be written out. + * + * @throws IOException if an I/O error occurs. + */ + @Override + public void flush() throws IOException { + byteBufferStreamOutput.flush(); + } + + /** + * Closes this stream and releases any system resources associated + * with it. If the stream is already closed then invoking this + * method has no effect. + * + *

As noted in {@link AutoCloseable#close()}, cases where the + * close may fail require careful attention. It is strongly advised + * to relinquish the underlying resources and to internally + * mark the {@code Closeable} as closed, prior to throwing + * the {@code IOException}. + * + * @throws IOException if an I/O error occurs + */ + @Override + public void close() throws IOException { + byteBufferStreamOutput.close(); + } +} diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java index b7fbdd18e2c7..5044b92d1a1e 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java @@ -23,6 +23,9 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; import java.util.HashMap; import java.util.Map; @@ -35,6 +38,7 @@ import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClientException; import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput; import org.apache.hadoop.ozone.shell.OzoneAddress; import org.apache.commons.codec.digest.DigestUtils; @@ -97,10 +101,34 @@ protected void execute(OzoneClient client, OzoneAddress address) int chunkSize = (int) getConf().getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES); - try (InputStream input = new FileInputStream(dataFile); - OutputStream output = bucket.createKey(keyName, dataFile.length(), - replicationConfig, keyMetadata)) { - IOUtils.copyBytes(input, output, chunkSize); + + if (dataFile.length() <= chunkSize) { + if (isVerbose()) { + out().println("API: async"); + } + try (InputStream input = new FileInputStream(dataFile); + OutputStream output = bucket.createKey(keyName, dataFile.length(), + replicationConfig, keyMetadata)) { + IOUtils.copyBytes(input, output, chunkSize); + } + } else { + if (isVerbose()) { + out().println("API: streaming"); + } + try (RandomAccessFile raf = new RandomAccessFile(dataFile, "r"); + OzoneDataStreamOutput out = bucket.createStreamKey(keyName, + dataFile.length(), replicationConfig, keyMetadata)) { + FileChannel ch = raf.getChannel(); + long len = raf.length(); + long off = 0; + while (len > 0) { + long writeLen = Math.min(len, chunkSize); + ByteBuffer bb = ch.map(FileChannel.MapMode.READ_ONLY, off, writeLen); + out.write(bb); + off += writeLen; + len -= writeLen; + } + } } } diff --git a/pom.xml b/pom.xml index d8a6252e4f77..031b5e289383 100644 --- a/pom.xml +++ b/pom.xml @@ -72,7 +72,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs ${ozone.version} - 2.2.0 + 2.3.0-da5d868-SNAPSHOT 0.7.0