yandrey321 commented on code in PR #6613:
URL: https://github.com/apache/ozone/pull/6613#discussion_r2491838022
##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java:
##########
@@ -2046,6 +2055,86 @@ public void deleteUnreferenced(Container container, long
localID)
}
}
+ @Override
+ public ContainerCommandResponseProto readBlock(
+ ContainerCommandRequestProto request, Container kvContainer,
+ DispatcherContext dispatcherContext,
+ StreamObserver<ContainerCommandResponseProto> streamObserver) {
+
+ if (kvContainer.getContainerData().getLayoutVersion() != FILE_PER_BLOCK) {
+ return ContainerUtils.logAndReturnError(LOG,
+ new StorageContainerException("Only File Per Block is supported",
IO_EXCEPTION), request);
+ }
+
+ ContainerCommandResponseProto responseProto = null;
+ if (!request.hasReadBlock()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Malformed Read Block request. trace ID: {}",
request.getTraceID());
+ }
+ return malformedRequest(request);
+ }
+ try {
+ ReadBlockRequestProto readBlock = request.getReadBlock();
+
+ BlockID blockID = BlockID.getFromProtobuf(readBlock.getBlockID());
+ // This is a new api the block should always be checked.
+ BlockUtils.verifyReplicaIdx(kvContainer, blockID);
+ BlockUtils.verifyBCSId(kvContainer, blockID);
+
+ File blockFile =
FILE_PER_BLOCK.getChunkFile(kvContainer.getContainerData(), blockID, "unused");
+
+ BlockData blockData = getBlockManager().getBlock(kvContainer, blockID);
+ List<ContainerProtos.ChunkInfo> chunkInfos = blockData.getChunks();
+ // To get the chunksize, check the first chunk. Either there is only 1
chunk and its the largest, or there are
+ // multiple chunks and they are all the same size except the last one.
+ long bytesPerChunk = chunkInfos.get(0).getLen();
+ // The bytes per checksum is stored in the checksum data of each chunk,
so check the first chunk as they all
+ // must be the same.
+ ContainerProtos.ChecksumType checksumType =
chunkInfos.get(0).getChecksumData().getType();
+ ChecksumData checksumData = null;
+ int bytesPerChecksum = STREAMING_BYTES_PER_CHUNK;
+ if (checksumType == ContainerProtos.ChecksumType.NONE) {
+ checksumData = new ChecksumData(checksumType, 0);
+ } else {
+ bytesPerChecksum =
chunkInfos.get(0).getChecksumData().getBytesPerChecksum();
+ }
+ // We have to align the read to checksum boundaries, so whatever offset
is requested, we have to move back to the
+ // previous checksum boundary.
+ // eg if bytesPerChecksum is 512, and the requested offset is 600, we
have to move back to 512.
+ // If the checksum type is NONE, we don't have to do this, but using no
checksums should be rare in practice and
+ // it simplifies the code to always do this.
+ long adjustedOffset = readBlock.getOffset() - readBlock.getOffset() %
bytesPerChecksum;
+ try (RandomAccessFile file = new RandomAccessFile(blockFile, "r");
+ FileChannel channel = file.getChannel()) {
+ ByteBuffer buffer = ByteBuffer.allocate(bytesPerChecksum);
+ channel.position(adjustedOffset);
+ while (channel.read(buffer) != -1) {
+ buffer.flip();
+ if (checksumType != ContainerProtos.ChecksumType.NONE) {
+ // As the checksums are stored "chunk by chunk", we need to figure
out which chunk we start reading from,
+ // and its offset to pull out the correct checksum bytes for each
read.
+ int chunkIndex = (int) (adjustedOffset / bytesPerChunk);
+ int chunkOffset = (int) (adjustedOffset % bytesPerChunk);
+ int checksumIndex = chunkOffset / bytesPerChecksum;
+ ByteString checksum =
blockData.getChunks().get(chunkIndex).getChecksumData().getChecksums(checksumIndex);
+ checksumData = new ChecksumData(checksumType, bytesPerChecksum,
Collections.singletonList(checksum));
+ }
+ streamObserver.onNext(getReadBlockResponse(request, checksumData,
buffer, adjustedOffset));
+ buffer.clear();
+
+ adjustedOffset += bytesPerChecksum;
+ }
+ }
+ // TODO metrics.incContainerBytesStats(Type.ReadBlock,
readBlock.getLen());
Review Comment:
can we add metrics to see how it performs vs. non-streamed implementation?
##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java:
##########
@@ -85,11 +89,25 @@ public BlockExtendedInputStream create(ReplicationConfig
repConfig,
return new ECBlockInputStreamProxy((ECReplicationConfig)repConfig,
blockInfo, xceiverFactory, refreshFunction,
ecBlockStreamFactory, config);
+ } else if (config.isStreamReadBlock() &&
allDataNodesSupportStreamBlock(pipeline)) {
Review Comment:
Can we do streaming for EC?
##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java:
##########
@@ -172,6 +180,7 @@ public class KeyValueHandler extends Handler {
private static final Logger LOG = LoggerFactory.getLogger(
KeyValueHandler.class);
+ private static final int STREAMING_BYTES_PER_CHUNK = 1024 * 64;
Review Comment:
Can we have a configuration param for the size?
##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java:
##########
@@ -0,0 +1,443 @@
+/*
+ * 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.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.CanUnbuffer;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.StreamingReadResponse;
+import org.apache.hadoop.hdds.scm.StreamingReaderSpi;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+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.common.ChecksumData;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An {@link java.io.InputStream} called from KeyInputStream to read a block
from the
+ * container.
+ */
+public class StreamBlockInputStream extends BlockExtendedInputStream
+ implements Seekable, CanUnbuffer, ByteBufferReadable {
+ private static final Logger LOG =
LoggerFactory.getLogger(StreamBlockInputStream.class);
+ private static final int EOF = -1;
+ private static final Throwable CANCELLED_EXCEPTION = new
Throwable("Cancelled by client");
+
+ private final BlockID blockID;
+ private final long blockLength;
+ private final AtomicReference<Pipeline> pipelineRef = new
AtomicReference<>();
+ private final AtomicReference<Token<OzoneBlockTokenIdentifier>> tokenRef =
new AtomicReference<>();
+ private XceiverClientFactory xceiverClientFactory;
+ private XceiverClientSpi xceiverClient;
+
+ private ByteBuffer buffer;
+ private long position = 0;
+ private boolean initialized = false;
+ private StreamingReader streamingReader;
+
+ private final boolean verifyChecksum;
+ private final Function<BlockID, BlockLocationInfo> refreshFunction;
+ private final RetryPolicy retryPolicy;
+ private int retries = 0;
+
+ public StreamBlockInputStream(
+ BlockID blockID, long length, Pipeline pipeline,
+ Token<OzoneBlockTokenIdentifier> token,
+ XceiverClientFactory xceiverClientFactory,
+ Function<BlockID, BlockLocationInfo> refreshFunction,
+ OzoneClientConfig config) throws IOException {
+ this.blockID = blockID;
+ this.blockLength = length;
+ pipelineRef.set(setPipeline(pipeline));
+ tokenRef.set(token);
+ this.xceiverClientFactory = xceiverClientFactory;
+ this.verifyChecksum = config.isChecksumVerify();
+ this.retryPolicy = getReadRetryPolicy(config);
+ this.refreshFunction = refreshFunction;
+ }
+
+ @Override
+ public BlockID getBlockID() {
+ return blockID;
+ }
+
+ @Override
+ public long getLength() {
+ return blockLength;
+ }
+
+ @Override
+ public synchronized long getPos() {
+ return position;
+ }
+
+ @Override
+ public synchronized int read() throws IOException {
+ checkOpen();
+ if (!dataAvailableToRead()) {
+ return EOF;
+ }
+ position++;
+ return buffer.get();
+ }
+
+ @Override
+ public synchronized int read(byte[] b, int off, int len) throws IOException {
+ ByteBuffer tmpBuffer = ByteBuffer.wrap(b, off, len);
+ return read(tmpBuffer);
+ }
+
+ @Override
+ public synchronized int read(ByteBuffer targetBuf) throws IOException {
+ checkOpen();
+ int read = 0;
+ while (targetBuf.hasRemaining()) {
+ if (!dataAvailableToRead()) {
+ break;
+ }
+ int toCopy = Math.min(buffer.remaining(), targetBuf.remaining());
+ ByteBuffer tmpBuf = buffer.duplicate();
+ tmpBuf.limit(tmpBuf.position() + toCopy);
+ targetBuf.put(tmpBuf);
+ buffer.position(tmpBuf.position());
+ position += toCopy;
+ read += toCopy;
+ }
+ return read > 0 ? read : EOF;
+ }
+
+ private boolean dataAvailableToRead() throws IOException {
+ if (position >= blockLength) {
+ return false;
+ }
+ initialize();
+ if (buffer == null || buffer.remaining() == 0) {
+ int loaded = fillBuffer();
+ return loaded != EOF;
+ }
+ return true;
+ }
+
+ @Override
+ protected int readWithStrategy(ByteReaderStrategy strategy) throws
IOException {
+ throw new NotImplementedException("readWithStrategy is not implemented.");
+ }
+
+ @Override
+ public synchronized void seek(long pos) throws IOException {
+ checkOpen();
+ if (pos < 0) {
+ throw new IOException("Cannot seek to negative offset");
+ }
+ if (pos > blockLength) {
+ throw new IOException("Cannot seek after the end of the block");
+ }
+ if (pos == position) {
+ return;
+ }
+ closeStream();
+ position = pos;
+ }
+
+ @Override
+ // The seekable interface indicates that seekToNewSource should seek to a
new source of the data,
+ // ie a different datanode. This is not supported for now.
+ public synchronized boolean seekToNewSource(long l) throws IOException {
+ return false;
+ }
+
+ @Override
+ public synchronized void unbuffer() {
+ releaseClient();
+ }
+
+ private void closeStream() {
+ if (streamingReader != null) {
+ streamingReader.cancel();
+ streamingReader = null;
+ }
+ initialized = false;
+ buffer = null;
+ }
+
+ protected synchronized void checkOpen() throws IOException {
+ if (xceiverClientFactory == null) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED + " Block: "
+ blockID);
+ }
+ }
+
+ protected synchronized void acquireClient() throws IOException {
+ checkOpen();
+ if (xceiverClient == null) {
+ final Pipeline pipeline = pipelineRef.get();
+ try {
+ xceiverClient =
xceiverClientFactory.acquireClientForReadData(pipeline);
+ } catch (IOException ioe) {
+ LOG.warn("Failed to acquire client for pipeline {}, block {}",
pipeline, blockID);
+ throw ioe;
+ }
+ }
+ }
+
+ private void initialize() throws IOException {
+ if (initialized) {
+ return;
+ }
+ while (true) {
+ try {
+ acquireClient();
+ streamingReader = new StreamingReader();
+ ContainerProtocolCalls.readBlock(xceiverClient, position, blockID,
tokenRef.get(),
+ pipelineRef.get().getReplicaIndexes(), streamingReader);
+ initialized = true;
+ return;
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ handleExceptions(new IOException("Interrupted", ie));
+ } catch (IOException ioe) {
+ handleExceptions(ioe);
+ }
+ }
+ }
+
+ private void handleExceptions(IOException cause) throws IOException {
+ if (cause instanceof StorageContainerException ||
isConnectivityIssue(cause)) {
+ if (shouldRetryRead(cause, retryPolicy, retries++)) {
+ releaseClient();
+ refreshBlockInfo(cause);
+ LOG.warn("Refreshing block data to read block {} due to {}", blockID,
cause.getMessage());
+ } else {
+ throw cause;
+ }
+ } else {
+ throw cause;
+ }
+ }
+
+ private int fillBuffer() throws IOException {
+ if (!streamingReader.hasNext()) {
+ return EOF;
+ }
+ buffer = streamingReader.readNext();
+ return buffer == null ? EOF : buffer.limit();
+ }
+
+ protected synchronized void releaseClient() {
+ if (xceiverClientFactory != null && xceiverClient != null) {
+ closeStream();
+ xceiverClientFactory.releaseClientForReadData(xceiverClient, false);
+ xceiverClient = null;
+ }
+ }
+
+ @Override
+ public synchronized void close() throws IOException {
+ releaseClient();
+ xceiverClientFactory = null;
+ }
+
+ private void refreshBlockInfo(IOException cause) throws IOException {
+ refreshBlockInfo(cause, blockID, pipelineRef, tokenRef, refreshFunction);
+ }
+
+ private synchronized void releaseStreamResources(StreamingReadResponse
response) {
+ if (xceiverClient != null) {
+ xceiverClient.completeStreamRead(response);
+ }
+ }
+
+ /**
+ * Implementation of a StreamObserver used to received and buffer streaming
GRPC reads.
+ */
+ public class StreamingReader implements StreamingReaderSpi {
+
+ private final BlockingQueue<ContainerProtos.ReadBlockResponseProto>
responseQueue = new LinkedBlockingQueue<>(1);
+ private final AtomicBoolean completed = new AtomicBoolean(false);
+ private final AtomicBoolean failed = new AtomicBoolean(false);
+ private final AtomicBoolean semaphoreReleased = new AtomicBoolean(false);
+ private final AtomicReference<Throwable> error = new AtomicReference<>();
+ private volatile StreamingReadResponse response;
+
+ public boolean hasNext() {
+ return !responseQueue.isEmpty() || !completed.get();
+ }
+
+ public ByteBuffer readNext() throws IOException {
+ if (failed.get()) {
+ Throwable cause = error.get();
+ throw new IOException("Streaming read failed", cause);
+ }
+
+ if (completed.get() && responseQueue.isEmpty()) {
+ return null; // Stream ended
+ }
+
+ ReadBlockResponseProto readBlock;
+ try {
+ readBlock = responseQueue.poll(30, TimeUnit.SECONDS);
Review Comment:
Can we add param for this timeout value?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]