From d6d7b2abd3a909540d283cb52b3d06d8fdf9b6ce Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 11 Mar 2024 16:40:39 +0800 Subject: [PATCH 001/114] implement data stream api --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 140 +++++++++++++----- .../hdds/scm/storage/DataStreamClient.java | 29 ++++ .../scm/storage/DataStreamClientImpl.java | 67 +++++++++ .../main/proto/DatanodeClientProtocol.proto | 19 +++ 4 files changed, 222 insertions(+), 33 deletions(-) create mode 100644 hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClient.java create mode 100644 hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClientImpl.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 0a38e6604897..dc1fda9806ac 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.InterruptedIOException; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -39,6 +40,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc.XceiverClientProtocolServiceStub; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -350,39 +352,7 @@ private XceiverClientReply sendCommandWithRetry( // In case of an exception or an error, we will try to read from the // datanodes in the pipeline in a round-robin fashion. XceiverClientReply reply = new XceiverClientReply(null); - List datanodeList = null; - - DatanodeBlockID blockID = null; - if (request.getCmdType() == ContainerProtos.Type.GetBlock) { - blockID = request.getGetBlock().getBlockID(); - } else if (request.getCmdType() == ContainerProtos.Type.ReadChunk) { - blockID = request.getReadChunk().getBlockID(); - } else if (request.getCmdType() == ContainerProtos.Type.GetSmallFile) { - blockID = request.getGetSmallFile().getBlock().getBlockID(); - } - - if (blockID != null) { - // Check if the DN to which the GetBlock command was sent has been cached. - DatanodeDetails cachedDN = getBlockDNcache.get(blockID); - if (cachedDN != null) { - datanodeList = pipeline.getNodes(); - int getBlockDNCacheIndex = datanodeList.indexOf(cachedDN); - if (getBlockDNCacheIndex > 0) { - // Pull the Cached DN to the top of the DN list - Collections.swap(datanodeList, 0, getBlockDNCacheIndex); - } - } - } - if (datanodeList == null) { - if (topologyAwareRead) { - datanodeList = pipeline.getNodesInOrder(); - } else { - datanodeList = pipeline.getNodes(); - // Shuffle datanode list so that clients do not read in the same order - // every time. - Collections.shuffle(datanodeList); - } - } + List datanodeList = getDatanodeList(request); for (DatanodeDetails dn : datanodeList) { try { @@ -552,6 +522,73 @@ public void onCompleted() { return new XceiverClientReply(replyFuture); } + public CompletableFuture sendCommandOnlyRead( + ContainerCommandRequestProto request, ByteBuffer buffer, + List validators) throws SCMSecurityException { + XceiverClientReply reply = new XceiverClientReply(null); + List datanodeList = getDatanodeList(request); + CompletableFuture future = new CompletableFuture<>(); + for (DatanodeDetails dn : datanodeList) { + try { + checkOpen(dn); + UUID dnID = dn.getUuid(); + semaphore.acquire(); + final StreamObserver requestObserver = + asyncStubs.get(dnID).withDeadlineAfter(timeout, TimeUnit.SECONDS) + .send(new StreamObserver() { + @Override + public void onNext( + ContainerCommandResponseProto responseProto) { + for (Validator validator : validators) { + try { + validator.accept(request, responseProto); + } catch (IOException e) { + LOG.debug("Failed to execute command {} on datanode {}", + processForDebug(request), dn, e); + + } + } + ReadBlockResponseProto readChunkResponse = + responseProto.getReadBlock(); + if (readChunkResponse.hasData()) { + buffer.put(readChunkResponse.getData() + .asReadOnlyByteBuffer()); + } else if (readChunkResponse.hasDataBuffers()) { + readChunkResponse.getDataBuffers().getBuffersList() + .stream().forEach( + data -> buffer.put(data.asReadOnlyByteBuffer())); + } + } + + @Override + public void onError(Throwable t) { + future.completeExceptionally(t); + } + + @Override + public void onCompleted() { + future.complete(request.getReadBlock().getBlockID()); + semaphore.release(); + } + }); + reply.addDatanode(dn); + requestObserver.onNext(request); + requestObserver.onCompleted(); + } catch (IOException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Failed to execute command {} on datanode {}", + processForDebug(request), dn, e); + } + } catch (InterruptedException e) { + LOG.error("Command execution was interrupted ", e); + Thread.currentThread().interrupt(); + } + } + + + return future; + } + private synchronized void checkOpen(DatanodeDetails dn) throws IOException { if (closed) { @@ -612,4 +649,41 @@ public static Logger getLogger() { public void setTimeout(long timeout) { this.timeout = timeout; } + + private List getDatanodeList( + ContainerCommandRequestProto request) { + List datanodeList = null; + DatanodeBlockID blockID = null; + if (request.getCmdType() == ContainerProtos.Type.GetBlock) { + blockID = request.getGetBlock().getBlockID(); + } else if (request.getCmdType() == ContainerProtos.Type.ReadChunk) { + blockID = request.getReadChunk().getBlockID(); + } else if (request.getCmdType() == ContainerProtos.Type.GetSmallFile) { + blockID = request.getGetSmallFile().getBlock().getBlockID(); + } + + if (blockID != null) { + // Check if the DN to which the GetBlock command was sent has been cached. + DatanodeDetails cachedDN = getBlockDNcache.get(blockID); + if (cachedDN != null) { + datanodeList = pipeline.getNodes(); + int getBlockDNCacheIndex = datanodeList.indexOf(cachedDN); + if (getBlockDNCacheIndex > 0) { + // Pull the Cached DN to the top of the DN list + Collections.swap(datanodeList, 0, getBlockDNCacheIndex); + } + } + } + if (datanodeList == null) { + if (topologyAwareRead) { + datanodeList = pipeline.getNodesInOrder(); + } else { + datanodeList = pipeline.getNodes(); + // Shuffle datanode list so that clients do not read in the same order + // every time. + Collections.shuffle(datanodeList); + } + } + return datanodeList; + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClient.java new file mode 100644 index 000000000000..e347f8120315 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClient.java @@ -0,0 +1,29 @@ +/* + * 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 org.apache.hadoop.hdds.scm.storage.DataStreamClientImpl.DataStreamInput; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; + + +/** + * An interface to stream data. + */ +public interface DataStreamClient { + DataStreamInput streamReadOnly(ContainerProtos.ContainerCommandRequestProto request); +} diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClientImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClientImpl.java new file mode 100644 index 000000000000..bd4ca9b4cc67 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClientImpl.java @@ -0,0 +1,67 @@ +/* + * 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 org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.scm.XceiverClientGrpc; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.ratis.client.api.DataStreamOutput; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Streaming client implementation + * allows client to create streams and send asynchronously. + */ +public class DataStreamClientImpl { + + public DataStreamInput streamReadOnly( + ContainerCommandRequestProto request, XceiverClientSpi client, + List validators) { + return new DataStreamInput(request, client, validators); + } + + /** An asynchronous input stream. */ + public final class DataStreamInput { + private ContainerCommandRequestProto request; + private XceiverClientSpi client; + private List validators; + + DataStreamInput( + ContainerCommandRequestProto request, XceiverClientSpi client, + List validators) { + this.request = request; + this.client = client; + this.validators = validators; + } + CompletableFuture read(ByteBuffer buffer) { + try { + return ((XceiverClientGrpc) client).sendCommandOnlyRead(request, buffer, validators); + } catch (SCMSecurityException e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 718e2a108c77..d351a1bf7cba 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -196,6 +196,7 @@ message ContainerCommandRequestProto { optional GetBlockRequestProto getBlock = 13; optional DeleteBlockRequestProto deleteBlock = 14 [deprecated = true]; optional ListBlockRequestProto listBlock = 15; + optional ReadBlockRequestProto readBlock = 25; optional ReadChunkRequestProto readChunk = 16; optional WriteChunkRequestProto writeChunk = 17; @@ -228,6 +229,7 @@ message ContainerCommandResponseProto { optional GetBlockResponseProto getBlock = 12; optional DeleteBlockResponseProto deleteBlock = 13; optional ListBlockResponseProto listBlock = 14; + optional ReadBlockResponseProto readBlock = 22; optional WriteChunkResponseProto writeChunk = 15; optional ReadChunkResponseProto readChunk = 16; @@ -373,6 +375,23 @@ message ListBlockResponseProto { repeated BlockData blockData = 1; } +message ReadBlockRequestProto { + required DatanodeBlockID blockID = 1; + required uint64 offset = 2; + required uint64 len = 3; + optional ReadChunkVersion version = 4; +} + +message ReadBlockResponseProto { + required DatanodeBlockID blockID = 1; + required ChunkInfo chunkData = 2; + // Chunk data should be returned in one of the two for + oneof responseData { + bytes data = 3; // Chunk data is returned as single buffer for V0 + DataBuffers dataBuffers = 4; // Chunk data is returned as a list of buffers + } +} + // Chunk Operations message ChunkInfo { From d8a519de085977bce87d0e221e99f679e8d71458 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 11 Mar 2024 19:42:16 +0800 Subject: [PATCH 002/114] implement XceiverClientGrpc#sendCommandOnlyRead --- .../hdds/scm/storage/DataStreamClient.java | 29 -------- .../scm/storage/DataStreamClientImpl.java | 67 ------------------- 2 files changed, 96 deletions(-) delete mode 100644 hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClient.java delete mode 100644 hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClientImpl.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClient.java deleted file mode 100644 index e347f8120315..000000000000 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClient.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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 org.apache.hadoop.hdds.scm.storage.DataStreamClientImpl.DataStreamInput; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; - - -/** - * An interface to stream data. - */ -public interface DataStreamClient { - DataStreamInput streamReadOnly(ContainerProtos.ContainerCommandRequestProto request); -} diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClientImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClientImpl.java deleted file mode 100644 index bd4ca9b4cc67..000000000000 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamClientImpl.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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 org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; -import org.apache.hadoop.hdds.scm.XceiverClientGrpc; -import org.apache.hadoop.hdds.scm.XceiverClientSpi; -import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator; -import org.apache.hadoop.hdds.security.exception.SCMSecurityException; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.ratis.client.api.DataStreamOutput; - -import java.nio.ByteBuffer; -import java.util.List; -import java.util.concurrent.CompletableFuture; - -/** - * Streaming client implementation - * allows client to create streams and send asynchronously. - */ -public class DataStreamClientImpl { - - public DataStreamInput streamReadOnly( - ContainerCommandRequestProto request, XceiverClientSpi client, - List validators) { - return new DataStreamInput(request, client, validators); - } - - /** An asynchronous input stream. */ - public final class DataStreamInput { - private ContainerCommandRequestProto request; - private XceiverClientSpi client; - private List validators; - - DataStreamInput( - ContainerCommandRequestProto request, XceiverClientSpi client, - List validators) { - this.request = request; - this.client = client; - this.validators = validators; - } - CompletableFuture read(ByteBuffer buffer) { - try { - return ((XceiverClientGrpc) client).sendCommandOnlyRead(request, buffer, validators); - } catch (SCMSecurityException e) { - throw new RuntimeException(e); - } - } - } -} From ffebd667115d41b8fa5ec432e40e072022daff49 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 12 Mar 2024 08:07:01 +0800 Subject: [PATCH 003/114] read data to buffers --- .../apache/hadoop/hdds/scm/XceiverClientGrpc.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index dc1fda9806ac..ad47a36289c4 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -54,12 +54,14 @@ import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.opentracing.Scope; import io.opentracing.Span; import io.opentracing.util.GlobalTracer; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; @@ -523,7 +525,7 @@ public void onCompleted() { } public CompletableFuture sendCommandOnlyRead( - ContainerCommandRequestProto request, ByteBuffer buffer, + ContainerCommandRequestProto request, List buffers, List validators) throws SCMSecurityException { XceiverClientReply reply = new XceiverClientReply(null); List datanodeList = getDatanodeList(request); @@ -551,12 +553,13 @@ public void onNext( ReadBlockResponseProto readChunkResponse = responseProto.getReadBlock(); if (readChunkResponse.hasData()) { - buffer.put(readChunkResponse.getData() + buffers.add(readChunkResponse.getData() .asReadOnlyByteBuffer()); } else if (readChunkResponse.hasDataBuffers()) { - readChunkResponse.getDataBuffers().getBuffersList() - .stream().forEach( - data -> buffer.put(data.asReadOnlyByteBuffer())); + buffers.addAll(readChunkResponse.getDataBuffers() + .getBuffersList().stream() + .map(ByteString::asReadOnlyByteBuffer) + .collect(Collectors.toList())); } } From a6ed056c4dc45505a7f961f4601081b2b8aed360 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 00:58:12 +0800 Subject: [PATCH 004/114] create NewBlockInputStream to support Streaming data --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 119 ++- .../hdds/scm/storage/NewBlockInputStream.java | 677 ++++++++++++++++++ .../scm/storage/DummyNewBlockInputStream.java | 141 ++++ .../scm/storage/TestNewBlockInputStream.java | 301 ++++++++ .../main/proto/DatanodeClientProtocol.proto | 6 + 5 files changed, 1171 insertions(+), 73 deletions(-) create mode 100644 hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java create mode 100644 hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java create mode 100644 hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index ad47a36289c4..28d35ec6f45c 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.io.InterruptedIOException; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -54,14 +53,12 @@ import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.opentracing.Scope; import io.opentracing.Span; import io.opentracing.util.GlobalTracer; -import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; @@ -109,12 +106,12 @@ public class XceiverClientGrpc extends XceiverClientSpi { * Constructs a client that can communicate with the Container framework on * data nodes via DatanodeClientProtocol. * - * @param pipeline - Pipeline that defines the machines. - * @param config -- Ozone Config + * @param pipeline - Pipeline that defines the machines. + * @param config -- Ozone Config * @param trustManager - a {@link ClientTrustManager} with proper CA handling. */ public XceiverClientGrpc(Pipeline pipeline, ConfigurationSource config, - ClientTrustManager trustManager) { + ClientTrustManager trustManager) { super(); Preconditions.checkNotNull(pipeline); Preconditions.checkNotNull(config); @@ -366,7 +363,11 @@ private XceiverClientReply sendCommandWithRetry( // sendCommandAsyncCall will create a new channel and async stub // in case these don't exist for the specific datanode. reply.addDatanode(dn); - responseProto = sendCommandAsync(request, dn).getResponse().get(); + if (responseProto.getCmdType() == ContainerProtos.Type.ReadBlock) { + responseProto = sendCommandAsyncReadOnly(request, dn).getResponse().get(); + } else { + responseProto = sendCommandAsync(request, dn).getResponse().get(); + } if (validators != null && !validators.isEmpty()) { for (Validator validator : validators) { validator.accept(request, responseProto); @@ -410,10 +411,10 @@ private XceiverClientReply sendCommandWithRetry( String message = "Failed to execute command {}"; if (LOG.isDebugEnabled()) { LOG.debug(message + " on the pipeline {}.", - processForDebug(request), pipeline); + processForDebug(request), pipeline); } else { LOG.error(message + " on the pipeline {}.", - request.getCmdType(), pipeline); + request.getCmdType(), pipeline); } throw ioException; } @@ -524,72 +525,44 @@ public void onCompleted() { return new XceiverClientReply(replyFuture); } - public CompletableFuture sendCommandOnlyRead( - ContainerCommandRequestProto request, List buffers, - List validators) throws SCMSecurityException { - XceiverClientReply reply = new XceiverClientReply(null); - List datanodeList = getDatanodeList(request); - CompletableFuture future = new CompletableFuture<>(); - for (DatanodeDetails dn : datanodeList) { - try { - checkOpen(dn); - UUID dnID = dn.getUuid(); - semaphore.acquire(); - final StreamObserver requestObserver = - asyncStubs.get(dnID).withDeadlineAfter(timeout, TimeUnit.SECONDS) - .send(new StreamObserver() { - @Override - public void onNext( - ContainerCommandResponseProto responseProto) { - for (Validator validator : validators) { - try { - validator.accept(request, responseProto); - } catch (IOException e) { - LOG.debug("Failed to execute command {} on datanode {}", - processForDebug(request), dn, e); - - } - } - ReadBlockResponseProto readChunkResponse = - responseProto.getReadBlock(); - if (readChunkResponse.hasData()) { - buffers.add(readChunkResponse.getData() - .asReadOnlyByteBuffer()); - } else if (readChunkResponse.hasDataBuffers()) { - buffers.addAll(readChunkResponse.getDataBuffers() - .getBuffersList().stream() - .map(ByteString::asReadOnlyByteBuffer) - .collect(Collectors.toList())); - } - } - - @Override - public void onError(Throwable t) { - future.completeExceptionally(t); - } - - @Override - public void onCompleted() { - future.complete(request.getReadBlock().getBlockID()); - semaphore.release(); - } - }); - reply.addDatanode(dn); - requestObserver.onNext(request); - requestObserver.onCompleted(); - } catch (IOException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Failed to execute command {} on datanode {}", - processForDebug(request), dn, e); - } - } catch (InterruptedException e) { - LOG.error("Command execution was interrupted ", e); - Thread.currentThread().interrupt(); - } - } + public XceiverClientReply sendCommandAsyncReadOnly( + ContainerCommandRequestProto request, DatanodeDetails dn) + throws IOException, InterruptedException { + CompletableFuture future = + new CompletableFuture<>(); + ContainerCommandResponseProto.Builder response = + ContainerCommandResponseProto.newBuilder(); + ContainerProtos.StreamDataResponseProto.Builder streamData = + ContainerProtos.StreamDataResponseProto.newBuilder(); + checkOpen(dn); + UUID dnID = dn.getUuid(); + semaphore.acquire(); + final StreamObserver requestObserver = + asyncStubs.get(dnID).withDeadlineAfter(timeout, TimeUnit.SECONDS) + .send(new StreamObserver() { + @Override + public void onNext( + ContainerCommandResponseProto responseProto) { + ReadBlockResponseProto readBlock = + responseProto.getReadBlock(); + streamData.addReadBlock(readBlock); + } - return future; + @Override + public void onError(Throwable t) { + future.completeExceptionally(t); + } + + @Override + public void onCompleted() { + semaphore.release(); + future.complete(response.setStreamData(streamData).build()); + } + }); + requestObserver.onNext(request); + requestObserver.onCompleted(); + return new XceiverClientReply(future); } private synchronized void checkOpen(DatanodeDetails dn) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java new file mode 100644 index 000000000000..4a6cf3698ff5 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -0,0 +1,677 @@ +/* + * 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.fs.ByteBufferReadable; +import org.apache.hadoop.fs.CanUnbuffer; +import org.apache.hadoop.fs.Seekable; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.client.HddsClientUtils; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; +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.ozone.common.utils.BufferUtils; +import org.apache.hadoop.security.token.Token; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; + +import static org.apache.hadoop.hdds.client.ReplicationConfig.getLegacyFactor; + +/** + * An {@link java.io.InputStream} called from KeyInputStream to read a block from the + * container. + */ +public class NewBlockInputStream extends InputStream + implements Seekable, CanUnbuffer, ByteBufferReadable { + private static final Logger LOG = + LoggerFactory.getLogger(NewBlockInputStream.class); + private final BlockID blockID; + private final long length; + private final AtomicReference pipelineRef = + new AtomicReference<>(); + private final AtomicReference> tokenRef = + new AtomicReference<>(); + private XceiverClientFactory xceiverClientFactory; + private XceiverClientSpi xceiverClient; + + private List bufferoffsets; + private int bufferIndex; + private long blockPosition = -1; + private List buffers; + private boolean allocated = false; + private long bufferOffsetWrtBlockDataData; + private long buffersSize; + private static final int EOF = -1; + private final List validators; + private final boolean verifyChecksum; + private final Function refreshFunction; + private final RetryPolicy retryPolicy = + HddsClientUtils.createRetryPolicy(3, TimeUnit.SECONDS.toMillis(1)); + private int retries; + + + public NewBlockInputStream( + BlockID blockID, long length, Pipeline pipeline, + Token token, boolean verifyChecksum, + XceiverClientFactory xceiverClientFactory, + Function refreshFunction) { + this.blockID = blockID; + this.length = length; + setPipeline(pipeline); + tokenRef.set(token); + this.xceiverClientFactory = xceiverClientFactory; + this.validators = ContainerProtocolCalls.toValidatorList( + (request, response) -> validateBlock(response)); + this.verifyChecksum = verifyChecksum; + this.refreshFunction = refreshFunction; + + } + + + public BlockID getBlockID() { + return blockID; + } + + public long getLength() { + return length; + } + + @Override + public long getPos() { + if (length == 0) { + return 0; + } + if (blockPosition >= 0) { + return blockPosition; + } + + if (allocated && !buffersHaveData() && !dataRemainingInBlock()) { + Preconditions.checkState( + bufferOffsetWrtBlockDataData + buffersSize == length, + "EOF detected but not at the last byte of the chunk"); + return length; + } + if (buffersHaveData()) { + // BufferOffset w.r.t to ChunkData + BufferOffset w.r.t buffers + + // Position of current Buffer + return bufferOffsetWrtBlockDataData + bufferoffsets.get(bufferIndex) + + buffers.get(bufferIndex).position(); + } + if (buffersAllocated()) { + return bufferOffsetWrtBlockDataData + buffersSize; + } + return 0; + } + + @Override + public synchronized int read() throws IOException { + checkOpen(); + while (true) { + int dataout = EOF; + try { + + + acquireClient(); + int available = prepareRead(1); + + + if (available == EOF) { + // There is no more data in the chunk stream. The buffers should have + // been released by now + Preconditions.checkState(buffers == null); + } else { + dataout = Byte.toUnsignedInt(buffers.get(bufferIndex).get()); + } + + if (bufferEOF()) { + releaseBuffers(bufferIndex); + } + } catch (SCMSecurityException ex) { + throw ex; + } catch (StorageContainerException e) { + if (shouldRetryRead(e)) { + releaseClient(); + } else { + throw e; + } + } + return dataout; + } + + } + + @Override + public synchronized int read(byte[] b, int off, int len) throws IOException { + // According to the JavaDocs for InputStream, it is recommended that + // subclasses provide an override of bulk read if possible for performance + // reasons. In addition to performance, we need to do it for correctness + // reasons. The Ozone REST service uses PipedInputStream and + // PipedOutputStream to relay HTTP response data between a Jersey thread and + // a Netty thread. It turns out that PipedInputStream/PipedOutputStream + // have a subtle dependency (bug?) on the wrapped stream providing separate + // implementations of single-byte read and bulk read. Without this, get key + // responses might close the connection before writing all of the bytes + // advertised in the Content-Length. + if (b == null) { + throw new NullPointerException(); + } + if (off < 0 || len < 0 || len > b.length - off) { + throw new IndexOutOfBoundsException(); + } + if (len == 0) { + return 0; + } + int total = 0; + while (true) { + try { + acquireClient(); + while (len > 0) { + int available = prepareRead(len); + if (available == EOF) { + // There is no more data in the chunk stream. The buffers should have + // been released by now + Preconditions.checkState(buffers == null); + return total != 0 ? total : EOF; + } + buffers.get(bufferIndex).get(b, off + total, available); + len -= available; + total += available; + + if (bufferEOF()) { + releaseBuffers(bufferIndex); + } + } + } catch (SCMSecurityException ex) { + throw ex; + } catch (StorageContainerException e) { + if (shouldRetryRead(e)) { + releaseClient(); + } else { + throw e; + } + } + return total; + } + } + + @Override + public void close() throws IOException { + releaseClient(); + releaseBuffers(); + xceiverClientFactory = null; + } + + @Override + public synchronized int read(ByteBuffer byteBuffer) throws IOException { + if (byteBuffer == null) { + throw new NullPointerException(); + } + int len = byteBuffer.remaining(); + if (len == 0) { + return 0; + } + int total = 0; + while (true) { + try { + acquireClient(); + while (len > 0) { + int available = prepareRead(len); + if (available == EOF) { + // There is no more data in the chunk stream. The buffers should have + // been released by now + Preconditions.checkState(buffers == null); + return total != 0 ? total : EOF; + } + ByteBuffer readBuf = buffers.get(bufferIndex); + ByteBuffer tmpBuf = readBuf.duplicate(); + tmpBuf.limit(tmpBuf.position() + available); + byteBuffer.put(tmpBuf); + readBuf.position(tmpBuf.position()); + + len -= available; + total += available; + + if (bufferEOF()) { + releaseBuffers(bufferIndex); + } + } + } catch (SCMSecurityException ex) { + throw ex; + } catch (StorageContainerException e) { + if (shouldRetryRead(e)) { + releaseClient(); + } else { + throw e; + } + } + return total; + + } + } + + @Override + public synchronized void seek(long pos) throws IOException { + if (pos < 0 || pos > length) { + if (pos == 0) { + // It is possible for length and pos to be zero in which case + // seek should return instead of throwing exception + return; + } + throw new EOFException("EOF encountered at pos: " + pos + " for block: " + blockID); + } + + if (buffersHavePosition(pos)) { + // The bufferPosition is w.r.t the current chunk. + // Adjust the bufferIndex and position to the seeked position. + adjustBufferPosition(pos - bufferOffsetWrtBlockDataData); + } else { + blockPosition = pos; + } + } + + @Override + public synchronized boolean seekToNewSource(long l) throws IOException { + return false; + } + + @Override + public synchronized void unbuffer() { + blockPosition = getPos(); + releaseClient(); + releaseBuffers(); + } + + private void setPipeline(Pipeline pipeline) { + if (pipeline == null) { + return; + } + + // irrespective of the container state, we will always read via Standalone + // protocol. + boolean okForRead = + pipeline.getType() == HddsProtos.ReplicationType.STAND_ALONE + || pipeline.getType() == HddsProtos.ReplicationType.EC; + Pipeline readPipeline = okForRead ? pipeline : Pipeline.newBuilder(pipeline) + .setReplicationConfig(StandaloneReplicationConfig.getInstance( + getLegacyFactor(pipeline.getReplicationConfig()))) + .build(); + pipelineRef.set(readPipeline); + } + + protected synchronized void checkOpen() throws IOException { + if (xceiverClientFactory == null) { + throw new IOException("BlockInputStream has been closed."); + } + } + + protected synchronized void acquireClient() throws IOException { + if (xceiverClientFactory != null && xceiverClient == null) { + xceiverClient = xceiverClientFactory.acquireClientForReadData( + pipelineRef.get()); + } + } + + private synchronized int prepareRead(int len) throws IOException { + for (;;) { + if (blockPosition >= 0) { + if (buffersHavePosition(blockPosition)) { + // The current buffers have the seeked position. Adjust the buffer + // index and position to point to the chunkPosition. + adjustBufferPosition(blockPosition - bufferOffsetWrtBlockDataData); + } else { + // Read a required chunk data to fill the buffers with seeked + // position data + readDataFromContainer(len); + } + } + if (buffersHaveData()) { + // Data is available from buffers + ByteBuffer bb = buffers.get(bufferIndex); + return Math.min(len, bb.remaining()); + } else if (dataRemainingInBlock()) { + // There is more data in the chunk stream which has not + // been read into the buffers yet. + readDataFromContainer(len); + } else { + // All available input from this chunk stream has been consumed. + return EOF; + } + } + + + } + + private boolean buffersHavePosition(long pos) { + // Check if buffers have been allocated + if (buffersAllocated()) { + // Check if the current buffers cover the input position + // Released buffers should not be considered when checking if position + // is available + return pos >= bufferOffsetWrtBlockDataData + + bufferoffsets.get(0) && + pos < bufferOffsetWrtBlockDataData + buffersSize; + } + return false; + } + + /** + * Check if the buffers have been allocated data and false otherwise. + */ + @VisibleForTesting + protected boolean buffersAllocated() { + return buffers != null && !buffers.isEmpty(); + } + + /** + * Adjust the buffers position to account for seeked position and/ or checksum + * boundary reads. + * @param bufferPosition the position to which the buffers must be advanced + */ + private void adjustBufferPosition(long bufferPosition) { + // The bufferPosition is w.r.t the current buffers. + // Adjust the bufferIndex and position to the seeked bufferPosition. + if (bufferIndex >= buffers.size()) { + bufferIndex = Collections.binarySearch(bufferoffsets, bufferPosition); + } else if (bufferPosition < bufferoffsets.get(bufferIndex)) { + bufferIndex = Collections.binarySearch( + bufferoffsets.subList(0, bufferIndex), bufferPosition); + } else if (bufferPosition >= bufferoffsets.get(bufferIndex) + + buffers.get(bufferIndex).limit()) { + bufferIndex = Collections.binarySearch(bufferoffsets.subList( + bufferIndex + 1, buffers.size()), bufferPosition); + } + if (bufferIndex < 0) { + bufferIndex = -bufferIndex - 2; + } + + buffers.get(bufferIndex).position( + (int) (bufferPosition - bufferoffsets.get(bufferIndex))); + + // Reset buffers > bufferIndex to position 0. We do this to reset any + // previous reads/ seeks which might have updated any buffer position. + // For buffers < bufferIndex, we do not need to reset the position as it + // not required for this read. If a seek was done to a position in the + // previous indices, the buffer position reset would be performed in the + // seek call. + for (int i = bufferIndex + 1; i < buffers.size(); i++) { + buffers.get(i).position(0); + } + + // Reset the chunkPosition as chunk stream has been initialized i.e. the + // buffers have been allocated. + blockPosition = -1; + } + + /** + * Reads full or partial Chunk from DN Container based on the current + * position of the ChunkInputStream, the number of bytes of data to read + * and the checksum boundaries. + * If successful, then the read data in saved in the buffers so that + * subsequent read calls can utilize it. + * @param len number of bytes of data to be read + * @throws IOException if there is an I/O error while performing the call + * to Datanode + */ + private void readDataFromContainer(int len) throws IOException { + // index of first byte to be read from the chunk + long startByteIndex; + if (blockPosition >= 0) { + // If seek operation was called to advance the buffer position, the + // chunk should be read from that position onwards. + startByteIndex = blockPosition; + } else { + // Start reading the chunk from the last chunkPosition onwards. + startByteIndex = bufferOffsetWrtBlockDataData + buffersSize; + } + + // bufferOffsetWrtChunkData and buffersSize are updated after the data + // is read from Container and put into the buffers, but if read fails + // and is retried, we need the previous position. Position is reset after + // successful read in adjustBufferPosition() + blockPosition = getPos(); + + // Adjust the chunkInfo so that only the required bytes are read from + // the chunk. + + bufferOffsetWrtBlockDataData = readData(startByteIndex, len); + long tempOffset = 0L; + bufferoffsets = new ArrayList<>(buffers.size()); + for (ByteBuffer buffer : buffers) { + bufferoffsets.add(tempOffset); + tempOffset += buffer.limit(); + buffersSize += buffer.limit(); + + } + bufferIndex = 0; + allocated = true; + // If the stream was seeked to position before, then the buffer + // position should be adjusted as the reads happen at checksum boundaries. + // The buffers position might need to be adjusted for the following + // scenarios: + // 1. Stream was seeked to a position before the chunk was read + // 2. Chunk was read from index < the current position to account for + // checksum boundaries. + adjustBufferPosition(startByteIndex - bufferOffsetWrtBlockDataData); + + } + + @VisibleForTesting + protected long readData(long startByteIndex, long len) + throws IOException { + ReadBlockRequestProto.Builder readBlockRequest = + ReadBlockRequestProto.newBuilder() + .setBlockID(blockID.getDatanodeBlockIDProtobuf()) + .setLen(startByteIndex) + .setOffset(len) + .setVersion(ContainerProtos.ReadChunkVersion.V1); + ContainerCommandRequestProto request = ContainerCommandRequestProto + .newBuilder().setReadBlock(readBlockRequest).build(); + ContainerProtos.ContainerCommandResponseProto response = + xceiverClient.sendCommand(request, validators); + response.getStreamData().getReadBlockList().stream().map(readBlock -> { + if (readBlock.hasData()) { + return this.buffers.add(readBlock.getData().asReadOnlyByteBuffer()); + } else if (readBlock.hasDataBuffers()) { + return this.buffers.addAll(BufferUtils.getReadOnlyByteBuffers( + readBlock.getDataBuffers().getBuffersList())); + } else { + throw new RuntimeException("Unexpected error while reading chunk data " + + "from container. No data returned."); + } + }); + return response.getStreamData().getReadBlock(0) + .getChunkData().getOffset(); + } + + /** + * Check if the buffers have any data remaining between the current + * position and the limit. + */ + private boolean buffersHaveData() { + boolean hasData = false; + if (buffersAllocated()) { + int buffersLen = buffers.size(); + while (bufferIndex < buffersLen) { + ByteBuffer buffer = buffers.get(bufferIndex); + if (buffer != null && buffer.hasRemaining()) { + // current buffer has data + hasData = true; + break; + } else { + if (bufferIndex < buffersLen - 1) { + // move to next available buffer + ++bufferIndex; + Preconditions.checkState(bufferIndex < buffers.size()); + } else { + // no more buffers remaining + break; + } + } + } + } + + return hasData; + } + + /** + * Check if there is more data in the chunk which has not yet been read + * into the buffers. + */ + private boolean dataRemainingInBlock() { + long bufferPos; + if (blockPosition >= 0) { + bufferPos = blockPosition; + } else { + bufferPos = bufferOffsetWrtBlockDataData + buffersSize; + } + + return bufferPos < length; + } + + /** + * Check if current buffer had been read till the end. + */ + private boolean bufferEOF() { + return allocated && !buffers.get(bufferIndex).hasRemaining(); + } + + /** + * Release the buffers upto the given index. + * @param releaseUptoBufferIndex bufferIndex (inclusive) upto which the + * buffers must be released + */ + private void releaseBuffers(int releaseUptoBufferIndex) { + int buffersLen = buffers.size(); + if (releaseUptoBufferIndex == buffersLen - 1) { + // Before releasing all the buffers, if chunk EOF is not reached, then + // chunkPosition should be set to point to the last position of the + // buffers. This should be done so that getPos() can return the current + // chunk position + blockPosition = bufferOffsetWrtBlockDataData + + bufferoffsets.get(releaseUptoBufferIndex) + + buffers.get(releaseUptoBufferIndex).capacity(); + // Release all the buffers + releaseBuffers(); + } else { + buffers = buffers.subList(releaseUptoBufferIndex + 1, buffersLen); + bufferoffsets = bufferoffsets.subList( + releaseUptoBufferIndex + 1, buffersLen); + } + } + + /** + * If EOF is reached, release the buffers. + */ + private void releaseBuffers() { + buffers = null; + bufferIndex = 0; + // We should not reset bufferOffsetWrtChunkData and buffersSize here + // because when getPos() is called in chunkStreamEOF() we use these + // values and determine whether chunk is read completely or not. + } + + protected synchronized void releaseClient() { + if (xceiverClientFactory != null && xceiverClient != null) { + xceiverClientFactory.releaseClientForReadData(xceiverClient, false); + xceiverClient = null; + } + } + + private void validateBlock( + ContainerProtos.ContainerCommandResponseProto response + ) throws OzoneChecksumException { + + ContainerProtos.ReadBlockResponseProto readBlock = response.getReadBlock(); + List byteStrings; + boolean isV0 = false; + + if (readBlock.hasData()) { + ByteString byteString = readBlock.getData(); + byteStrings = new ArrayList<>(); + byteStrings.add(byteString); + isV0 = true; + } else { + byteStrings = readBlock.getDataBuffers().getBuffersList(); + } + ContainerProtos.ChunkInfo chunkInfo = + readBlock.getChunkData(); + if (verifyChecksum) { + ChecksumData checksumData = ChecksumData.getFromProtoBuf( + chunkInfo.getChecksumData()); + + // ChecksumData stores checksum for each 'numBytesPerChecksum' + // number of bytes in a list. Compute the index of the first + // checksum to match with the read data + + long relativeOffset = chunkInfo.getOffset() - byteStrings.size(); + int bytesPerChecksum = checksumData.getBytesPerChecksum(); + int startIndex = (int) (relativeOffset / bytesPerChecksum); + Checksum.verifyChecksum(byteStrings, checksumData, startIndex, + isV0); + } + } + + @VisibleForTesting + protected void setBuffers(List buffers) { + this.buffers = buffers; + } + + private boolean shouldRetryRead(IOException cause) throws IOException { + RetryPolicy.RetryAction retryAction; + try { + retryAction = retryPolicy.shouldRetry(cause, ++retries, 0, true); + } catch (IOException e) { + throw e; + } catch (Exception e) { + throw new IOException(e); + } + return retryAction.action == RetryPolicy.RetryAction.RetryDecision.RETRY; + } + + @VisibleForTesting + public boolean isVerifyChecksum() { + return verifyChecksum; + } +} diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java new file mode 100644 index 000000000000..969a7936faf6 --- /dev/null +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java @@ -0,0 +1,141 @@ +/* + * 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.util.*; +import java.util.function.Function; + +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; +import org.apache.hadoop.ozone.common.ChecksumData; +import org.apache.hadoop.ozone.common.utils.BufferUtils; +import org.apache.hadoop.security.token.Token; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; + +/** + * A dummy NewBlockInputStream to mock read block call to DN. + */ +class DummyNewBlockInputStream extends NewBlockInputStream { + + private final List readByteBuffers = new ArrayList<>(); + private final List chunks; + private final long[] chunkOffsets; + private final Map chunkDataMap; + + @SuppressWarnings("parameternumber") + DummyNewBlockInputStream( + BlockID blockId, + long blockLen, + Pipeline pipeline, + Token token, + boolean verifyChecksum, + XceiverClientFactory xceiverClientManager, + Function refreshFunction, + List chunks, + Map chunkDataMap) { + super(blockId, blockLen, pipeline, token, verifyChecksum, + xceiverClientManager, refreshFunction); + this.chunks = chunks; + this.chunkDataMap = chunkDataMap; + chunkOffsets = new long[chunks.size()]; + long temp = 0; + for (int i = 0; i < chunks.size(); i++) { + chunkOffsets[i] = temp; + temp += chunks.get(i).getLen(); + } + } + + @Override + protected synchronized void checkOpen() throws IOException { + // No action needed + } + + @Override + protected void acquireClient() { + // No action needed + } + + @Override + protected void releaseClient() { + // no-op + } + + @Override + protected long readData(long offset, long len) { + int chunkIndex = Arrays.binarySearch(chunkOffsets, offset); + if (chunkIndex < 0) { + chunkIndex = -chunkIndex - 2; + } + ChunkInfo chunkInfo = chunks.get(chunkIndex); + readByteBuffers.clear(); + long chunkOffset = offset - chunkInfo.getOffset(); + if (isVerifyChecksum()) { + ChecksumData checksumData = ChecksumData.getFromProtoBuf( + chunkInfo.getChecksumData()); + int bytesPerChecksum = checksumData.getBytesPerChecksum(); + chunkOffset = (chunkOffset / bytesPerChecksum) * bytesPerChecksum; + } + long bufferOffsetWrtBlockDataData = chunkOffsets[chunkIndex] + chunkOffset; + while (len > 0) { + ChunkInfo currentChunk = chunks.get(chunkIndex); + int bufferCapacity = currentChunk.getChecksumData().getBytesPerChecksum(); + long chunkLen = currentChunk.getLen(); + long remainingToRead = Math.min(chunkLen, len); + if (isVerifyChecksum()) { + if (len < chunkLen) { + ChecksumData checksumData = ChecksumData.getFromProtoBuf( + chunkInfo.getChecksumData()); + int bytesPerChecksum = checksumData.getBytesPerChecksum(); + remainingToRead =( len / bytesPerChecksum + 1) * bytesPerChecksum; + } else { + remainingToRead = chunkLen; + } + } + + long bufferLen; + while (remainingToRead > 0) { + if (remainingToRead < bufferCapacity) { + bufferLen = remainingToRead; + } else { + bufferLen = bufferCapacity; + } + System.out.println(bufferLen); + ByteString byteString = ByteString.copyFrom(chunkDataMap.get(chunks.get(chunkIndex).getChunkName()), + (int) chunkOffset, (int) bufferLen); + + readByteBuffers.add(byteString); + + chunkOffset += bufferLen; + remainingToRead -= bufferLen; + len -= bufferLen; + } + chunkOffset = 0; + chunkIndex++; + } + setBuffers(BufferUtils.getReadOnlyByteBuffers(readByteBuffers)); + return bufferOffsetWrtBlockDataData; + } + + public List getReadByteBuffers() { + return readByteBuffers; + } +} diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java new file mode 100644 index 000000000000..b93293c0e0f4 --- /dev/null +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java @@ -0,0 +1,301 @@ +/* + * 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.EOFException; +import java.util.*; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; + +import com.google.common.primitives.Bytes; +import com.sun.tools.javac.util.Log; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.client.ContainerBlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.ozone.common.Checksum; + +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +/** + * Tests for {@link TestNewBlockInputStream}'s functionality. + */ +public class TestNewBlockInputStream { + private int blockSize; + private static final int CHUNK_SIZE = 100; + private static final int BYTES_PER_CHECKSUM = 20; + private static final Random RANDOM = new Random(); + private static final AtomicLong CONTAINER_ID = new AtomicLong(); + private DummyNewBlockInputStream blockStream; + private byte[] blockData; + private List chunks; + private Map chunkDataMap; + private Checksum checksum; + private Function refreshFunction; + private BlockID blockID; + private static String CHUNK_NAME = "chunk-"; + + + @BeforeEach + public void setup() throws Exception { + refreshFunction = mock(Function.class); + blockID = new BlockID(new ContainerBlockID(1, 1)); + checksum = new Checksum(ChecksumType.CRC32, BYTES_PER_CHECKSUM); + createChunkList(5); + + Pipeline pipeline = MockPipeline.createSingleNodePipeline(); + blockStream = new DummyNewBlockInputStream(blockID, blockSize, pipeline, + null, true, null, refreshFunction, chunks, chunkDataMap); + } + + /** + * Create a mock list of chunks. The first n-1 chunks of length CHUNK_SIZE + * and the last chunk with length CHUNK_SIZE/2. + */ + private void createChunkList(int numChunks) + throws Exception { + + chunks = new ArrayList<>(numChunks); + chunkDataMap = new HashMap<>(); + blockData = new byte[0]; + int i, chunkLen; + byte[] byteData; + String chunkName; + + for (i = 0; i < numChunks; i++) { + chunkName = CHUNK_NAME + i; + chunkLen = CHUNK_SIZE; + if (i == numChunks - 1) { + chunkLen = CHUNK_SIZE / 2; + } + byteData = generateRandomData(chunkLen); + ChunkInfo chunkInfo = ChunkInfo.newBuilder() + .setChunkName(chunkName) + .setOffset(0) + .setLen(chunkLen) + .setChecksumData(checksum.computeChecksum( + byteData, 0, chunkLen).getProtoBufMessage()) + .build(); + + chunkDataMap.put(chunkName, byteData); + chunks.add(chunkInfo); + + blockSize += chunkLen; + blockData = Bytes.concat(blockData, byteData); + } + } + + static byte[] generateRandomData(int length) { + byte[] bytes = new byte[length]; + RANDOM.nextBytes(bytes); + return bytes; + } + + /** + * Match readData with the chunkData byte-wise. + * @param readData Data read through ChunkInputStream + * @param inputDataStartIndex first index (inclusive) in chunkData to compare + * with read data + * @param length the number of bytes of data to match starting from + * inputDataStartIndex + */ + private void matchWithInputData(byte[] readData, int inputDataStartIndex, + int length) { + for (int i = inputDataStartIndex; i < inputDataStartIndex + length; i++) { + assertEquals(blockData[i], readData[i - inputDataStartIndex], "i: " + i); + } + } + + private void matchWithInputData(List byteStrings, + int inputDataStartIndex, int length) { + int offset = inputDataStartIndex; + int totalBufferLen = 0; + for (ByteString byteString : byteStrings) { + int bufferLen = byteString.size(); + matchWithInputData(byteString.toByteArray(), offset, bufferLen); + offset += bufferLen; + totalBufferLen += bufferLen; + } + assertEquals(length, totalBufferLen); + } + + /** + * Seek to a position and verify through getPos(). + */ + private void seekAndVerify(int pos) throws Exception { + blockStream.seek(pos); + assertEquals(pos, blockStream.getPos(), + "Current position of buffer does not match with the sought position"); + } + + @Test + public void testFullChunkRead() throws Exception { + byte[] b = new byte[blockSize]; + blockStream.read(b, 0, blockSize); + matchWithInputData(b, 0, blockSize); + } + + @Test + public void testPartialChunkRead() throws Exception { + int len = blockSize / 2; + byte[] b = new byte[len]; + + blockStream.read(b, 0, len); + + matchWithInputData(b, 0, len); + + // To read chunk data from index 0 to 49 (len = 50), we need to read + // chunk from offset 0 to 60 as the checksum boundary is at every 20 + // bytes. Verify that 60 bytes of chunk data are read and stored in the + // buffers. Since checksum boundary is at every 20 bytes, there should be + // 60/20 number of buffers. + matchWithInputData(blockStream.getReadByteBuffers(), 0, 240); + } + + @Test + public void testSeek() throws Exception { + seekAndVerify(0); + EOFException eofException = assertThrows(EOFException.class, () -> seekAndVerify( blockSize+ 1)); + assertThat(eofException).hasMessage("EOF encountered at pos: " + (blockSize + 1) + " for block: " + blockID); + + // Seek before read should update the ChunkInputStream#chunkPosition + seekAndVerify(25); + + // Read from the sought position. + // Reading from index 25 to 54 should result in the ChunkInputStream + // copying chunk data from index 20 to 59 into the buffers (checksum + // boundaries). + byte[] b = new byte[30]; + blockStream.read(b, 0, 30); + matchWithInputData(b, 25, 30); + matchWithInputData(blockStream.getReadByteBuffers(), 20, 40); + + // After read, the position of the chunkStream is evaluated from the + // buffers and the chunkPosition should be reset to -1. + + // Only the last BYTES_PER_CHECKSUM will be cached in the buffers as + // buffers are released after each checksum boundary is read. So the + // buffers should contain data from index 40 to 59. + // Seek to a position within the cached buffers. ChunkPosition should + // still not be used to set the position. + seekAndVerify(45); + + // Seek to a position outside the current cached buffers. In this case, the + // chunkPosition should be updated to the seeked position. + seekAndVerify(75); + + // Read upto checksum boundary should result in all the buffers being + // released and hence chunkPosition updated with current position of chunk. + seekAndVerify(25); + b = new byte[15]; + blockStream.read(b, 0, 15); + matchWithInputData(b, 25, 15); + } + + @Test + public void testSeekAndRead() throws Exception { + // Seek to a position and read data + seekAndVerify(50); + byte[] b1 = new byte[20]; + blockStream.read(b1, 0, 20); + matchWithInputData(b1, 50, 20); + + // Next read should start from the position of the last read + 1 i.e. 70 + byte[] b2 = new byte[20]; + blockStream.read(b2, 0, 20); + matchWithInputData(b2, 70, 20); + } + + @Test + public void testUnbuffered() throws Exception { + byte[] b1 = new byte[20]; + blockStream.read(b1, 0, 20); + matchWithInputData(b1, 0, 20); + + blockStream.unbuffer(); + + assertFalse(blockStream.buffersAllocated()); + + // Next read should start from the position of the last read + 1 i.e. 20 + byte[] b2 = new byte[20]; + blockStream.read(b2, 0, 20); + matchWithInputData(b2, 20, 20); + } + +// @Test +// public void connectsToNewPipeline() throws Exception { +// // GIVEN +// Pipeline pipeline = MockPipeline.createSingleNodePipeline(); +// Pipeline newPipeline = MockPipeline.createSingleNodePipeline(); +// +// Token token = mock(Token.class); +// when(token.encodeToUrlString()) +// .thenReturn("oldToken"); +// Token newToken = mock(Token.class); +// when(newToken.encodeToUrlString()) +// .thenReturn("newToken"); +// +// AtomicReference pipelineRef = new AtomicReference<>(pipeline); +// AtomicReference> tokenRef = new AtomicReference<>(token); +// +// XceiverClientFactory clientFactory = mock(XceiverClientFactory.class); +// XceiverClientSpi client = mock(XceiverClientSpi.class); +// when(clientFactory.acquireClientForReadData(any())) +// .thenReturn(client); +// ArgumentCaptor requestCaptor = +// ArgumentCaptor.forClass(ContainerCommandRequestProto.class); +// when(client.getPipeline()) +// .thenAnswer(invocation -> pipelineRef.get()); +// when(client.sendCommand(requestCaptor.capture(), any())) +// .thenAnswer(invocation -> +// getReadChunkResponse( +// requestCaptor.getValue(), +// ChunkBuffer.wrap(ByteBuffer.wrap(blockData)), +// ByteStringConversion::safeWrap)); +// +// try (BlockInputStream subject = new ChunkInputStream(blockStream, blockID, +// clientFactory, pipelineRef::get, false, tokenRef::get)) { +// // WHEN +// subject.unbuffer(); +// pipelineRef.set(newPipeline); +// tokenRef.set(newToken); +// byte[] buffer = new byte[CHUNK_SIZE]; +// int read = subject.read(buffer); +// +// // THEN +// assertEquals(CHUNK_SIZE, read); +// assertArrayEquals(blockData, buffer); +// verify(clientFactory).acquireClientForReadData(newPipeline); +// verify(newToken).encodeToUrlString(); +// } +// } +} diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index d351a1bf7cba..117aeaa8d79e 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -88,6 +88,7 @@ enum Type { GetBlock = 7; DeleteBlock = 8; ListBlock = 9; + ReadBlock = 21; ReadChunk = 10; DeleteChunk = 11; @@ -230,6 +231,7 @@ message ContainerCommandResponseProto { optional DeleteBlockResponseProto deleteBlock = 13; optional ListBlockResponseProto listBlock = 14; optional ReadBlockResponseProto readBlock = 22; + optional StreamDataResponseProto streamData = 23; optional WriteChunkResponseProto writeChunk = 15; optional ReadChunkResponseProto readChunk = 16; @@ -392,6 +394,10 @@ message ReadBlockResponseProto { } } +message StreamDataResponseProto { + repeated ReadBlockResponseProto readBlock = 1; +} + // Chunk Operations message ChunkInfo { From fbd20eba26dcae5a011ab310824e758e28bd78dd Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 17:55:39 +0800 Subject: [PATCH 005/114] fix checkstyle --- .../org/apache/hadoop/hdds/scm/XceiverClientGrpc.java | 2 +- .../hadoop/hdds/scm/storage/NewBlockInputStream.java | 2 +- .../hdds/scm/storage/TestNewBlockInputStream.java | 10 +++++----- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 28d35ec6f45c..4a5820c681e6 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -363,7 +363,7 @@ private XceiverClientReply sendCommandWithRetry( // sendCommandAsyncCall will create a new channel and async stub // in case these don't exist for the specific datanode. reply.addDatanode(dn); - if (responseProto.getCmdType() == ContainerProtos.Type.ReadBlock) { + if (request.getCmdType() == ContainerProtos.Type.ReadBlock) { responseProto = sendCommandAsyncReadOnly(request, dn).getResponse().get(); } else { responseProto = sendCommandAsync(request, dn).getResponse().get(); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index 4a6cf3698ff5..bb7bdd6d58e4 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -119,7 +119,7 @@ public long getLength() { } @Override - public long getPos() { + public synchronized long getPos() { if (length == 0) { return 0; } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java index b93293c0e0f4..6cda1d33d683 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java @@ -19,12 +19,15 @@ package org.apache.hadoop.hdds.scm.storage; import java.io.EOFException; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import com.google.common.primitives.Bytes; -import com.sun.tools.javac.util.Log; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; @@ -38,13 +41,10 @@ import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; /** * Tests for {@link TestNewBlockInputStream}'s functionality. From a6636047279038775aa0e9527267816277d10b99 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 18:13:03 +0800 Subject: [PATCH 006/114] fix checkstyle --- .../hadoop/hdds/scm/storage/NewBlockInputStream.java | 2 +- .../hadoop/hdds/scm/storage/DummyNewBlockInputStream.java | 7 +++++-- .../hadoop/hdds/scm/storage/TestNewBlockInputStream.java | 4 ++-- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index bb7bdd6d58e4..3aa6c4cb479d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -454,7 +454,7 @@ private void adjustBufferPosition(long bufferPosition) { * @throws IOException if there is an I/O error while performing the call * to Datanode */ - private void readDataFromContainer(int len) throws IOException { + private synchronized void readDataFromContainer(int len) throws IOException { // index of first byte to be read from the chunk long startByteIndex; if (blockPosition >= 0) { diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java index 969a7936faf6..6665204c69f7 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java @@ -18,7 +18,10 @@ package org.apache.hadoop.hdds.scm.storage; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import java.util.function.Function; import org.apache.hadoop.hdds.client.BlockID; @@ -105,7 +108,7 @@ protected long readData(long offset, long len) { ChecksumData checksumData = ChecksumData.getFromProtoBuf( chunkInfo.getChecksumData()); int bytesPerChecksum = checksumData.getBytesPerChecksum(); - remainingToRead =( len / bytesPerChecksum + 1) * bytesPerChecksum; + remainingToRead = (len / bytesPerChecksum + 1) * bytesPerChecksum; } else { remainingToRead = chunkLen; } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java index 6cda1d33d683..772adcb798a9 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java @@ -62,7 +62,7 @@ public class TestNewBlockInputStream { private Checksum checksum; private Function refreshFunction; private BlockID blockID; - private static String CHUNK_NAME = "chunk-"; + private static final String CHUNK_NAME = "chunk-"; @BeforeEach @@ -184,7 +184,7 @@ public void testPartialChunkRead() throws Exception { @Test public void testSeek() throws Exception { seekAndVerify(0); - EOFException eofException = assertThrows(EOFException.class, () -> seekAndVerify( blockSize+ 1)); + EOFException eofException = assertThrows(EOFException.class, () -> seekAndVerify(blockSize + 1)); assertThat(eofException).hasMessage("EOF encountered at pos: " + (blockSize + 1) + " for block: " + blockID); // Seek before read should update the ChunkInputStream#chunkPosition From 0f35371754a51c06d541fce97421fde61fed9fe2 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 19:13:26 +0800 Subject: [PATCH 007/114] fix synchronized --- .../org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index 3aa6c4cb479d..0d8aa0a72922 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -237,7 +237,7 @@ public synchronized int read(byte[] b, int off, int len) throws IOException { } @Override - public void close() throws IOException { + public synchronized void close() throws IOException { releaseClient(); releaseBuffers(); xceiverClientFactory = null; From 80329dc4d83267e32b2b3d36aabf6b5e542f1509 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 19:38:26 +0800 Subject: [PATCH 008/114] fix synchronized --- .../org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index 0d8aa0a72922..896821835563 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -654,7 +654,7 @@ private void validateBlock( } @VisibleForTesting - protected void setBuffers(List buffers) { + protected synchronized void setBuffers(List buffers) { this.buffers = buffers; } From e398d17f84ef9ebd046ac3209d5e2c05f4605cf8 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 19:52:43 +0800 Subject: [PATCH 009/114] fix synchronized --- .../org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index 896821835563..7a5904d7002e 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -498,7 +498,7 @@ private synchronized void readDataFromContainer(int len) throws IOException { } @VisibleForTesting - protected long readData(long startByteIndex, long len) + protected synchronized long readData(long startByteIndex, long len) throws IOException { ReadBlockRequestProto.Builder readBlockRequest = ReadBlockRequestProto.newBuilder() From a40426636357f153a626811f1cdd2023bc5812c9 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 21:38:14 +0800 Subject: [PATCH 010/114] fix synchronized --- .../apache/hadoop/hdds/scm/storage/NewBlockInputStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index 7a5904d7002e..e42da1eb9d2d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -474,7 +474,6 @@ private synchronized void readDataFromContainer(int len) throws IOException { // Adjust the chunkInfo so that only the required bytes are read from // the chunk. - bufferOffsetWrtBlockDataData = readData(startByteIndex, len); long tempOffset = 0L; bufferoffsets = new ArrayList<>(buffers.size()); @@ -498,8 +497,9 @@ private synchronized void readDataFromContainer(int len) throws IOException { } @VisibleForTesting - protected synchronized long readData(long startByteIndex, long len) + protected long readData(long startByteIndex, long len) throws IOException { + buffers = new ArrayList<>(); ReadBlockRequestProto.Builder readBlockRequest = ReadBlockRequestProto.newBuilder() .setBlockID(blockID.getDatanodeBlockIDProtobuf()) From c57c2e6204bbc1e09bf7dc44fafdc866fa5c8653 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 21:56:27 +0800 Subject: [PATCH 011/114] fix synchronized --- .../apache/hadoop/hdds/scm/storage/NewBlockInputStream.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index e42da1eb9d2d..a79b97fb220e 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -510,11 +510,11 @@ protected long readData(long startByteIndex, long len) .newBuilder().setReadBlock(readBlockRequest).build(); ContainerProtos.ContainerCommandResponseProto response = xceiverClient.sendCommand(request, validators); - response.getStreamData().getReadBlockList().stream().map(readBlock -> { + response.getStreamData().getReadBlockList().forEach(readBlock -> { if (readBlock.hasData()) { - return this.buffers.add(readBlock.getData().asReadOnlyByteBuffer()); + buffers.add(readBlock.getData().asReadOnlyByteBuffer()); } else if (readBlock.hasDataBuffers()) { - return this.buffers.addAll(BufferUtils.getReadOnlyByteBuffers( + buffers.addAll(BufferUtils.getReadOnlyByteBuffers( readBlock.getDataBuffers().getBuffersList())); } else { throw new RuntimeException("Unexpected error while reading chunk data " + From 4dc908210e0d80973a0539485b146a67169564e5 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 22:15:16 +0800 Subject: [PATCH 012/114] fix synchronized --- .../hadoop/hdds/scm/storage/NewBlockInputStream.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index a79b97fb220e..c31e03a34641 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; @@ -510,17 +511,18 @@ protected long readData(long startByteIndex, long len) .newBuilder().setReadBlock(readBlockRequest).build(); ContainerProtos.ContainerCommandResponseProto response = xceiverClient.sendCommand(request, validators); - response.getStreamData().getReadBlockList().forEach(readBlock -> { + List readBlocks = response.getStreamData().getReadBlockList(); + for (ReadBlockResponseProto readBlock : readBlocks) { if (readBlock.hasData()) { buffers.add(readBlock.getData().asReadOnlyByteBuffer()); } else if (readBlock.hasDataBuffers()) { buffers.addAll(BufferUtils.getReadOnlyByteBuffers( readBlock.getDataBuffers().getBuffersList())); } else { - throw new RuntimeException("Unexpected error while reading chunk data " + + throw new IOException("Unexpected error while reading chunk data " + "from container. No data returned."); } - }); + } return response.getStreamData().getReadBlock(0) .getChunkData().getOffset(); } From af4a25b5a2d482e9122b46a653a94fbc20ef6499 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 18 Mar 2024 23:43:50 +0800 Subject: [PATCH 013/114] ignore find bugs in TestNewBlockInputStream --- hadoop-hdds/client/dev-support/findbugsExcludeFile.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml b/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml index 7b874905bf8a..a24100620acc 100644 --- a/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml @@ -32,4 +32,8 @@ + + + + From 79ae3eb4a87275ea8a65790c459a47133b706e84 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 19 Mar 2024 04:54:44 +0800 Subject: [PATCH 014/114] clean up --- .../hdds/scm/storage/NewBlockInputStream.java | 40 +++++------- .../scm/storage/TestNewBlockInputStream.java | 61 +++---------------- .../keyvalue/ReadBlockResponseStream.java | 2 + 3 files changed, 24 insertions(+), 79 deletions(-) create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ReadBlockResponseStream.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index c31e03a34641..f3a211c2cb36 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -135,7 +135,7 @@ public synchronized long getPos() { return length; } if (buffersHaveData()) { - // BufferOffset w.r.t to ChunkData + BufferOffset w.r.t buffers + + // BufferOffset w.r.t to BlockData + BufferOffset w.r.t buffers + // Position of current Buffer return bufferOffsetWrtBlockDataData + bufferoffsets.get(bufferIndex) + buffers.get(bufferIndex).position(); @@ -211,7 +211,7 @@ public synchronized int read(byte[] b, int off, int len) throws IOException { while (len > 0) { int available = prepareRead(len); if (available == EOF) { - // There is no more data in the chunk stream. The buffers should have + // There is no more data in the block stream. The buffers should have // been released by now Preconditions.checkState(buffers == null); return total != 0 ? total : EOF; @@ -260,7 +260,7 @@ public synchronized int read(ByteBuffer byteBuffer) throws IOException { while (len > 0) { int available = prepareRead(len); if (available == EOF) { - // There is no more data in the chunk stream. The buffers should have + // There is no more data in the block stream. The buffers should have // been released by now Preconditions.checkState(buffers == null); return total != 0 ? total : EOF; @@ -304,7 +304,7 @@ public synchronized void seek(long pos) throws IOException { } if (buffersHavePosition(pos)) { - // The bufferPosition is w.r.t the current chunk. + // The bufferPosition is w.r.t the current block. // Adjust the bufferIndex and position to the seeked position. adjustBufferPosition(pos - bufferOffsetWrtBlockDataData); } else { @@ -359,10 +359,10 @@ private synchronized int prepareRead(int len) throws IOException { if (blockPosition >= 0) { if (buffersHavePosition(blockPosition)) { // The current buffers have the seeked position. Adjust the buffer - // index and position to point to the chunkPosition. + // index and position to point to the buffer position. adjustBufferPosition(blockPosition - bufferOffsetWrtBlockDataData); } else { - // Read a required chunk data to fill the buffers with seeked + // Read a required block data to fill the buffers with seeked // position data readDataFromContainer(len); } @@ -372,11 +372,11 @@ private synchronized int prepareRead(int len) throws IOException { ByteBuffer bb = buffers.get(bufferIndex); return Math.min(len, bb.remaining()); } else if (dataRemainingInBlock()) { - // There is more data in the chunk stream which has not + // There is more data in the block stream which has not // been read into the buffers yet. readDataFromContainer(len); } else { - // All available input from this chunk stream has been consumed. + // All available input from this block stream has been consumed. return EOF; } } @@ -440,7 +440,7 @@ private void adjustBufferPosition(long bufferPosition) { buffers.get(i).position(0); } - // Reset the chunkPosition as chunk stream has been initialized i.e. the + // Reset the blockPosition as chunk stream has been initialized i.e. the // buffers have been allocated. blockPosition = -1; } @@ -456,14 +456,14 @@ private void adjustBufferPosition(long bufferPosition) { * to Datanode */ private synchronized void readDataFromContainer(int len) throws IOException { - // index of first byte to be read from the chunk + // index of first byte to be read from the block long startByteIndex; if (blockPosition >= 0) { // If seek operation was called to advance the buffer position, the // chunk should be read from that position onwards. startByteIndex = blockPosition; } else { - // Start reading the chunk from the last chunkPosition onwards. + // Start reading the block from the last blockPosition onwards. startByteIndex = bufferOffsetWrtBlockDataData + buffersSize; } @@ -472,9 +472,6 @@ private synchronized void readDataFromContainer(int len) throws IOException { // and is retried, we need the previous position. Position is reset after // successful read in adjustBufferPosition() blockPosition = getPos(); - - // Adjust the chunkInfo so that only the required bytes are read from - // the chunk. bufferOffsetWrtBlockDataData = readData(startByteIndex, len); long tempOffset = 0L; bufferoffsets = new ArrayList<>(buffers.size()); @@ -486,13 +483,6 @@ private synchronized void readDataFromContainer(int len) throws IOException { } bufferIndex = 0; allocated = true; - // If the stream was seeked to position before, then the buffer - // position should be adjusted as the reads happen at checksum boundaries. - // The buffers position might need to be adjusted for the following - // scenarios: - // 1. Stream was seeked to a position before the chunk was read - // 2. Chunk was read from index < the current position to account for - // checksum boundaries. adjustBufferPosition(startByteIndex - bufferOffsetWrtBlockDataData); } @@ -587,10 +577,10 @@ private boolean bufferEOF() { private void releaseBuffers(int releaseUptoBufferIndex) { int buffersLen = buffers.size(); if (releaseUptoBufferIndex == buffersLen - 1) { - // Before releasing all the buffers, if chunk EOF is not reached, then - // chunkPosition should be set to point to the last position of the + // Before releasing all the buffers, if block EOF is not reached, then + // blockPosition should be set to point to the last position of the // buffers. This should be done so that getPos() can return the current - // chunk position + // block position blockPosition = bufferOffsetWrtBlockDataData + bufferoffsets.get(releaseUptoBufferIndex) + buffers.get(releaseUptoBufferIndex).capacity(); @@ -610,7 +600,7 @@ private void releaseBuffers() { buffers = null; bufferIndex = 0; // We should not reset bufferOffsetWrtChunkData and buffersSize here - // because when getPos() is called in chunkStreamEOF() we use these + // because when getPos() is called we use these // values and determine whether chunk is read completely or not. } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java index 772adcb798a9..e14f3355ee0f 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java @@ -173,11 +173,11 @@ public void testPartialChunkRead() throws Exception { matchWithInputData(b, 0, len); - // To read chunk data from index 0 to 49 (len = 50), we need to read - // chunk from offset 0 to 60 as the checksum boundary is at every 20 + // To read block data from index 0 to 225 (len = 225), we need to read + // chunk from offset 0 to 240 as the checksum boundary is at every 20 // bytes. Verify that 60 bytes of chunk data are read and stored in the // buffers. Since checksum boundary is at every 20 bytes, there should be - // 60/20 number of buffers. + // 240/20 number of buffers. matchWithInputData(blockStream.getReadByteBuffers(), 0, 240); } @@ -187,11 +187,11 @@ public void testSeek() throws Exception { EOFException eofException = assertThrows(EOFException.class, () -> seekAndVerify(blockSize + 1)); assertThat(eofException).hasMessage("EOF encountered at pos: " + (blockSize + 1) + " for block: " + blockID); - // Seek before read should update the ChunkInputStream#chunkPosition + // Seek before read should update the BlockInputStream#blockPosition seekAndVerify(25); // Read from the sought position. - // Reading from index 25 to 54 should result in the ChunkInputStream + // Reading from index 25 to 54 should result in the BlockInputStream // copying chunk data from index 20 to 59 into the buffers (checksum // boundaries). byte[] b = new byte[30]; @@ -199,13 +199,13 @@ public void testSeek() throws Exception { matchWithInputData(b, 25, 30); matchWithInputData(blockStream.getReadByteBuffers(), 20, 40); - // After read, the position of the chunkStream is evaluated from the + // After read, the position of the blockStream is evaluated from the // buffers and the chunkPosition should be reset to -1. // Only the last BYTES_PER_CHECKSUM will be cached in the buffers as // buffers are released after each checksum boundary is read. So the // buffers should contain data from index 40 to 59. - // Seek to a position within the cached buffers. ChunkPosition should + // Seek to a position within the cached buffers. BlockPosition should // still not be used to set the position. seekAndVerify(45); @@ -251,51 +251,4 @@ public void testUnbuffered() throws Exception { matchWithInputData(b2, 20, 20); } -// @Test -// public void connectsToNewPipeline() throws Exception { -// // GIVEN -// Pipeline pipeline = MockPipeline.createSingleNodePipeline(); -// Pipeline newPipeline = MockPipeline.createSingleNodePipeline(); -// -// Token token = mock(Token.class); -// when(token.encodeToUrlString()) -// .thenReturn("oldToken"); -// Token newToken = mock(Token.class); -// when(newToken.encodeToUrlString()) -// .thenReturn("newToken"); -// -// AtomicReference pipelineRef = new AtomicReference<>(pipeline); -// AtomicReference> tokenRef = new AtomicReference<>(token); -// -// XceiverClientFactory clientFactory = mock(XceiverClientFactory.class); -// XceiverClientSpi client = mock(XceiverClientSpi.class); -// when(clientFactory.acquireClientForReadData(any())) -// .thenReturn(client); -// ArgumentCaptor requestCaptor = -// ArgumentCaptor.forClass(ContainerCommandRequestProto.class); -// when(client.getPipeline()) -// .thenAnswer(invocation -> pipelineRef.get()); -// when(client.sendCommand(requestCaptor.capture(), any())) -// .thenAnswer(invocation -> -// getReadChunkResponse( -// requestCaptor.getValue(), -// ChunkBuffer.wrap(ByteBuffer.wrap(blockData)), -// ByteStringConversion::safeWrap)); -// -// try (BlockInputStream subject = new ChunkInputStream(blockStream, blockID, -// clientFactory, pipelineRef::get, false, tokenRef::get)) { -// // WHEN -// subject.unbuffer(); -// pipelineRef.set(newPipeline); -// tokenRef.set(newToken); -// byte[] buffer = new byte[CHUNK_SIZE]; -// int read = subject.read(buffer); -// -// // THEN -// assertEquals(CHUNK_SIZE, read); -// assertArrayEquals(blockData, buffer); -// verify(clientFactory).acquireClientForReadData(newPipeline); -// verify(newToken).encodeToUrlString(); -// } -// } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ReadBlockResponseStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ReadBlockResponseStream.java new file mode 100644 index 000000000000..fcb9e04ede1f --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ReadBlockResponseStream.java @@ -0,0 +1,2 @@ +package org.apache.hadoop.ozone.container.keyvalue;public class ReadBlockResponseStream { +} From b1b301ea91323d8cf7c6a1adf7b098247566f416 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 9 Apr 2024 22:27:35 +0800 Subject: [PATCH 015/114] implement server side stream data --- .../hdds/scm/storage/NewBlockInputStream.java | 86 ++++++++++++------ .../io/BlockInputStreamFactoryImpl.java | 4 +- .../io/TestBlockInputStreamFactoryImpl.java | 3 +- .../container/common/impl/HddsDispatcher.java | 80 +++++++++++++++++ .../interfaces/ContainerDispatcher.java | 12 +++ .../container/common/interfaces/Handler.java | 7 ++ .../server/ratis/DispatcherContext.java | 8 ++ .../container/keyvalue/KeyValueHandler.java | 89 +++++++++++++++++++ .../keyvalue/TestKeyValueHandler.java | 1 + 9 files changed, 258 insertions(+), 32 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index f3a211c2cb36..f2e59867db50 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.fs.ByteBufferReadable; import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.Seekable; @@ -29,6 +30,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; @@ -49,7 +51,6 @@ import java.io.EOFException; import java.io.IOException; -import java.io.InputStream; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; @@ -64,7 +65,7 @@ * An {@link java.io.InputStream} called from KeyInputStream to read a block from the * container. */ -public class NewBlockInputStream extends InputStream +public class NewBlockInputStream extends BlockExtendedInputStream implements Seekable, CanUnbuffer, ByteBufferReadable { private static final Logger LOG = LoggerFactory.getLogger(NewBlockInputStream.class); @@ -292,6 +293,11 @@ public synchronized int read(ByteBuffer byteBuffer) throws IOException { } } + @Override + protected int readWithStrategy(ByteReaderStrategy strategy) throws IOException { + throw new NotImplementedException("readWithStrategy is not implemented."); + } + @Override public synchronized void seek(long pos) throws IOException { if (pos < 0 || pos > length) { @@ -613,35 +619,57 @@ protected synchronized void releaseClient() { private void validateBlock( ContainerProtos.ContainerCommandResponseProto response - ) throws OzoneChecksumException { + ) throws IOException { + + StreamDataResponseProto streamData = response.getStreamData(); + for (ReadBlockResponseProto readBlock : streamData.getReadBlockList()) { + List byteStrings; + boolean isV0 = false; + + ContainerProtos.ChunkInfo chunkInfo = + readBlock.getChunkData(); + if (chunkInfo.getLen() <= 0) { + throw new IOException("Failed to get chunk: chunkName == " + + chunkInfo.getChunkName() + "len == " + chunkInfo.getLen()); + } + if (readBlock.hasData()) { + ByteString byteString = readBlock.getData(); + if (byteString.size() != chunkInfo.getLen()) { + // Bytes read from chunk should be equal to chunk size. + throw new OzoneChecksumException(String.format( + "Inconsistent read for chunk=%s len=%d bytesRead=%d", + chunkInfo.getChunkName(), chunkInfo.getLen(), + byteString.size())); + } + byteStrings = new ArrayList<>(); + byteStrings.add(byteString); + isV0 = true; + } else { + byteStrings = readBlock.getDataBuffers().getBuffersList(); + long buffersLen = BufferUtils.getBuffersLen(byteStrings); + if (buffersLen != chunkInfo.getLen()) { + // Bytes read from chunk should be equal to chunk size. + throw new OzoneChecksumException(String.format( + "Inconsistent read for chunk=%s len=%d bytesRead=%d", + chunkInfo.getChunkName(), chunkInfo.getLen(), + buffersLen)); + } + } - ContainerProtos.ReadBlockResponseProto readBlock = response.getReadBlock(); - List byteStrings; - boolean isV0 = false; + if (verifyChecksum) { + ChecksumData checksumData = ChecksumData.getFromProtoBuf( + chunkInfo.getChecksumData()); - if (readBlock.hasData()) { - ByteString byteString = readBlock.getData(); - byteStrings = new ArrayList<>(); - byteStrings.add(byteString); - isV0 = true; - } else { - byteStrings = readBlock.getDataBuffers().getBuffersList(); - } - ContainerProtos.ChunkInfo chunkInfo = - readBlock.getChunkData(); - if (verifyChecksum) { - ChecksumData checksumData = ChecksumData.getFromProtoBuf( - chunkInfo.getChecksumData()); - - // ChecksumData stores checksum for each 'numBytesPerChecksum' - // number of bytes in a list. Compute the index of the first - // checksum to match with the read data - - long relativeOffset = chunkInfo.getOffset() - byteStrings.size(); - int bytesPerChecksum = checksumData.getBytesPerChecksum(); - int startIndex = (int) (relativeOffset / bytesPerChecksum); - Checksum.verifyChecksum(byteStrings, checksumData, startIndex, - isV0); + // ChecksumData stores checksum for each 'numBytesPerChecksum' + // number of bytes in a list. Compute the index of the first + // checksum to match with the read data + + long relativeOffset = chunkInfo.getOffset() - byteStrings.size(); + int bytesPerChecksum = checksumData.getBytesPerChecksum(); + int startIndex = (int) (relativeOffset / bytesPerChecksum); + Checksum.verifyChecksum(byteStrings, checksumData, startIndex, + isV0); + } } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 40063f9ce492..6045b19b3305 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -24,8 +24,8 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; -import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; +import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.io.ElasticByteBufferPool; @@ -84,7 +84,7 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, blockInfo, verifyChecksum, xceiverFactory, refreshFunction, ecBlockStreamFactory); } else { - return new BlockInputStream(blockInfo.getBlockID(), blockInfo.getLength(), + return new NewBlockInputStream(blockInfo.getBlockID(), blockInfo.getLength(), pipeline, token, verifyChecksum, xceiverFactory, refreshFunction); } } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java index cf3f4f13ef94..c895e421c330 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; +import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -55,7 +56,7 @@ public void testNonECGivesBlockInputStream() { BlockExtendedInputStream stream = factory.create(repConfig, blockInfo, blockInfo.getPipeline(), blockInfo.getToken(), true, null, null); - assertInstanceOf(BlockInputStream.class, stream); + assertInstanceOf(NewBlockInputStream.class, stream); assertEquals(stream.getBlockID(), blockInfo.getBlockID()); assertEquals(stream.getLength(), blockInfo.getLength()); } 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 63e51bdfd588..224c1945e784 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 @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.protobuf.ServiceException; +import io.opentracing.Span; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.client.BlockID; @@ -41,6 +42,7 @@ import org.apache.hadoop.hdds.security.token.NoopTokenVerifier; import org.apache.hadoop.hdds.security.token.TokenVerifier; import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher; +import org.apache.hadoop.hdds.tracing.TracingUtil; import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics; import org.apache.hadoop.ozone.audit.AuditAction; import org.apache.hadoop.ozone.audit.AuditEventStatus; @@ -60,12 +62,15 @@ import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume.VolumeSet; +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; import org.apache.hadoop.ozone.container.ozoneimpl.OnDemandContainerDataScanner; import org.apache.hadoop.ozone.container.common.volume.VolumeUsage; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Time; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.thirdparty.com.google.protobuf.ProtocolMessageEnum; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; +import org.apache.ratis.util.UncheckedAutoCloseable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -785,6 +790,81 @@ public StateMachine.DataChannel getStreamDataChannel( } } + @Override + public void streamDataReadOnly(ContainerCommandRequestProto msg, + StreamObserver streamObserver, + DispatcherContext dispatcherContext) { + Type cmdType = msg.getCmdType(); + String traceID = msg.getTraceID(); + Span span = TracingUtil.importAndCreateSpan(cmdType.toString(), traceID); + AuditAction action = getAuditAction(msg.getCmdType()); + EventType eventType = getEventType(msg); + Map params = getAuditParams(msg); + + try (UncheckedAutoCloseable ignored = protocolMetrics.measure(cmdType)) { + Preconditions.checkNotNull(msg); + if (LOG.isTraceEnabled()) { + LOG.trace("Command {}, trace ID: {}.", msg.getCmdType(), + traceID); + } + + PerformanceStringBuilder perf = new PerformanceStringBuilder(); + ContainerCommandResponseProto responseProto = null; + long containerID = msg.getContainerID(); + Container container = getContainer(containerID); + long startTime = Time.monotonicNow(); + + if (DispatcherContext.op(dispatcherContext).validateToken()) { + validateToken(msg); + } + if (getMissingContainerSet().contains(containerID)) { + throw new StorageContainerException( + "ContainerID " + containerID + + " has been lost and and cannot be recreated on this DataNode", + ContainerProtos.Result.CONTAINER_MISSING); + } + if (container == null) { + throw new StorageContainerException( + "ContainerID " + containerID + " does not exist", + ContainerProtos.Result.CONTAINER_NOT_FOUND); + } + ContainerType containerType = getContainerType(container); + Handler handler = getHandler(containerType); + if (handler == null) { + throw new StorageContainerException("Invalid " + + "ContainerType " + containerType, + ContainerProtos.Result.CONTAINER_INTERNAL_ERROR); + } + perf.appendPreOpLatencyMs(Time.monotonicNow() - startTime); + handler.streamDataReadOnly( + msg, (KeyValueContainer) container, dispatcherContext, streamObserver); + long oPLatencyMS = Time.monotonicNow() - startTime; + metrics.incContainerOpsLatencies(cmdType, oPLatencyMS); + Result result = responseProto.getResult(); + if (result == Result.SUCCESS) { + audit(action, eventType, params, AuditEventStatus.SUCCESS, null); + } else { + OnDemandContainerDataScanner.scanContainer(container); + audit(action, eventType, params, AuditEventStatus.FAILURE, + new Exception(responseProto.getMessage())); + } + perf.appendOpLatencyMs(oPLatencyMS); + performanceAudit(action, params, perf, oPLatencyMS); + + } catch (StorageContainerException sce) { + audit(action, eventType, params, AuditEventStatus.FAILURE, sce); + streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, sce, msg)); + } catch (IOException ioe) { + final String s = ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED + + " for " + dispatcherContext + ": " + ioe.getMessage(); + final StorageContainerException sce = new StorageContainerException( + s, ioe, ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED); + streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, sce, msg)); + } finally { + span.finish(); + } + } + private static DNAction getAuditAction(Type cmdType) { switch (cmdType) { case CreateContainer : return DNAction.CREATE_CONTAINER; 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 d02bae0a35ad..a7421d4c13bc 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 @@ -26,6 +26,7 @@ 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 org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import java.util.Map; @@ -94,4 +95,15 @@ default StateMachine.DataChannel getStreamDataChannel( throw new UnsupportedOperationException( "getStreamDataChannel not supported."); } + + /** + * When reading data form client using stream, get StreamDataChannel. + */ + default void streamDataReadOnly( + ContainerCommandRequestProto msg, + StreamObserver streamObserver, + DispatcherContext dispatcherContext) { + 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 2ffb9d30d1f4..44624db7a287 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 @@ -35,9 +35,11 @@ import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; 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; import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker; import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; @@ -217,4 +219,9 @@ public void setClusterID(String clusterID) { this.clusterId = clusterID; } + public abstract void streamDataReadOnly( + ContainerCommandRequestProto msg, KeyValueContainer container, + DispatcherContext dispatcherContext, + StreamObserver streamObserver); + } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java index d6c976cb389e..98e22f734c38 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java @@ -33,6 +33,9 @@ public final class DispatcherContext { private static final DispatcherContext HANDLE_READ_CHUNK = newBuilder(Op.HANDLE_READ_CHUNK).build(); + + private static final DispatcherContext HANDLE_READ_BLOCK + = newBuilder(Op.HANDLE_READ_BLOCK).build(); private static final DispatcherContext HANDLE_WRITE_CHUNK = newBuilder(Op.HANDLE_WRITE_CHUNK).build(); private static final DispatcherContext HANDLE_GET_SMALL_FILE @@ -44,6 +47,10 @@ public static DispatcherContext getHandleReadChunk() { return HANDLE_READ_CHUNK; } + public static DispatcherContext getHandleReadBlock() { + return HANDLE_READ_BLOCK; + } + public static DispatcherContext getHandleWriteChunk() { return HANDLE_WRITE_CHUNK; } @@ -76,6 +83,7 @@ public enum Op { NULL, HANDLE_READ_CHUNK, + HANDLE_READ_BLOCK, HANDLE_WRITE_CHUNK, HANDLE_GET_SMALL_FILE, HANDLE_PUT_SMALL_FILE, 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 c9d6672ee885..3959fa2e66cd 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 @@ -48,6 +48,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetSmallFileRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; import org.apache.hadoop.hdds.scm.ByteStringConversion; @@ -119,6 +120,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1210,6 +1212,93 @@ public void deleteUnreferenced(Container container, long localID) } } + @Override + public void streamDataReadOnly( + ContainerCommandRequestProto request, KeyValueContainer kvContainer, + DispatcherContext dispatcherContext, + StreamObserver streamObserver) { + try { + if (!request.hasReadBlock()) { + throw new Exception("MALFORMED_REQUEST"); + } + ReadBlockRequestProto readBlock = request.getReadBlock(); + ChunkBuffer data; + + BlockID blockID = BlockID.getFromProtobuf( + readBlock.getBlockID()); + BlockData blockData = blockManager.getBlock(kvContainer, blockID); + List chunkInfos = blockData.getChunks(); + long blockOffset = 0; + int chunkIndex = -1; + for (int i = 0; i < chunkInfos.size(); i++) { + blockOffset += chunkInfos.get(i).getLen(); + if (blockOffset > readBlock.getOffset()) { + chunkIndex = i; + break; + } + } + + BlockUtils.verifyBCSId(kvContainer, blockID); + if (dispatcherContext == null) { + dispatcherContext = DispatcherContext.getHandleReadBlock(); + } + + boolean isReadChunkV0 = readBlock.getVersion() + .equals(ContainerProtos.ReadChunkVersion.V0); + + int bytesPerChecksum = chunkInfos.get(chunkIndex) + .getChecksumData().getBytesPerChecksum(); + long offset = readBlock.getOffset(); + long len = readBlock.getLen(); + long adjustedChunkOffset = + (offset / bytesPerChecksum) * bytesPerChecksum; + long adjustedChunkLen = ((len + offset - 1) / bytesPerChecksum + 1) + * bytesPerChecksum; + len += (offset - adjustedChunkOffset); + while (len > 0) { + ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); + long chunkLen = Math.min(adjustedChunkLen, chunk.getLen()); + ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunkInfos + .get(chunkIndex) + .toBuilder() + .setOffset(adjustedChunkOffset) + .setLen(chunkLen) + .build()); + // For older clients, set ReadDataIntoSingleBuffer to true so that + // all the data read from chunk file is returned as a single + // ByteString. Older clients cannot process data returned as a list + // of ByteStrings. + if (isReadChunkV0) { + chunkInfo.setReadDataIntoSingleBuffer(isReadChunkV0); + } + data = chunkManager.readChunk( + kvContainer, blockID, chunkInfo, dispatcherContext); + + Preconditions.checkNotNull(data, "Chunk data is null"); + streamObserver.onNext( + getReadChunkResponse(request, data, byteBufferToByteString)); + len -= chunkLen; + adjustedChunkOffset = 0; + adjustedChunkLen = ((len - 1) / bytesPerChecksum + 1) + * bytesPerChecksum; + } + + metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen()); + } catch (StorageContainerException ex) { + streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, ex, request)); + } catch (IOException ioe) { + streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, + new StorageContainerException("Read Block failed", ioe, IO_EXCEPTION), + request)); + } catch (Exception ex) { + if (LOG.isDebugEnabled()) { + LOG.debug("Malformed Read Chunk request. trace ID: {}", + request.getTraceID()); + } + streamObserver.onNext(malformedRequest(request)); + } + } + private String[] getFilesWithPrefix(String prefix, File chunkDir) { FilenameFilter filter = (dir, name) -> name.startsWith(prefix); return chunkDir.list(filter); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index b9c8feae16ce..343a4f23efbb 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.security.token.TokenVerifier; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; From ad0fd8d7b8e093a7a98b848192942d82939b1a18 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 9 Apr 2024 23:23:17 +0800 Subject: [PATCH 016/114] fix bug --- .../ozone/client/io/TestBlockInputStreamFactoryImpl.java | 1 - .../ozone/container/common/impl/HddsDispatcher.java | 5 ++--- .../hadoop/ozone/container/common/interfaces/Handler.java | 2 +- .../hadoop/ozone/container/keyvalue/KeyValueHandler.java | 8 +++++++- .../ozone/container/keyvalue/ReadBlockResponseStream.java | 2 -- .../ozone/container/keyvalue/TestKeyValueHandler.java | 1 - 6 files changed, 10 insertions(+), 9 deletions(-) delete mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ReadBlockResponseStream.java diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java index c895e421c330..42499e3372a2 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; -import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.junit.jupiter.api.Test; 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 224c1945e784..4e99ace2faac 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 @@ -836,12 +836,11 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, ContainerProtos.Result.CONTAINER_INTERNAL_ERROR); } perf.appendPreOpLatencyMs(Time.monotonicNow() - startTime); - handler.streamDataReadOnly( + boolean success = handler.streamDataReadOnly( msg, (KeyValueContainer) container, dispatcherContext, streamObserver); long oPLatencyMS = Time.monotonicNow() - startTime; metrics.incContainerOpsLatencies(cmdType, oPLatencyMS); - Result result = responseProto.getResult(); - if (result == Result.SUCCESS) { + if (success) { audit(action, eventType, params, AuditEventStatus.SUCCESS, null); } else { OnDemandContainerDataScanner.scanContainer(container); 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 44624db7a287..cf572dfc1738 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 @@ -219,7 +219,7 @@ public void setClusterID(String clusterID) { this.clusterId = clusterID; } - public abstract void streamDataReadOnly( + public abstract boolean streamDataReadOnly( ContainerCommandRequestProto msg, KeyValueContainer container, DispatcherContext dispatcherContext, StreamObserver streamObserver); 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 3959fa2e66cd..736bb9e38001 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 @@ -1213,10 +1213,11 @@ public void deleteUnreferenced(Container container, long localID) } @Override - public void streamDataReadOnly( + public boolean streamDataReadOnly( ContainerCommandRequestProto request, KeyValueContainer kvContainer, DispatcherContext dispatcherContext, StreamObserver streamObserver) { + boolean result = true; try { if (!request.hasReadBlock()) { throw new Exception("MALFORMED_REQUEST"); @@ -1286,16 +1287,21 @@ public void streamDataReadOnly( metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen()); } catch (StorageContainerException ex) { streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, ex, request)); + result = false; } catch (IOException ioe) { streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, new StorageContainerException("Read Block failed", ioe, IO_EXCEPTION), request)); + result = false; } catch (Exception ex) { if (LOG.isDebugEnabled()) { LOG.debug("Malformed Read Chunk request. trace ID: {}", request.getTraceID()); } streamObserver.onNext(malformedRequest(request)); + result = false; + } finally { + return result; } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ReadBlockResponseStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ReadBlockResponseStream.java deleted file mode 100644 index fcb9e04ede1f..000000000000 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ReadBlockResponseStream.java +++ /dev/null @@ -1,2 +0,0 @@ -package org.apache.hadoop.ozone.container.keyvalue;public class ReadBlockResponseStream { -} diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index 343a4f23efbb..b9c8feae16ce 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.security.token.TokenVerifier; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; From e86aee9357e015de09f975f81fb2c54fbd9f1c47 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 10 Apr 2024 00:05:10 +0800 Subject: [PATCH 017/114] fix bug --- .../container/common/impl/HddsDispatcher.java | 4 +-- .../container/common/interfaces/Handler.java | 2 +- .../container/keyvalue/KeyValueHandler.java | 28 ++++++++----------- 3 files changed, 14 insertions(+), 20 deletions(-) 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 4e99ace2faac..b60bde9f0871 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 @@ -836,11 +836,11 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, ContainerProtos.Result.CONTAINER_INTERNAL_ERROR); } perf.appendPreOpLatencyMs(Time.monotonicNow() - startTime); - boolean success = handler.streamDataReadOnly( + responseProto = handler.streamDataReadOnly( msg, (KeyValueContainer) container, dispatcherContext, streamObserver); long oPLatencyMS = Time.monotonicNow() - startTime; metrics.incContainerOpsLatencies(cmdType, oPLatencyMS); - if (success) { + if (responseProto == null) { audit(action, eventType, params, AuditEventStatus.SUCCESS, null); } else { OnDemandContainerDataScanner.scanContainer(container); 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 cf572dfc1738..64fe01f4c975 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 @@ -219,7 +219,7 @@ public void setClusterID(String clusterID) { this.clusterId = clusterID; } - public abstract boolean streamDataReadOnly( + public abstract ContainerCommandResponseProto streamDataReadOnly( ContainerCommandRequestProto msg, KeyValueContainer container, DispatcherContext dispatcherContext, StreamObserver streamObserver); 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 736bb9e38001..022a0cade274 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 @@ -1213,14 +1213,18 @@ public void deleteUnreferenced(Container container, long localID) } @Override - public boolean streamDataReadOnly( + public ContainerCommandResponseProto streamDataReadOnly( ContainerCommandRequestProto request, KeyValueContainer kvContainer, DispatcherContext dispatcherContext, StreamObserver streamObserver) { - boolean result = true; + ContainerCommandResponseProto responseProto = null; try { if (!request.hasReadBlock()) { - throw new Exception("MALFORMED_REQUEST"); + if (LOG.isDebugEnabled()) { + LOG.debug("Malformed Read Block request. trace ID: {}", + request.getTraceID()); + } + return malformedRequest(request); } ReadBlockRequestProto readBlock = request.getReadBlock(); ChunkBuffer data; @@ -1286,23 +1290,13 @@ public boolean streamDataReadOnly( metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen()); } catch (StorageContainerException ex) { - streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, ex, request)); - result = false; + responseProto = ContainerUtils.logAndReturnError(LOG, ex, request); } catch (IOException ioe) { - streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, + responseProto = ContainerUtils.logAndReturnError(LOG, new StorageContainerException("Read Block failed", ioe, IO_EXCEPTION), - request)); - result = false; - } catch (Exception ex) { - if (LOG.isDebugEnabled()) { - LOG.debug("Malformed Read Chunk request. trace ID: {}", - request.getTraceID()); - } - streamObserver.onNext(malformedRequest(request)); - result = false; - } finally { - return result; + request); } + return responseProto; } private String[] getFilesWithPrefix(String prefix, File chunkDir) { From 8d97d472618faf0c71988474c865aae7ef000041 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Sat, 13 Apr 2024 00:12:33 +0800 Subject: [PATCH 018/114] fix bug --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 20 +- .../hdds/scm/storage/NewBlockInputStream.java | 192 +++++++++++------- .../io/BlockInputStreamFactoryImpl.java | 1 + .../scm/storage/DummyNewBlockInputStream.java | 7 +- .../scm/storage/TestNewBlockInputStream.java | 6 +- .../ContainerCommandResponseBuilders.java | 31 +++ .../scm/storage/ContainerProtocolCalls.java | 48 +++++ .../hdds/scm/utils/ClientCommandsUtils.java | 10 + .../apache/hadoop/ozone/audit/DNAction.java | 3 +- .../container/common/impl/HddsDispatcher.java | 2 + .../transport/server/GrpcXceiverService.java | 13 +- .../container/keyvalue/KeyValueHandler.java | 9 +- .../main/proto/DatanodeClientProtocol.proto | 1 + .../ozone/client/MockXceiverClientSpi.java | 24 +++ 14 files changed, 277 insertions(+), 90 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 7531618a31c1..f5203f776510 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -42,6 +42,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc.XceiverClientProtocolServiceStub; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -589,13 +591,25 @@ public void onError(Throwable t) { @Override public void onCompleted() { - semaphore.release(); - future.complete(response.setStreamData(streamData).build()); + if (streamData.getReadBlockCount() > 0) { + future.complete(response.setStreamData(streamData) + .setCmdType(Type.StreamRead).setResult(Result.SUCCESS).build()); + } + if (!future.isDone()) { + future.completeExceptionally(new IOException( + "Stream completed but no reply for request " + + processForDebug(request))); + } } }); requestObserver.onNext(request); requestObserver.onCompleted(); - return new XceiverClientReply(future); + try { + return new XceiverClientReply(future); + } finally { + semaphore.release(); + } + } private synchronized void checkOpen(DatanodeDetails dn) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index af11f20f07f4..2f79b84e51e7 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -27,8 +27,7 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -47,6 +46,7 @@ import org.apache.hadoop.ozone.common.utils.BufferUtils; import org.apache.hadoop.security.token.Token; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.Status; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -152,38 +152,44 @@ public synchronized long getPos() { @Override public synchronized int read() throws IOException { checkOpen(); - while (true) { - int dataout = EOF; - try { - + int dataout = EOF; + int len = 1; + int available; + while (len > 0) { + try { acquireClient(); - int available = prepareRead(1); - - - if (available == EOF) { - // There is no more data in the chunk stream. The buffers should have - // been released by now - Preconditions.checkState(buffers == null); - } else { - dataout = Byte.toUnsignedInt(buffers.get(bufferIndex).get()); - } - - if (bufferEOF()) { - releaseBuffers(bufferIndex); - } + available = prepareRead(1); + retries = 0; } catch (SCMSecurityException ex) { throw ex; } catch (StorageContainerException e) { if (shouldRetryRead(e)) { releaseClient(); + refreshBlockInfo(e); + continue; } else { throw e; } } - return dataout; + if (available == EOF) { + // There is no more data in the chunk stream. The buffers should have + // been released by now + Preconditions.checkState(buffers == null); + } else { + dataout = Byte.toUnsignedInt(buffers.get(bufferIndex).get()); + } + + len -= available; + if (bufferEOF()) { + releaseBuffers(bufferIndex); + } } + + return dataout; + + } @Override @@ -208,36 +214,39 @@ public synchronized int read(byte[] b, int off, int len) throws IOException { return 0; } int total = 0; - while (true) { + int available; + while (len > 0) { try { acquireClient(); - while (len > 0) { - int available = prepareRead(len); - if (available == EOF) { - // There is no more data in the block stream. The buffers should have - // been released by now - Preconditions.checkState(buffers == null); - return total != 0 ? total : EOF; - } - buffers.get(bufferIndex).get(b, off + total, available); - len -= available; - total += available; - - if (bufferEOF()) { - releaseBuffers(bufferIndex); - } - } + available = prepareRead(len); + retries = 0; } catch (SCMSecurityException ex) { throw ex; } catch (StorageContainerException e) { if (shouldRetryRead(e)) { releaseClient(); + refreshBlockInfo(e); + continue; } else { throw e; } } - return total; + if (available == EOF) { + // There is no more data in the block stream. The buffers should have + // been released by now + Preconditions.checkState(buffers == null); + return total != 0 ? total : EOF; + } + buffers.get(bufferIndex).get(b, off + total, available); + len -= available; + total += available; + + if (bufferEOF()) { + releaseBuffers(bufferIndex); + } } + return total; + } @Override @@ -257,42 +266,43 @@ public synchronized int read(ByteBuffer byteBuffer) throws IOException { return 0; } int total = 0; - while (true) { + int available; + while (len > 0) { try { acquireClient(); - while (len > 0) { - int available = prepareRead(len); - if (available == EOF) { - // There is no more data in the block stream. The buffers should have - // been released by now - Preconditions.checkState(buffers == null); - return total != 0 ? total : EOF; - } - ByteBuffer readBuf = buffers.get(bufferIndex); - ByteBuffer tmpBuf = readBuf.duplicate(); - tmpBuf.limit(tmpBuf.position() + available); - byteBuffer.put(tmpBuf); - readBuf.position(tmpBuf.position()); - - len -= available; - total += available; - - if (bufferEOF()) { - releaseBuffers(bufferIndex); - } - } + available = prepareRead(len); + retries = 0; } catch (SCMSecurityException ex) { throw ex; } catch (StorageContainerException e) { if (shouldRetryRead(e)) { releaseClient(); + refreshBlockInfo(e); + continue; } else { throw e; } } - return total; + if (available == EOF) { + // There is no more data in the block stream. The buffers should have + // been released by now + Preconditions.checkState(buffers == null); + return total != 0 ? total : EOF; + } + ByteBuffer readBuf = buffers.get(bufferIndex); + ByteBuffer tmpBuf = readBuf.duplicate(); + tmpBuf.limit(tmpBuf.position() + available); + byteBuffer.put(tmpBuf); + readBuf.position(tmpBuf.position()); + + len -= available; + total += available; + if (bufferEOF()) { + releaseBuffers(bufferIndex); + } } + return total; } @Override @@ -498,18 +508,21 @@ private synchronized void readDataFromContainer(int len) throws IOException { @VisibleForTesting protected long readData(long startByteIndex, long len) throws IOException { + Pipeline pipeline = xceiverClient.getPipeline(); buffers = new ArrayList<>(); - ReadBlockRequestProto.Builder readBlockRequest = - ReadBlockRequestProto.newBuilder() - .setBlockID(blockID.getDatanodeBlockIDProtobuf()) - .setLen(startByteIndex) - .setOffset(len) - .setVersion(ContainerProtos.ReadChunkVersion.V1); - ContainerCommandRequestProto request = ContainerCommandRequestProto - .newBuilder().setReadBlock(readBlockRequest).build(); - ContainerProtos.ContainerCommandResponseProto response = - xceiverClient.sendCommand(request, validators); - List readBlocks = response.getStreamData().getReadBlockList(); + DatanodeBlockID.Builder blockBuilder = DatanodeBlockID + .newBuilder().setContainerID(blockID.getContainerID()) + .setLocalID(blockID.getLocalID()) + .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId()); + int replicaIndex = pipeline.getReplicaIndex(pipeline.getClosestNode()); + if (replicaIndex > 0) { + blockBuilder.setReplicaIndex(replicaIndex); + } + StreamDataResponseProto response = + ContainerProtocolCalls.readBlock(xceiverClient, startByteIndex, + len, blockBuilder.build(), validators, tokenRef.get()); + List readBlocks = response.getReadBlockList(); + for (ReadBlockResponseProto readBlock : readBlocks) { if (readBlock.hasData()) { buffers.add(readBlock.getData().asReadOnlyByteBuffer()); @@ -521,7 +534,7 @@ protected long readData(long startByteIndex, long len) "from container. No data returned."); } } - return response.getStreamData().getReadBlock(0) + return response.getReadBlock(0) .getChunkData().getOffset(); } @@ -658,7 +671,7 @@ private void validateBlock( } } - if (verifyChecksum) { + if (config.isChecksumVerify()) { ChecksumData checksumData = ChecksumData.getFromProtoBuf( chunkInfo.getChecksumData()); @@ -694,6 +707,33 @@ private boolean shouldRetryRead(IOException cause) throws IOException { @VisibleForTesting public boolean isVerifyChecksum() { - return verifyChecksum; + return config.isChecksumVerify(); } + + private void refreshBlockInfo(IOException cause) throws IOException { + LOG.info("Unable to read information for block {} from pipeline {}: {}", + blockID, pipelineRef.get().getId(), cause.getMessage()); + if (refreshFunction != null) { + LOG.debug("Re-fetching pipeline and block token for block {}", blockID); + BlockLocationInfo blockLocationInfo = refreshFunction.apply(blockID); + if (blockLocationInfo == null) { + LOG.debug("No new block location info for block {}", blockID); + } else { + LOG.debug("New pipeline for block {}: {}", blockID, + blockLocationInfo.getPipeline()); + setPipeline(blockLocationInfo.getPipeline()); + tokenRef.set(blockLocationInfo.getToken()); + } + } else { + throw cause; + } + } + + /** + * Check if this exception is because datanodes are not reachable. + */ + private boolean isConnectivityIssue(IOException ex) { + return Status.fromThrowable(ex).getCode() == Status.UNAVAILABLE.getCode(); + } + } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index cfeab5a9c697..35fb797b83b3 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java index 6665204c69f7..22399b555f33 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +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.security.token.OzoneBlockTokenIdentifier; @@ -50,13 +51,13 @@ class DummyNewBlockInputStream extends NewBlockInputStream { long blockLen, Pipeline pipeline, Token token, - boolean verifyChecksum, XceiverClientFactory xceiverClientManager, Function refreshFunction, + OzoneClientConfig config, List chunks, Map chunkDataMap) { - super(blockId, blockLen, pipeline, token, verifyChecksum, - xceiverClientManager, refreshFunction); + super(blockId, blockLen, pipeline, token, xceiverClientManager, + refreshFunction, config); this.chunks = chunks; this.chunkDataMap = chunkDataMap; chunkOffsets = new long[chunks.size()]; diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java index e14f3355ee0f..07c8b19cd4ee 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java @@ -30,8 +30,10 @@ import com.google.common.primitives.Bytes; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ContainerBlockID; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.ozone.common.Checksum; @@ -63,10 +65,12 @@ public class TestNewBlockInputStream { private Function refreshFunction; private BlockID blockID; private static final String CHUNK_NAME = "chunk-"; + private OzoneConfiguration conf = new OzoneConfiguration(); @BeforeEach public void setup() throws Exception { + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); refreshFunction = mock(Function.class); blockID = new BlockID(new ContainerBlockID(1, 1)); checksum = new Checksum(ChecksumType.CRC32, BYTES_PER_CHECKSUM); @@ -74,7 +78,7 @@ public void setup() throws Exception { Pipeline pipeline = MockPipeline.createSingleNodePipeline(); blockStream = new DummyNewBlockInputStream(blockID, blockSize, pipeline, - null, true, null, refreshFunction, chunks, chunkDataMap); + null, null, refreshFunction, clientConfig, chunks, chunkDataMap); } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index 73369fc9a38c..4cf81c746524 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -21,6 +21,8 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.function.Function; + +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; @@ -35,6 +37,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetSmallFileResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ListBlockResponseProto; @@ -307,6 +310,34 @@ public static ContainerCommandResponseProto getReadChunkResponse( .build(); } + public static ContainerCommandResponseProto getReadBlockResponse( + ContainerCommandRequestProto request, DatanodeBlockID blockID, + ChunkInfo chunkInfo, boolean isReadChunkV0, ChunkBuffer data, + Function byteBufferToByteString) { + + ReadBlockResponseProto.Builder response; + + if (isReadChunkV0) { + // V0 has all response data in a single ByteBuffer + response = ReadBlockResponseProto.newBuilder() + .setChunkData(chunkInfo) + .setData(data.toByteString(byteBufferToByteString)) + .setBlockID(blockID); + } else { + // V1 splits response data into a list of ByteBuffers + response = ReadBlockResponseProto.newBuilder() + .setChunkData(chunkInfo) + .setDataBuffers(DataBuffers.newBuilder() + .addAllBuffers(data.toByteStringList(byteBufferToByteString)) + .build()) + .setBlockID(blockID); + } + + return getSuccessResponseBuilder(request) + .setReadBlock(response) + .build(); + } + private ContainerCommandResponseBuilders() { throw new UnsupportedOperationException("no instances"); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 03b7844cc941..ee535cee8a34 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -50,10 +50,12 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; import org.apache.hadoop.hdds.scm.XceiverClientReply; @@ -729,4 +731,50 @@ public static List toValidatorList(Validator validator) { return datanodeToResponseMap; } + /** + * Calls the container protocol to read a chunk. + * + * @param xceiverClient client to perform call + * @param chunk information about chunk to read + * @param blockID ID of the block + * @param validators functions to validate the response + * @param token a token for this block (may be null) + * @return container protocol read chunk response + * @throws IOException if there is an I/O error while performing the call + */ + public static ContainerProtos.StreamDataResponseProto readBlock( + XceiverClientSpi xceiverClient, long offset, long len, DatanodeBlockID blockID, + List validators, + Token token) throws IOException { + ReadBlockRequestProto.Builder readBlockRequest = + ReadBlockRequestProto.newBuilder() + .setBlockID(blockID) + .setOffset(offset) + .setLen(len) + .setVersion(ContainerProtos.ReadChunkVersion.V1); + ContainerCommandRequestProto.Builder builder = + ContainerCommandRequestProto.newBuilder().setCmdType(Type.ReadBlock) + .setContainerID(blockID.getContainerID()) + .setReadBlock(readBlockRequest); + if (token != null) { + builder.setEncodedToken(token.encodeToUrlString()); + } + + + return tryEachDatanode(xceiverClient.getPipeline(), + d -> readBlock(xceiverClient, + validators, builder, d), + d -> toErrorMessage(blockID, d)); + } + + private static StreamDataResponseProto readBlock(XceiverClientSpi xceiverClient, + List validators, + ContainerCommandRequestProto.Builder builder, + DatanodeDetails datanode) throws IOException { + final ContainerCommandRequestProto request = builder + .setDatanodeUuid(datanode.getUuidString()).build(); + ContainerCommandResponseProto response = + xceiverClient.sendCommand(request, validators); + return response.getStreamData(); + } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java index c4202c9cc6aa..97911526210b 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.utils; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; /** * These methods should be merged with other similar utility classes. @@ -47,4 +48,13 @@ public static ContainerProtos.ReadChunkVersion getReadChunkVersion( return ContainerProtos.ReadChunkVersion.V0; } } + + public static ContainerProtos.ReadChunkVersion getReadChunkVersion( + ReadBlockRequestProto readBlock) { + if (readBlock.hasVersion()) { + return readBlock.getVersion(); + } else { + return ContainerProtos.ReadChunkVersion.V0; + } + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java index 73aff9ac830c..2528c76628d7 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/audit/DNAction.java @@ -39,7 +39,8 @@ public enum DNAction implements AuditAction { GET_SMALL_FILE, CLOSE_CONTAINER, GET_COMMITTED_BLOCK_LENGTH, - STREAM_INIT; + STREAM_INIT, + READ_BLOCK; @Override public String getAction() { 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 c1f6b060fa0b..7f67842c1231 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 @@ -855,6 +855,7 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, OnDemandContainerDataScanner.scanContainer(container); audit(action, eventType, params, AuditEventStatus.FAILURE, new Exception(responseProto.getMessage())); + streamObserver.onNext(responseProto); } perf.appendOpLatencyMs(oPLatencyMS); performanceAudit(action, params, perf, oPLatencyMS); @@ -894,6 +895,7 @@ private static DNAction getAuditAction(Type cmdType) { case CloseContainer : return DNAction.CLOSE_CONTAINER; case GetCommittedBlockLength : return DNAction.GET_COMMITTED_BLOCK_LENGTH; case StreamInit : return DNAction.STREAM_INIT; + case ReadBlock : return DNAction.READ_BLOCK; default : LOG.debug("Invalid command type - {}", cmdType); return null; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java index 9c3f29d0f0cc..30dd6fd309fc 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.transport.server; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; @@ -108,9 +109,13 @@ public StreamObserver send( @Override public void onNext(ContainerCommandRequestProto request) { try { - ContainerCommandResponseProto resp = - dispatcher.dispatch(request, null); - responseObserver.onNext(resp); + if (request.getCmdType() == Type.ReadBlock) { + dispatcher.streamDataReadOnly(request, responseObserver, null); + } else { + ContainerCommandResponseProto resp = + dispatcher.dispatch(request, null); + responseObserver.onNext(resp); + } } catch (Throwable e) { LOG.error("Got exception when processing" + " ContainerCommandRequestProto {}", request, e); @@ -133,7 +138,7 @@ public void onError(Throwable t) { @Override public void onCompleted() { if (isClosed.compareAndSet(false, true)) { - LOG.debug("ContainerCommand send completed"); + LOG.info("ContainerCommand send completed"); responseObserver.onCompleted(); } } 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 994d8371db05..fe9c6c8aa10d 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 @@ -106,6 +106,7 @@ import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getGetSmallFileResponseSuccess; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getListBlockResponse; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getPutFileResponseSuccess; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getReadBlockResponse; 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; @@ -1283,11 +1284,15 @@ public ContainerCommandResponseProto streamDataReadOnly( Preconditions.checkNotNull(data, "Chunk data is null"); streamObserver.onNext( - getReadChunkResponse(request, data, byteBufferToByteString)); + getReadBlockResponse(request, + blockData.getProtoBufMessage().getBlockID(), + chunkInfo.getProtoBufMessage(), isReadChunkV0, + data, byteBufferToByteString)); len -= chunkLen; - adjustedChunkOffset = 0; + adjustedChunkOffset += chunkLen; adjustedChunkLen = ((len - 1) / bytesPerChecksum + 1) * bytesPerChecksum; + chunkIndex++; } metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen()); diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 117aeaa8d79e..1cbe3a253b38 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -104,6 +104,7 @@ enum Type { StreamInit = 19; StreamWrite = 20; + StreamRead = 22; } diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java index 59eb49e5557c..1d6b58553156 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java @@ -20,6 +20,7 @@ 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.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; @@ -29,9 +30,12 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetCommittedBlockLengthResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; @@ -41,6 +45,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.function.Function; @@ -101,6 +106,9 @@ public XceiverClientReply sendCommandAsync( case ListBlock: return result(request, r -> r.setListBlock(listBlock(request.getContainerID()))); + case ReadBlock: + return result(request, + r -> r.setStreamData(readBlock(request.getReadBlock()))); default: throw new IllegalArgumentException( "Mock version of datanode call " + request.getCmdType() @@ -129,6 +137,22 @@ private ContainerProtos.ListBlockResponseProto listBlock(long containerID) { .addAllBlockData(datanodeStorage.listBlock(containerID)).build(); } + private ContainerProtos.StreamDataResponseProto readBlock( + ReadBlockRequestProto readBlock) { + BlockData blockData = datanodeStorage.getBlock(readBlock.getBlockID()); + List chunkInfos = blockData.getChunksList(); + StreamDataResponseProto.Builder builder = StreamDataResponseProto.newBuilder(); + for (ChunkInfo chunkInfo : chunkInfos) { + builder.addReadBlock(ReadBlockResponseProto.newBuilder() + .setChunkData(datanodeStorage + .readChunkInfo(blockData.getBlockID(), chunkInfo)) + .setData(datanodeStorage + .readChunkData(blockData.getBlockID(), chunkInfo)) + .setBlockID(blockData.getBlockID()).build()); + } + return builder.build(); + } + private PutBlockResponseProto putBlock(PutBlockRequestProto putBlock) { long length = 0; for (ChunkInfo chunk : putBlock.getBlockData().getChunksList()) { From b694448919f94d1c854f1d76eacf808c6d898fe0 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Sat, 13 Apr 2024 01:14:16 +0800 Subject: [PATCH 019/114] fix bug --- .../hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java | 1 - .../hdds/scm/protocolPB/ContainerCommandResponseBuilders.java | 1 - 2 files changed, 2 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 35fb797b83b3..cfeab5a9c697 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; -import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index 4cf81c746524..79a667f12383 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -22,7 +22,6 @@ import java.util.List; import java.util.function.Function; -import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; From d5dc9084e04840573c6faefc314a485cb14b65b8 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Sat, 13 Apr 2024 04:07:36 +0800 Subject: [PATCH 020/114] fix bug --- .../hdds/scm/storage/NewBlockInputStream.java | 13 +++++-------- .../ContainerCommandResponseBuilders.java | 9 ++++++--- .../ozone/container/keyvalue/KeyValueHandler.java | 3 ++- .../src/main/proto/DatanodeClientProtocol.proto | 5 +++-- 4 files changed, 16 insertions(+), 14 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index 2f79b84e51e7..62ef8e9a0191 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -88,7 +88,7 @@ public class NewBlockInputStream extends BlockExtendedInputStream private long buffersSize; private static final int EOF = -1; private final List validators; - private final OzoneClientConfig config; + private final boolean verifyChecksum; private final Function refreshFunction; private final RetryPolicy retryPolicy = HddsClientUtils.createRetryPolicy(3, TimeUnit.SECONDS.toMillis(1)); @@ -108,7 +108,7 @@ public NewBlockInputStream( this.xceiverClientFactory = xceiverClientFactory; this.validators = ContainerProtocolCalls.toValidatorList( (request, response) -> validateBlock(response)); - this.config = config; + this.verifyChecksum = config.isChecksumVerify(); this.refreshFunction = refreshFunction; } @@ -671,7 +671,7 @@ private void validateBlock( } } - if (config.isChecksumVerify()) { + if (verifyChecksum) { ChecksumData checksumData = ChecksumData.getFromProtoBuf( chunkInfo.getChecksumData()); @@ -679,10 +679,7 @@ private void validateBlock( // number of bytes in a list. Compute the index of the first // checksum to match with the read data - long relativeOffset = chunkInfo.getOffset() - byteStrings.size(); - int bytesPerChecksum = checksumData.getBytesPerChecksum(); - int startIndex = (int) (relativeOffset / bytesPerChecksum); - Checksum.verifyChecksum(byteStrings, checksumData, startIndex, + Checksum.verifyChecksum(byteStrings, checksumData, readBlock.getStartIndex(), isV0); } } @@ -707,7 +704,7 @@ private boolean shouldRetryRead(IOException cause) throws IOException { @VisibleForTesting public boolean isVerifyChecksum() { - return config.isChecksumVerify(); + return verifyChecksum; } private void refreshBlockInfo(IOException cause) throws IOException { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index 79a667f12383..535e75e0ff44 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -312,7 +312,8 @@ public static ContainerCommandResponseProto getReadChunkResponse( public static ContainerCommandResponseProto getReadBlockResponse( ContainerCommandRequestProto request, DatanodeBlockID blockID, ChunkInfo chunkInfo, boolean isReadChunkV0, ChunkBuffer data, - Function byteBufferToByteString) { + Function byteBufferToByteString, + int startIndex) { ReadBlockResponseProto.Builder response; @@ -321,7 +322,8 @@ public static ContainerCommandResponseProto getReadBlockResponse( response = ReadBlockResponseProto.newBuilder() .setChunkData(chunkInfo) .setData(data.toByteString(byteBufferToByteString)) - .setBlockID(blockID); + .setBlockID(blockID) + .setStartIndex(startIndex); } else { // V1 splits response data into a list of ByteBuffers response = ReadBlockResponseProto.newBuilder() @@ -329,7 +331,8 @@ public static ContainerCommandResponseProto getReadBlockResponse( .setDataBuffers(DataBuffers.newBuilder() .addAllBuffers(data.toByteStringList(byteBufferToByteString)) .build()) - .setBlockID(blockID); + .setBlockID(blockID) + .setStartIndex(startIndex); } return getSuccessResponseBuilder(request) 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 fe9c6c8aa10d..7846c39e69c0 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 @@ -1265,6 +1265,7 @@ public ContainerCommandResponseProto streamDataReadOnly( len += (offset - adjustedChunkOffset); while (len > 0) { ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); + int startIndex = (int)(adjustedChunkOffset - chunk.getOffset()) / bytesPerChecksum; long chunkLen = Math.min(adjustedChunkLen, chunk.getLen()); ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunkInfos .get(chunkIndex) @@ -1287,7 +1288,7 @@ public ContainerCommandResponseProto streamDataReadOnly( getReadBlockResponse(request, blockData.getProtoBufMessage().getBlockID(), chunkInfo.getProtoBufMessage(), isReadChunkV0, - data, byteBufferToByteString)); + data, byteBufferToByteString, startIndex)); len -= chunkLen; adjustedChunkOffset += chunkLen; adjustedChunkLen = ((len - 1) / bytesPerChecksum + 1) diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 1cbe3a253b38..155d4f4b36ed 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -388,10 +388,11 @@ message ReadBlockRequestProto { message ReadBlockResponseProto { required DatanodeBlockID blockID = 1; required ChunkInfo chunkData = 2; + required uint32 startIndex = 3; // Chunk data should be returned in one of the two for oneof responseData { - bytes data = 3; // Chunk data is returned as single buffer for V0 - DataBuffers dataBuffers = 4; // Chunk data is returned as a list of buffers + bytes data = 4; // Chunk data is returned as single buffer for V0 + DataBuffers dataBuffers = 5; // Chunk data is returned as a list of buffers } } From 74eac1e241ec4cb830b26a0b4a2a8c5ae3312aa0 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 15 Apr 2024 17:57:22 +0800 Subject: [PATCH 021/114] fix bug --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 7 ++++++ .../hdds/scm/storage/NewBlockInputStream.java | 6 +++-- .../io/BlockInputStreamFactoryImpl.java | 1 + .../ContainerCommandResponseBuilders.java | 10 ++++---- .../transport/server/ratis/CSMMetrics.java | 1 + .../container/keyvalue/KeyValueHandler.java | 25 +++++++++++++------ .../main/proto/DatanodeClientProtocol.proto | 2 +- 7 files changed, 37 insertions(+), 15 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index f5203f776510..d127cee230ab 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -64,6 +64,7 @@ import io.opentracing.Scope; import io.opentracing.Span; import io.opentracing.util.GlobalTracer; +import org.apache.hadoop.util.Time; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; @@ -572,7 +573,11 @@ public XceiverClientReply sendCommandAsyncReadOnly( ContainerProtos.StreamDataResponseProto.newBuilder(); checkOpen(dn); UUID dnID = dn.getUuid(); + Type cmdType = request.getCmdType(); semaphore.acquire(); + long requestTime = System.currentTimeMillis(); + metrics.incrPendingContainerOpsMetrics(cmdType); + final StreamObserver requestObserver = asyncStubs.get(dnID).withDeadlineAfter(timeout, TimeUnit.SECONDS) .send(new StreamObserver() { @@ -607,6 +612,8 @@ public void onCompleted() { try { return new XceiverClientReply(future); } finally { + metrics.decrPendingContainerOpsMetrics(cmdType); + metrics.addContainerOpsLatency(cmdType, System.currentTimeMillis() - requestTime); semaphore.release(); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index 62ef8e9a0191..4dfc61afc441 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -90,8 +90,7 @@ public class NewBlockInputStream extends BlockExtendedInputStream private final List validators; private final boolean verifyChecksum; private final Function refreshFunction; - private final RetryPolicy retryPolicy = - HddsClientUtils.createRetryPolicy(3, TimeUnit.SECONDS.toMillis(1)); + private final RetryPolicy retryPolicy; private int retries; @@ -110,6 +109,9 @@ public NewBlockInputStream( (request, response) -> validateBlock(response)); this.verifyChecksum = config.isChecksumVerify(); this.refreshFunction = refreshFunction; + this.retryPolicy = + HddsClientUtils.createRetryPolicy(config.getMaxReadRetryCount(), + TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index cfeab5a9c697..35fb797b83b3 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index 535e75e0ff44..8145622b5cbc 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -322,8 +322,7 @@ public static ContainerCommandResponseProto getReadBlockResponse( response = ReadBlockResponseProto.newBuilder() .setChunkData(chunkInfo) .setData(data.toByteString(byteBufferToByteString)) - .setBlockID(blockID) - .setStartIndex(startIndex); + .setBlockID(blockID); } else { // V1 splits response data into a list of ByteBuffers response = ReadBlockResponseProto.newBuilder() @@ -331,10 +330,11 @@ public static ContainerCommandResponseProto getReadBlockResponse( .setDataBuffers(DataBuffers.newBuilder() .addAllBuffers(data.toByteStringList(byteBufferToByteString)) .build()) - .setBlockID(blockID) - .setStartIndex(startIndex); + .setBlockID(blockID); + } + if (startIndex >= 0) { + response.setStartIndex(startIndex); } - return getSuccessResponseBuilder(request) .setReadBlock(response) .build(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java index 87572768e4af..eb9176a0b17a 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java @@ -84,6 +84,7 @@ public static CSMMetrics create(RaftGroupId gid) { } public void incNumWriteStateMachineOps() { + numWriteStateMachineOps.incr(); } 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 7846c39e69c0..7963835ccaa1 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 @@ -1258,14 +1258,23 @@ public ContainerCommandResponseProto streamDataReadOnly( .getChecksumData().getBytesPerChecksum(); long offset = readBlock.getOffset(); long len = readBlock.getLen(); - long adjustedChunkOffset = - (offset / bytesPerChecksum) * bytesPerChecksum; - long adjustedChunkLen = ((len + offset - 1) / bytesPerChecksum + 1) - * bytesPerChecksum; + long adjustedChunkOffset, adjustedChunkLen; + if (validateChunkChecksumData) { + adjustedChunkOffset = + (offset / bytesPerChecksum) * bytesPerChecksum; + adjustedChunkLen = ((len + offset - 1) / bytesPerChecksum + 1) + * bytesPerChecksum; + } else { + adjustedChunkOffset = offset; + adjustedChunkLen = len; + } + int startIndex = -1; len += (offset - adjustedChunkOffset); while (len > 0) { ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); - int startIndex = (int)(adjustedChunkOffset - chunk.getOffset()) / bytesPerChecksum; + if (validateChunkChecksumData) { + startIndex = (int)(adjustedChunkOffset - chunk.getOffset()) / bytesPerChecksum; + } long chunkLen = Math.min(adjustedChunkLen, chunk.getLen()); ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunkInfos .get(chunkIndex) @@ -1291,8 +1300,10 @@ public ContainerCommandResponseProto streamDataReadOnly( data, byteBufferToByteString, startIndex)); len -= chunkLen; adjustedChunkOffset += chunkLen; - adjustedChunkLen = ((len - 1) / bytesPerChecksum + 1) - * bytesPerChecksum; + if (validateChunkChecksumData) { + adjustedChunkLen = ((len - 1) / bytesPerChecksum + 1) + * bytesPerChecksum; + } chunkIndex++; } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 155d4f4b36ed..f500ddae9baa 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -388,7 +388,7 @@ message ReadBlockRequestProto { message ReadBlockResponseProto { required DatanodeBlockID blockID = 1; required ChunkInfo chunkData = 2; - required uint32 startIndex = 3; + optional uint32 startIndex = 3; // Chunk data should be returned in one of the two for oneof responseData { bytes data = 4; // Chunk data is returned as single buffer for V0 From 29c8f80990505251c9d5263451668b50ddef15fd Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 16 Apr 2024 02:57:10 +0800 Subject: [PATCH 022/114] fix bug --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 1 - .../io/BlockInputStreamFactoryImpl.java | 1 - .../container/keyvalue/KeyValueHandler.java | 79 +++++++++++-------- 3 files changed, 48 insertions(+), 33 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index d127cee230ab..44319ba3b929 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -64,7 +64,6 @@ import io.opentracing.Scope; import io.opentracing.Span; import io.opentracing.util.GlobalTracer; -import org.apache.hadoop.util.Time; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 35fb797b83b3..cfeab5a9c697 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; -import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; 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 7963835ccaa1..795d25a9160e 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 @@ -36,6 +36,7 @@ import java.util.function.Function; import com.google.common.util.concurrent.Striped; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.client.BlockID; @@ -52,6 +53,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; import org.apache.hadoop.hdds.scm.ByteStringConversion; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.utils.HddsServerUtil; @@ -1254,34 +1256,30 @@ public ContainerCommandResponseProto streamDataReadOnly( boolean isReadChunkV0 = readBlock.getVersion() .equals(ContainerProtos.ReadChunkVersion.V0); - int bytesPerChecksum = chunkInfos.get(chunkIndex) - .getChecksumData().getBytesPerChecksum(); long offset = readBlock.getOffset(); - long len = readBlock.getLen(); + long len = readBlock.getLen(); long adjustedChunkOffset, adjustedChunkLen; - if (validateChunkChecksumData) { - adjustedChunkOffset = - (offset / bytesPerChecksum) * bytesPerChecksum; - adjustedChunkLen = ((len + offset - 1) / bytesPerChecksum + 1) - * bytesPerChecksum; - } else { - adjustedChunkOffset = offset; - adjustedChunkLen = len; - } - int startIndex = -1; - len += (offset - adjustedChunkOffset); - while (len > 0) { + do { + int startIndex = -1; ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); - if (validateChunkChecksumData) { - startIndex = (int)(adjustedChunkOffset - chunk.getOffset()) / bytesPerChecksum; + if (conf.getObject(OzoneClientConfig.class).isChecksumVerify()) { + Pair adjustedOffsetAndLength = + computeChecksumBoundaries(chunk, offset, len); + adjustedChunkOffset = adjustedOffsetAndLength.getLeft(); + adjustedChunkLen = adjustedOffsetAndLength.getRight(); + startIndex = (int) adjustedChunkOffset / + chunk.getChecksumData().getBytesPerChecksum(); + adjustedChunkOffset += chunk.getOffset(); + } else { + adjustedChunkOffset = offset; + adjustedChunkLen = Math.min( + chunk.getLen() + chunk.getOffset() - offset, len); } - long chunkLen = Math.min(adjustedChunkLen, chunk.getLen()); - ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunkInfos - .get(chunkIndex) - .toBuilder() - .setOffset(adjustedChunkOffset) - .setLen(chunkLen) - .build()); + + ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf( + ContainerProtos.ChunkInfo.newBuilder(chunk) + .setOffset(adjustedChunkOffset) + .setLen(adjustedChunkLen).build()); // For older clients, set ReadDataIntoSingleBuffer to true so that // all the data read from chunk file is returned as a single // ByteString. Older clients cannot process data returned as a list @@ -1293,19 +1291,20 @@ public ContainerCommandResponseProto streamDataReadOnly( kvContainer, blockID, chunkInfo, dispatcherContext); Preconditions.checkNotNull(data, "Chunk data is null"); + if (DispatcherContext.op(dispatcherContext).readFromTmpFile()) { + validateChunkChecksumData(data, chunkInfo); + metrics.incBytesReadStateMachine(chunkInfo.getLen()); + metrics.incNumReadStateMachine(); + } streamObserver.onNext( getReadBlockResponse(request, blockData.getProtoBufMessage().getBlockID(), chunkInfo.getProtoBufMessage(), isReadChunkV0, data, byteBufferToByteString, startIndex)); - len -= chunkLen; - adjustedChunkOffset += chunkLen; - if (validateChunkChecksumData) { - adjustedChunkLen = ((len - 1) / bytesPerChecksum + 1) - * bytesPerChecksum; - } + len -= adjustedChunkLen + adjustedChunkOffset - offset; + offset = adjustedChunkOffset + adjustedChunkLen; chunkIndex++; - } + } while (len > 0); metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen()); } catch (StorageContainerException ex) { @@ -1318,6 +1317,24 @@ public ContainerCommandResponseProto streamDataReadOnly( return responseProto; } + private Pair computeChecksumBoundaries( + ContainerProtos.ChunkInfo chunkInfo, long startByteIndex, long dataLen) { + + int bytesPerChecksum = chunkInfo.getChecksumData().getBytesPerChecksum(); + long chunkOffset = chunkInfo.getOffset(); + startByteIndex = startByteIndex - chunkOffset; + // index of the last byte to be read from chunk, inclusively. + final long endByteIndex = startByteIndex + dataLen - 1; + + long adjustedChunkOffset = (startByteIndex / bytesPerChecksum) + * bytesPerChecksum; // inclusive + final long endIndex = ((endByteIndex / bytesPerChecksum) + 1) + * bytesPerChecksum; // exclusive + long adjustedChunkLen = + Math.min(endIndex, chunkInfo.getLen()) - adjustedChunkOffset; + return Pair.of(adjustedChunkOffset, adjustedChunkLen); + } + private String[] getFilesWithPrefix(String prefix, File chunkDir) { FilenameFilter filter = (dir, name) -> name.startsWith(prefix); return chunkDir.list(filter); From 741effbe6c6a7dfe8efc7d67885f7b4c1b27b45a Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 16 Apr 2024 04:54:51 +0800 Subject: [PATCH 023/114] fix bug --- .../hdds/scm/storage/NewBlockInputStream.java | 2 +- .../hdds/scm/storage/ContainerProtocolCalls.java | 16 +++++++++------- .../container/keyvalue/KeyValueHandler.java | 2 +- .../src/main/proto/DatanodeClientProtocol.proto | 3 ++- 4 files changed, 13 insertions(+), 10 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index 4dfc61afc441..de000b6331cd 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -522,7 +522,7 @@ protected long readData(long startByteIndex, long len) } StreamDataResponseProto response = ContainerProtocolCalls.readBlock(xceiverClient, startByteIndex, - len, blockBuilder.build(), validators, tokenRef.get()); + len, blockBuilder.build(), validators, tokenRef.get(), verifyChecksum); List readBlocks = response.getReadBlockList(); for (ReadBlockResponseProto readBlock : readBlocks) { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index ee535cee8a34..be9779b44eac 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -734,22 +734,24 @@ public static List toValidatorList(Validator validator) { /** * Calls the container protocol to read a chunk. * - * @param xceiverClient client to perform call - * @param chunk information about chunk to read - * @param blockID ID of the block - * @param validators functions to validate the response - * @param token a token for this block (may be null) + * @param chunk information about chunk to read + * @param xceiverClient client to perform call + * @param blockID ID of the block + * @param validators functions to validate the response + * @param token a token for this block (may be null) + * @param verifyChecksum * @return container protocol read chunk response * @throws IOException if there is an I/O error while performing the call */ public static ContainerProtos.StreamDataResponseProto readBlock( XceiverClientSpi xceiverClient, long offset, long len, DatanodeBlockID blockID, - List validators, - Token token) throws IOException { + List validators, Token token, + boolean verifyChecksum) throws IOException { ReadBlockRequestProto.Builder readBlockRequest = ReadBlockRequestProto.newBuilder() .setBlockID(blockID) .setOffset(offset) + .setVerifyChecksum(verifyChecksum) .setLen(len) .setVersion(ContainerProtos.ReadChunkVersion.V1); ContainerCommandRequestProto.Builder builder = 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 795d25a9160e..66d3e3eb72fe 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 @@ -1262,7 +1262,7 @@ public ContainerCommandResponseProto streamDataReadOnly( do { int startIndex = -1; ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); - if (conf.getObject(OzoneClientConfig.class).isChecksumVerify()) { + if (readBlock.getVerifyChecksum()) { Pair adjustedOffsetAndLength = computeChecksumBoundaries(chunk, offset, len); adjustedChunkOffset = adjustedOffsetAndLength.getLeft(); diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index f500ddae9baa..e9d1f9ae758f 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -382,7 +382,8 @@ message ReadBlockRequestProto { required DatanodeBlockID blockID = 1; required uint64 offset = 2; required uint64 len = 3; - optional ReadChunkVersion version = 4; + required bool verifyChecksum = 4; + optional ReadChunkVersion version = 5; } message ReadBlockResponseProto { From f1d4d7ff5de6adb7abde3367985416ffaa1d5ffc Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 16 Apr 2024 05:37:07 +0800 Subject: [PATCH 024/114] fix checkstyle --- .../apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java | 1 - 1 file changed, 1 deletion(-) 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 66d3e3eb72fe..8b879f83524b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -53,7 +53,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; import org.apache.hadoop.hdds.scm.ByteStringConversion; -import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.utils.HddsServerUtil; From 6a6737527ca609cc7da9454cef94adbf7cd26eab Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 17 Apr 2024 00:29:25 +0800 Subject: [PATCH 025/114] fix bug --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 12 +- .../hdds/scm/storage/NewBlockInputStream.java | 14 +-- ...tStream.java => TestBlockInputStream.java} | 105 ++++++++---------- .../client/rpc/read/TestKeyInputStream.java | 35 ++---- 4 files changed, 66 insertions(+), 100 deletions(-) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/{TestChunkInputStream.java => TestBlockInputStream.java} (71%) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 44319ba3b929..d00f2be739f3 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -608,14 +608,10 @@ public void onCompleted() { }); requestObserver.onNext(request); requestObserver.onCompleted(); - try { - return new XceiverClientReply(future); - } finally { - metrics.decrPendingContainerOpsMetrics(cmdType); - metrics.addContainerOpsLatency(cmdType, System.currentTimeMillis() - requestTime); - semaphore.release(); - } - + metrics.decrPendingContainerOpsMetrics(cmdType); + metrics.addContainerOpsLatency(cmdType, System.currentTimeMillis() - requestTime); + semaphore.release(); + return new XceiverClientReply(future); } private synchronized void checkOpen(DatanodeDetails dn) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index de000b6331cd..a342e2a5f793 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -46,7 +46,6 @@ import org.apache.hadoop.ozone.common.utils.BufferUtils; import org.apache.hadoop.security.token.Token; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.apache.ratis.thirdparty.io.grpc.Status; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -439,7 +438,7 @@ private void adjustBufferPosition(long bufferPosition) { bufferIndex = Collections.binarySearch( bufferoffsets.subList(0, bufferIndex), bufferPosition); } else if (bufferPosition >= bufferoffsets.get(bufferIndex) + - buffers.get(bufferIndex).limit()) { + buffers.get(bufferIndex).capacity()) { bufferIndex = Collections.binarySearch(bufferoffsets.subList( bufferIndex + 1, buffers.size()), bufferPosition); } @@ -494,6 +493,7 @@ private synchronized void readDataFromContainer(int len) throws IOException { blockPosition = getPos(); bufferOffsetWrtBlockDataData = readData(startByteIndex, len); long tempOffset = 0L; + buffersSize = 0L; bufferoffsets = new ArrayList<>(buffers.size()); for (ByteBuffer buffer : buffers) { bufferoffsets.add(tempOffset); @@ -728,11 +728,9 @@ private void refreshBlockInfo(IOException cause) throws IOException { } } - /** - * Check if this exception is because datanodes are not reachable. - */ - private boolean isConnectivityIssue(IOException ex) { - return Status.fromThrowable(ex).getCode() == Status.UNAVAILABLE.getCode(); + @VisibleForTesting + public ByteBuffer[] getCachedBuffers() { + return buffers == null ? null : + BufferUtils.getReadOnlyByteBuffers(buffers.toArray(new ByteBuffer[0])); } - } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java similarity index 71% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java index c5301ba4194f..dab080bda606 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; +import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.io.KeyInputStream; @@ -36,7 +36,7 @@ /** * Tests {@link ChunkInputStream}. */ -class TestChunkInputStream extends TestInputStreamBase { +class TestBlockInputStream extends TestInputStreamBase { /** * Run the tests as a single test method to avoid needing a new mini-cluster @@ -50,7 +50,7 @@ void testAll(ContainerLayoutVersion layout) throws Exception { try (OzoneClient client = cluster.newClient()) { TestBucket bucket = TestBucket.newBuilder(client).build(); - testChunkReadBuffers(bucket); + testBlockReadBuffers(bucket); testBufferRelease(bucket); testCloseReleasesBuffers(bucket); } @@ -62,28 +62,25 @@ void testAll(ContainerLayoutVersion layout) throws Exception { * Test to verify that data read from chunks is stored in a list of buffers * with max capacity equal to the bytes per checksum. */ - private void testChunkReadBuffers(TestBucket bucket) throws Exception { + private void testBlockReadBuffers(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - BlockInputStream block0Stream = - (BlockInputStream)keyInputStream.getPartStreams().get(0); - block0Stream.initialize(); - - ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0); + NewBlockInputStream block0Stream = + (NewBlockInputStream)keyInputStream.getPartStreams().get(0); // To read 1 byte of chunk data, ChunkInputStream should get one full // checksum boundary worth of data from Container and store it in buffers. - chunk0Stream.read(new byte[1]); - checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 1, 0, + block0Stream.read(new byte[1]); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, BYTES_PER_CHECKSUM); // Read > checksum boundary of data from chunk0 int readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); - byte[] readData = readDataFromChunk(chunk0Stream, 0, readDataLen); + byte[] readData = readDataFromChunk(block0Stream, 0, readDataLen); bucket.validateData(inputData, 0, readData); // The first checksum boundary size of data was already existing in the @@ -91,7 +88,7 @@ private void testChunkReadBuffers(TestBucket bucket) throws Exception { // boundary size of data will be fetched again to read the remaining data. // Hence there should be 1 checksum boundary size of data stored in the // ChunkStreams buffers at the end of the read. - checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 1, 0, + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, BYTES_PER_CHECKSUM); // Seek to a position in the third checksum boundary (so that current @@ -102,24 +99,24 @@ private void testChunkReadBuffers(TestBucket bucket) throws Exception { // and the second buffer should have BYTES_PER_CHECKSUM capacity. readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); int offset = 2 * BYTES_PER_CHECKSUM + 1; - readData = readDataFromChunk(chunk0Stream, offset, readDataLen); + readData = readDataFromChunk(block0Stream, offset, readDataLen); bucket.validateData(inputData, offset, readData); - checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 2, 1, + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 2, 1, BYTES_PER_CHECKSUM); // Read the full chunk data -1 and verify that all chunk data is read into // buffers. We read CHUNK_SIZE - 1 as otherwise all the buffers will be // released once all chunk data is read. - readData = readDataFromChunk(chunk0Stream, 0, CHUNK_SIZE - 1); + readData = readDataFromChunk(block0Stream, 0, CHUNK_SIZE - 1); bucket.validateData(inputData, 0, readData); int expectedNumBuffers = CHUNK_SIZE / BYTES_PER_CHECKSUM; - checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), expectedNumBuffers, expectedNumBuffers - 1, BYTES_PER_CHECKSUM); // Read the last byte of chunk and verify that the buffers are released. - chunk0Stream.read(new byte[1]); - assertNull(chunk0Stream.getCachedBuffers(), + block0Stream.read(new byte[1]); + assertNull(block0Stream.getCachedBuffers(), "ChunkInputStream did not release buffers after reaching EOF."); } } @@ -129,17 +126,15 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { bucket.writeRandomBytes(keyName, CHUNK_SIZE); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - BlockInputStream block0Stream = - (BlockInputStream) keyInputStream.getPartStreams().get(0); - block0Stream.initialize(); + NewBlockInputStream block0Stream = + (NewBlockInputStream) keyInputStream.getPartStreams().get(0); - ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0); - readDataFromChunk(chunk0Stream, 0, 1); - assertNotNull(chunk0Stream.getCachedBuffers()); + readDataFromChunk(block0Stream, 0, 1); + assertNotNull(block0Stream.getCachedBuffers()); - chunk0Stream.close(); + block0Stream.close(); - assertNull(chunk0Stream.getCachedBuffers()); + assertNull(block0Stream.getCachedBuffers()); } } @@ -153,70 +148,67 @@ private void testBufferRelease(TestBucket bucket) throws Exception { try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - BlockInputStream block0Stream = - (BlockInputStream)keyInputStream.getPartStreams().get(0); - block0Stream.initialize(); - - ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0); + NewBlockInputStream block0Stream = + (NewBlockInputStream)keyInputStream.getPartStreams().get(0); // Read checksum boundary - 1 bytes of data int readDataLen = BYTES_PER_CHECKSUM - 1; - byte[] readData = readDataFromChunk(chunk0Stream, 0, readDataLen); + byte[] readData = readDataFromChunk(block0Stream, 0, readDataLen); bucket.validateData(inputData, 0, readData); // There should be 1 byte of data remaining in the buffer which is not // yet read. Hence, the buffer should not be released. - checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, BYTES_PER_CHECKSUM); - assertEquals(1, chunk0Stream.getCachedBuffers()[0].remaining()); + assertEquals(1, block0Stream.getCachedBuffers()[0].remaining()); // Reading the last byte in the buffer should result in all the buffers // being released. - readData = readDataFromChunk(chunk0Stream, 1); + readData = readDataFromChunk(block0Stream, 1); bucket.validateData(inputData, readDataLen, readData); - assertNull(chunk0Stream.getCachedBuffers(), + assertNull(block0Stream.getCachedBuffers(), "Chunk stream buffers not released after last byte is read"); // Read more data to get the data till the next checksum boundary. readDataLen = BYTES_PER_CHECKSUM / 2; - readData = readDataFromChunk(chunk0Stream, readDataLen); + readData = readDataFromChunk(block0Stream, readDataLen); // There should be one buffer and the buffer should not be released as // there is data pending to be read from the buffer - checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 1, 0, + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, BYTES_PER_CHECKSUM); - ByteBuffer lastCachedBuffer = chunk0Stream.getCachedBuffers()[0]; + ByteBuffer lastCachedBuffer = block0Stream.getCachedBuffers()[0]; assertEquals(BYTES_PER_CHECKSUM - readDataLen, lastCachedBuffer.remaining()); // Read more than the remaining data in buffer (but less than the next // checksum boundary). - int position = (int) chunk0Stream.getPos(); + int position = (int) block0Stream.getPos(); readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; - readData = readDataFromChunk(chunk0Stream, readDataLen); + readData = readDataFromChunk(block0Stream, readDataLen); bucket.validateData(inputData, position, readData); // After reading the remaining data in the buffer, the buffer should be // released and next checksum size of data must be read into the buffers - checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 1, 0, + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, BYTES_PER_CHECKSUM); // Verify that the previously cached buffer is released by comparing it // with the current cached buffer assertNotEquals(lastCachedBuffer, - chunk0Stream.getCachedBuffers()[0]); + block0Stream.getCachedBuffers()[0]); } } - private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, + private byte[] readDataFromChunk(NewBlockInputStream blockInputStream, int offset, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; - chunkInputStream.seek(offset); - chunkInputStream.read(readData, 0, readDataLength); + blockInputStream.seek(offset); + blockInputStream.read(readData, 0, readDataLength); return readData; } - private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, + private byte[] readDataFromChunk(NewBlockInputStream blockInputStream, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; - chunkInputStream.read(readData, 0, readDataLength); + blockInputStream.read(readData, 0, readDataLength); return readData; } @@ -232,18 +224,13 @@ private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, private void checkBufferSizeAndCapacity(ByteBuffer[] buffers, int expectedNumBuffers, int numReleasedBuffers, long expectedBufferCapacity) { - assertEquals(expectedNumBuffers, buffers.length, - "ChunkInputStream does not have expected number of " + + assertEquals(expectedNumBuffers - numReleasedBuffers, buffers.length, + "BlockInputStream does not have expected number of " + "ByteBuffers"); for (int i = 0; i < buffers.length; i++) { - if (i <= numReleasedBuffers - 1) { - // This buffer should have been released and hence null - assertNull(buffers[i], - "ChunkInputStream Buffer not released after being read"); - } else { - assertEquals(expectedBufferCapacity, buffers[i].capacity(), - "ChunkInputStream ByteBuffer capacity is wrong"); - } + assertEquals(expectedBufferCapacity, buffers[i].capacity(), + "BlockInputStream ByteBuffer capacity is wrong"); + } } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java index 3ab8ae31188c..3e9d6c936464 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java @@ -29,11 +29,12 @@ import org.apache.hadoop.hdds.client.RatisReplicationConfig; 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.Type; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientMetrics; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; -import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; +import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.io.KeyInputStream; @@ -156,26 +157,10 @@ private void testInputStreams(TestBucket bucket) throws Exception { int readBlockLength = 0; for (BlockExtendedInputStream stream : blockStreams) { - BlockInputStream blockStream = (BlockInputStream) stream; + NewBlockInputStream blockStream = (NewBlockInputStream) stream; int blockStreamLength = Math.min(BLOCK_SIZE, dataLength - readBlockLength); assertEquals(blockStreamLength, blockStream.getLength()); - - int expectedNumChunkStreams = - BufferUtils.getNumberOfBins(blockStreamLength, CHUNK_SIZE); - blockStream.initialize(); - List chunkStreams = blockStream.getChunkStreams(); - assertEquals(expectedNumChunkStreams, chunkStreams.size()); - - int readChunkLength = 0; - for (ChunkInputStream chunkStream : chunkStreams) { - int chunkStreamLength = Math.min(CHUNK_SIZE, - blockStreamLength - readChunkLength); - assertEquals(chunkStreamLength, chunkStream.getRemaining()); - - readChunkLength += chunkStreamLength; - } - readBlockLength += blockStreamLength; } } @@ -248,7 +233,7 @@ public void testSeek(TestBucket bucket) throws Exception { long writeChunkCount = metrics.getContainerOpCountMetrics( ContainerProtos.Type.WriteChunk); long readChunkCount = metrics.getContainerOpCountMetrics( - ContainerProtos.Type.ReadChunk); + Type.ReadBlock); String keyName = getNewKeyName(); // write data spanning 3 chunks @@ -267,15 +252,15 @@ public void testSeek(TestBucket bucket) throws Exception { // Seek operation should not result in any readChunk operation. assertEquals(readChunkCount, metrics - .getContainerOpCountMetrics(ContainerProtos.Type.ReadChunk)); + .getContainerOpCountMetrics(Type.ReadBlock)); byte[] readData = new byte[CHUNK_SIZE]; keyInputStream.read(readData, 0, CHUNK_SIZE); // Since we read data from index 150 to 250 and the chunk boundary is // 100 bytes, we need to read 2 chunks. - assertEquals(readChunkCount + 2, - metrics.getContainerOpCountMetrics(ContainerProtos.Type.ReadChunk)); + assertEquals(readChunkCount + 1, + metrics.getContainerOpCountMetrics(Type.ReadBlock)); keyInputStream.close(); @@ -360,15 +345,15 @@ private void testSkip(TestBucket bucket) throws Exception { // Skip operation should not result in any readChunk operation. assertEquals(readChunkCount, metrics - .getContainerOpCountMetrics(ContainerProtos.Type.ReadChunk)); + .getContainerOpCountMetrics(Type.ReadBlock)); byte[] readData = new byte[CHUNK_SIZE]; keyInputStream.read(readData, 0, CHUNK_SIZE); // Since we reading data from index 150 to 250 and the chunk boundary is // 100 bytes, we need to read 2 chunks. - assertEquals(readChunkCount + 2, - metrics.getContainerOpCountMetrics(ContainerProtos.Type.ReadChunk)); + assertEquals(readChunkCount + 1, + metrics.getContainerOpCountMetrics(Type.ReadBlock)); keyInputStream.close(); From b4cfd3f1b2e94b33554122ab3ab3a2766080b470 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 17 Apr 2024 00:38:45 +0800 Subject: [PATCH 026/114] fix checkstyle --- .../apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java index 3e9d6c936464..b2b48a126f6f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientMetrics; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; -import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; From 91631ac8c6d97dfff11add7fd8aad6e893388f5e Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 17 Apr 2024 01:07:31 +0800 Subject: [PATCH 027/114] fix bug --- .../org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index a342e2a5f793..d3440defc8c9 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -729,7 +729,7 @@ private void refreshBlockInfo(IOException cause) throws IOException { } @VisibleForTesting - public ByteBuffer[] getCachedBuffers() { + public synchronized ByteBuffer[] getCachedBuffers() { return buffers == null ? null : BufferUtils.getReadOnlyByteBuffers(buffers.toArray(new ByteBuffer[0])); } From 6e36ec1e2d90d15a1bc1d65f9491643f02200213 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 17 Apr 2024 23:09:22 +0800 Subject: [PATCH 028/114] fix bug --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 17 ++- .../hdds/scm/storage/NewBlockInputStream.java | 44 ++++++++ .../io/BlockInputStreamFactoryImpl.java | 3 +- .../org/apache/hadoop/hdds/HddsUtils.java | 6 ++ .../main/proto/DatanodeClientProtocol.proto | 40 +++---- .../om/TestOmContainerLocationCache.java | 101 +++++------------- 6 files changed, 114 insertions(+), 97 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index d00f2be739f3..7fdd63b42344 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -585,12 +585,22 @@ public void onNext( ContainerCommandResponseProto responseProto) { ReadBlockResponseProto readBlock = responseProto.getReadBlock(); - streamData.addReadBlock(readBlock); + if (responseProto.getResult() == Result.SUCCESS) { + streamData.addReadBlock(readBlock); + } else { + future.complete( + ContainerCommandResponseProto.newBuilder(responseProto) + .setCmdType(Type.StreamRead).build()); + } } @Override public void onError(Throwable t) { future.completeExceptionally(t); + metrics.decrPendingContainerOpsMetrics(cmdType); + metrics.addContainerOpsLatency( + cmdType, System.currentTimeMillis() - requestTime); + } @Override @@ -604,12 +614,13 @@ public void onCompleted() { "Stream completed but no reply for request " + processForDebug(request))); } + metrics.decrPendingContainerOpsMetrics(cmdType); + metrics.addContainerOpsLatency( + cmdType, System.currentTimeMillis() - requestTime); } }); requestObserver.onNext(request); requestObserver.onCompleted(); - metrics.decrPendingContainerOpsMetrics(cmdType); - metrics.addContainerOpsLatency(cmdType, System.currentTimeMillis() - requestTime); semaphore.release(); return new XceiverClientReply(future); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java index d3440defc8c9..2490fc32ed49 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java @@ -46,6 +46,7 @@ import org.apache.hadoop.ozone.common.utils.BufferUtils; import org.apache.hadoop.security.token.Token; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.Status; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -172,6 +173,18 @@ public synchronized int read() throws IOException { } else { throw e; } + } catch (IOException ioe) { + if (shouldRetryRead(ioe)) { + if (isConnectivityIssue(ioe)) { + releaseClient(); + refreshBlockInfo(ioe); + } else { + releaseClient(); + } + continue; + } else { + throw ioe; + } } if (available == EOF) { // There is no more data in the chunk stream. The buffers should have @@ -231,6 +244,18 @@ public synchronized int read(byte[] b, int off, int len) throws IOException { } else { throw e; } + } catch (IOException ioe) { + if (shouldRetryRead(ioe)) { + if (isConnectivityIssue(ioe)) { + releaseClient(); + refreshBlockInfo(ioe); + } else { + releaseClient(); + } + continue; + } else { + throw ioe; + } } if (available == EOF) { // There is no more data in the block stream. The buffers should have @@ -283,6 +308,18 @@ public synchronized int read(ByteBuffer byteBuffer) throws IOException { } else { throw e; } + } catch (IOException ioe) { + if (shouldRetryRead(ioe)) { + if (isConnectivityIssue(ioe)) { + releaseClient(); + refreshBlockInfo(ioe); + } else { + releaseClient(); + } + continue; + } else { + throw ioe; + } } if (available == EOF) { // There is no more data in the block stream. The buffers should have @@ -733,4 +770,11 @@ public synchronized ByteBuffer[] getCachedBuffers() { return buffers == null ? null : BufferUtils.getReadOnlyByteBuffers(buffers.toArray(new ByteBuffer[0])); } + + /** + * Check if this exception is because datanodes are not reachable. + */ + private boolean isConnectivityIssue(IOException ex) { + return Status.fromThrowable(ex).getCode() == Status.UNAVAILABLE.getCode(); + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index cfeab5a9c697..91dd6057659f 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -86,7 +86,8 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, blockInfo, xceiverFactory, refreshFunction, ecBlockStreamFactory, config); } else { - return new NewBlockInputStream(blockInfo.getBlockID(), blockInfo.getLength(), + return new NewBlockInputStream( + blockInfo.getBlockID(), blockInfo.getLength(), pipeline, token, xceiverFactory, refreshFunction, config); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java index f5a00e2dbe46..e0c0737efc2c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java @@ -418,6 +418,7 @@ public static boolean isReadOnly( switch (proto.getCmdType()) { case ReadContainer: case ReadChunk: + case ReadBlock: case ListBlock: case GetBlock: case GetSmallFile: @@ -474,6 +475,7 @@ public static boolean requireBlockToken( case PutBlock: case PutSmallFile: case ReadChunk: + case ReadBlock: case WriteChunk: return true; default: @@ -549,6 +551,10 @@ public static BlockID getBlockID(ContainerCommandRequestProtoOrBuilder msg) { blockID = msg.getReadChunk().getBlockID(); } break; + case ReadBlock: + if (msg.hasReadBlock()) { + blockID = msg.getReadBlock().getBlockID(); + } case WriteChunk: if (msg.hasWriteChunk()) { blockID = msg.getWriteChunk().getBlockID(); diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index e9d1f9ae758f..9b6f002b3280 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -88,7 +88,6 @@ enum Type { GetBlock = 7; DeleteBlock = 8; ListBlock = 9; - ReadBlock = 21; ReadChunk = 10; DeleteChunk = 11; @@ -104,6 +103,7 @@ enum Type { StreamInit = 19; StreamWrite = 20; + ReadBlock = 21; StreamRead = 22; } @@ -198,19 +198,19 @@ message ContainerCommandRequestProto { optional GetBlockRequestProto getBlock = 13; optional DeleteBlockRequestProto deleteBlock = 14 [deprecated = true]; optional ListBlockRequestProto listBlock = 15; - optional ReadBlockRequestProto readBlock = 25; + optional ReadBlockRequestProto readBlock = 16; - optional ReadChunkRequestProto readChunk = 16; - optional WriteChunkRequestProto writeChunk = 17; - optional DeleteChunkRequestProto deleteChunk = 18 [deprecated = true]; - optional ListChunkRequestProto listChunk = 19; + optional ReadChunkRequestProto readChunk = 17; + optional WriteChunkRequestProto writeChunk = 18; + optional DeleteChunkRequestProto deleteChunk = 19 [deprecated = true]; + optional ListChunkRequestProto listChunk = 20; - optional PutSmallFileRequestProto putSmallFile = 20; - optional GetSmallFileRequestProto getSmallFile = 21; - optional GetCommittedBlockLengthRequestProto getCommittedBlockLength = 22; + optional PutSmallFileRequestProto putSmallFile = 21; + optional GetSmallFileRequestProto getSmallFile = 22; + optional GetCommittedBlockLengthRequestProto getCommittedBlockLength = 23; - optional string encodedToken = 23; - optional uint32 version = 24; + optional string encodedToken = 24; + optional uint32 version = 25; } message ContainerCommandResponseProto { @@ -231,18 +231,18 @@ message ContainerCommandResponseProto { optional GetBlockResponseProto getBlock = 12; optional DeleteBlockResponseProto deleteBlock = 13; optional ListBlockResponseProto listBlock = 14; - optional ReadBlockResponseProto readBlock = 22; - optional StreamDataResponseProto streamData = 23; + optional ReadBlockResponseProto readBlock = 15; - optional WriteChunkResponseProto writeChunk = 15; - optional ReadChunkResponseProto readChunk = 16; - optional DeleteChunkResponseProto deleteChunk = 17; - optional ListChunkResponseProto listChunk = 18; + optional WriteChunkResponseProto writeChunk = 16; + optional ReadChunkResponseProto readChunk = 17; + optional DeleteChunkResponseProto deleteChunk = 18; + optional ListChunkResponseProto listChunk = 19; - optional PutSmallFileResponseProto putSmallFile = 19; - optional GetSmallFileResponseProto getSmallFile = 20; + optional PutSmallFileResponseProto putSmallFile = 20; + optional GetSmallFileResponseProto getSmallFile = 21; - optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21; + optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 22; + optional StreamDataResponseProto streamData = 23; } message ContainerDataProto { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java index 2ae69dc3c96f..3139b573024f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java @@ -28,16 +28,15 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; 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.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetCommittedBlockLengthResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; @@ -292,8 +291,7 @@ public void containerCachedInHappyCase() throws Exception { verify(mockScmContainerClient, times(1)) .getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get())); - mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); - mockReadChunk(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); try (InputStream is = key1.getContent()) { byte[] read = new byte[(int) key1.getDataSize()]; IOUtils.read(is, read); @@ -367,11 +365,8 @@ public void containerRefreshedAfterDatanodeGetBlockError( try (InputStream is = key1.getContent()) { // Simulate dn1 got errors, and the container's moved to dn2. - mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, - dnException, dnResponseCode); mockScmGetContainerPipeline(CONTAINER_ID.get(), DN2); - mockGetBlock(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); - mockReadChunk(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadBlock(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); byte[] read = new byte[(int) key1.getDataSize()]; IOUtils.read(is, read); @@ -415,12 +410,10 @@ public void containerRefreshedAfterDatanodeReadChunkError( try (InputStream is = key1.getContent()) { // simulate dn1 goes down, the container's to dn2. - mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); - mockReadChunk(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, + mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, dnException, dnResponseCode); mockScmGetContainerPipeline(CONTAINER_ID.get(), DN2); - mockGetBlock(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); - mockReadChunk(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadBlock(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); byte[] read = new byte[(int) key1.getDataSize()]; IOUtils.read(is, read); @@ -464,7 +457,7 @@ public void containerNotRefreshedAfterDatanodeGetBlockError( try (InputStream is = key1.getContent()) { // simulate dn1 got errors, and the container's moved to dn2. - mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, ex, + mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, ex, errorCode); assertThrows(expectedEx, @@ -508,8 +501,7 @@ public void containerNotRefreshedAfterDatanodeReadChunkError( try (InputStream is = key1.getContent()) { // simulate dn1 got errors, and the container's moved to dn2. - mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); - mockReadChunk(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, + mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, dnException, dnResponseCode); assertThrows(expectedEx, @@ -561,8 +553,7 @@ public void containerRefreshedOnEmptyPipelines() throws Exception { // but the empty pipeline is not cached, and when some data node is back. mockScmGetContainerPipeline(CONTAINER_ID.get(), DN1); - mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); - mockReadChunk(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); // the subsequent effort to read the key is success. OzoneKeyDetails updatedKey1 = bucket.getKey(keyName); try (InputStream is = updatedKey1.getContent()) { @@ -668,41 +659,42 @@ private void mockScmGetContainerPipelineEmpty(long containerId) newHashSet(containerId))).thenReturn(containerWithPipelines); } - private void mockGetBlock(XceiverClientGrpc mockDnProtocol, - long containerId, long localId, - byte[] data, - Exception exception, - Result errorCode) throws Exception { - + private void mockReadBlock(XceiverClientGrpc mockDnProtocol, + long containerId, long localId, + byte[] data, + Exception exception, + Result errorCode) throws Exception { final CompletableFuture response; if (exception != null) { response = new CompletableFuture<>(); response.completeExceptionally(exception); } else if (errorCode != null) { - ContainerCommandResponseProto getBlockResp = + ContainerCommandResponseProto readBlock = ContainerCommandResponseProto.newBuilder() .setResult(errorCode) - .setCmdType(Type.GetBlock) + .setCmdType(Type.ReadBlock) .build(); - response = completedFuture(getBlockResp); + response = completedFuture(readBlock); } else { - ContainerCommandResponseProto getBlockResp = + ContainerCommandResponseProto readBlock = ContainerCommandResponseProto.newBuilder() - .setGetBlock(GetBlockResponseProto.newBuilder() - .setBlockData(BlockData.newBuilder() - .addChunks(createChunkInfo(data)) + .setStreamData(StreamDataResponseProto.newBuilder() + .addReadBlock(ReadBlockResponseProto.newBuilder() .setBlockID(createBlockId(containerId, localId)) + .setChunkData(createChunkInfo(data)) + .setData(ByteString.copyFrom(data)) .build()) - .build() - ) + .build()) .setResult(Result.SUCCESS) - .setCmdType(Type.GetBlock) + .setCmdType(Type.StreamRead) .build(); - response = completedFuture(getBlockResp); + response = completedFuture(readBlock); } + doAnswer(invocation -> new XceiverClientReply(response)) .when(mockDnProtocol) - .sendCommandAsync(argThat(matchCmd(Type.GetBlock)), any()); + .sendCommandAsyncReadOnly(argThat(matchCmd(Type.ReadBlock)), any()); + } @Nonnull @@ -716,43 +708,6 @@ private ChunkInfo createChunkInfo(byte[] data) throws Exception { .build(); } - private void mockReadChunk(XceiverClientGrpc mockDnProtocol, - long containerId, long localId, - byte[] data, - Exception exception, - Result errorCode) throws Exception { - final CompletableFuture response; - if (exception != null) { - response = new CompletableFuture<>(); - response.completeExceptionally(exception); - } else if (errorCode != null) { - ContainerCommandResponseProto readChunkResp = - ContainerCommandResponseProto.newBuilder() - .setResult(errorCode) - .setCmdType(Type.ReadChunk) - .build(); - response = completedFuture(readChunkResp); - } else { - ContainerCommandResponseProto readChunkResp = - ContainerCommandResponseProto.newBuilder() - .setReadChunk(ReadChunkResponseProto.newBuilder() - .setBlockID(createBlockId(containerId, localId)) - .setChunkData(createChunkInfo(data)) - .setData(ByteString.copyFrom(data)) - .build() - ) - .setResult(Result.SUCCESS) - .setCmdType(Type.ReadChunk) - .build(); - response = completedFuture(readChunkResp); - } - - doAnswer(invocation -> new XceiverClientReply(response)) - .when(mockDnProtocol) - .sendCommandAsync(argThat(matchCmd(Type.ReadChunk)), any()); - - } - private static Pipeline createPipeline(DatanodeDetails dn) { return createPipeline(Collections.singletonList(dn)); } From e0b1d2b6eee467b9df523f8d2941ce2d42fca39b Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 17 Apr 2024 23:25:45 +0800 Subject: [PATCH 029/114] fix bug --- .../common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java index e0c0737efc2c..a04642960bbc 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java @@ -555,6 +555,7 @@ public static BlockID getBlockID(ContainerCommandRequestProtoOrBuilder msg) { if (msg.hasReadBlock()) { blockID = msg.getReadBlock().getBlockID(); } + break; case WriteChunk: if (msg.hasWriteChunk()) { blockID = msg.getWriteChunk().getBlockID(); From 0e8576e01f2e6b36236cf69e5cff2a9757855e44 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 17 Apr 2024 23:38:35 +0800 Subject: [PATCH 030/114] fix bug --- .../src/main/proto/DatanodeClientProtocol.proto | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 9b6f002b3280..ff2d878cd78b 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -231,17 +231,17 @@ message ContainerCommandResponseProto { optional GetBlockResponseProto getBlock = 12; optional DeleteBlockResponseProto deleteBlock = 13; optional ListBlockResponseProto listBlock = 14; - optional ReadBlockResponseProto readBlock = 15; - optional WriteChunkResponseProto writeChunk = 16; - optional ReadChunkResponseProto readChunk = 17; - optional DeleteChunkResponseProto deleteChunk = 18; - optional ListChunkResponseProto listChunk = 19; + optional WriteChunkResponseProto writeChunk = 15; + optional ReadChunkResponseProto readChunk = 16; + optional DeleteChunkResponseProto deleteChunk = 17; + optional ListChunkResponseProto listChunk = 18; - optional PutSmallFileResponseProto putSmallFile = 20; - optional GetSmallFileResponseProto getSmallFile = 21; + optional PutSmallFileResponseProto putSmallFile = 19; + optional GetSmallFileResponseProto getSmallFile = 20; - optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 22; + optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21; + optional ReadBlockResponseProto readBlock = 22; optional StreamDataResponseProto streamData = 23; } From f243c46e03b57737678d78b39518db444485d4f7 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 17 Apr 2024 23:41:12 +0800 Subject: [PATCH 031/114] fix bug --- .../src/main/proto/DatanodeClientProtocol.proto | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index ff2d878cd78b..9b6f002b3280 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -231,17 +231,17 @@ message ContainerCommandResponseProto { optional GetBlockResponseProto getBlock = 12; optional DeleteBlockResponseProto deleteBlock = 13; optional ListBlockResponseProto listBlock = 14; + optional ReadBlockResponseProto readBlock = 15; - optional WriteChunkResponseProto writeChunk = 15; - optional ReadChunkResponseProto readChunk = 16; - optional DeleteChunkResponseProto deleteChunk = 17; - optional ListChunkResponseProto listChunk = 18; + optional WriteChunkResponseProto writeChunk = 16; + optional ReadChunkResponseProto readChunk = 17; + optional DeleteChunkResponseProto deleteChunk = 18; + optional ListChunkResponseProto listChunk = 19; - optional PutSmallFileResponseProto putSmallFile = 19; - optional GetSmallFileResponseProto getSmallFile = 20; + optional PutSmallFileResponseProto putSmallFile = 20; + optional GetSmallFileResponseProto getSmallFile = 21; - optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21; - optional ReadBlockResponseProto readBlock = 22; + optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 22; optional StreamDataResponseProto streamData = 23; } From 80eb936538a4e518ce3c91eef469155f17db718c Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 18 Apr 2024 00:22:26 +0800 Subject: [PATCH 032/114] fix bug --- .../main/proto/DatanodeClientProtocol.proto | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 9b6f002b3280..1e57719a81c5 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -198,19 +198,19 @@ message ContainerCommandRequestProto { optional GetBlockRequestProto getBlock = 13; optional DeleteBlockRequestProto deleteBlock = 14 [deprecated = true]; optional ListBlockRequestProto listBlock = 15; - optional ReadBlockRequestProto readBlock = 16; - optional ReadChunkRequestProto readChunk = 17; - optional WriteChunkRequestProto writeChunk = 18; - optional DeleteChunkRequestProto deleteChunk = 19 [deprecated = true]; - optional ListChunkRequestProto listChunk = 20; + optional ReadChunkRequestProto readChunk = 16; + optional WriteChunkRequestProto writeChunk = 17; + optional DeleteChunkRequestProto deleteChunk = 18 [deprecated = true]; + optional ListChunkRequestProto listChunk = 19; - optional PutSmallFileRequestProto putSmallFile = 21; - optional GetSmallFileRequestProto getSmallFile = 22; - optional GetCommittedBlockLengthRequestProto getCommittedBlockLength = 23; + optional PutSmallFileRequestProto putSmallFile = 20; + optional GetSmallFileRequestProto getSmallFile = 21; + optional GetCommittedBlockLengthRequestProto getCommittedBlockLength = 22; - optional string encodedToken = 24; - optional uint32 version = 25; + optional string encodedToken = 23; + optional uint32 version = 24; + optional ReadBlockRequestProto readBlock = 25; } message ContainerCommandResponseProto { @@ -231,17 +231,17 @@ message ContainerCommandResponseProto { optional GetBlockResponseProto getBlock = 12; optional DeleteBlockResponseProto deleteBlock = 13; optional ListBlockResponseProto listBlock = 14; - optional ReadBlockResponseProto readBlock = 15; - optional WriteChunkResponseProto writeChunk = 16; - optional ReadChunkResponseProto readChunk = 17; - optional DeleteChunkResponseProto deleteChunk = 18; - optional ListChunkResponseProto listChunk = 19; + optional WriteChunkResponseProto writeChunk = 15; + optional ReadChunkResponseProto readChunk = 16; + optional DeleteChunkResponseProto deleteChunk = 17; + optional ListChunkResponseProto listChunk = 18; - optional PutSmallFileResponseProto putSmallFile = 20; - optional GetSmallFileResponseProto getSmallFile = 21; + optional PutSmallFileResponseProto putSmallFile = 19; + optional GetSmallFileResponseProto getSmallFile = 20; - optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 22; + optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21; + optional ReadBlockResponseProto readBlock = 22; optional StreamDataResponseProto streamData = 23; } From b26b51d9946d0692730457b309add50e88e12383 Mon Sep 17 00:00:00 2001 From: Chung En Lee Date: Thu, 2 May 2024 19:43:04 +0000 Subject: [PATCH 033/114] fix protobuf --- .../src/main/proto/DatanodeClientProtocol.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index e5029337f484..2395fd4e8e72 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -212,7 +212,7 @@ message ContainerCommandRequestProto { optional string encodedToken = 23; optional uint32 version = 24; optional EchoRequestProto echo = 25; - optional ReadBlockRequestProto readBlock = 25; + optional ReadBlockRequestProto readBlock = 26; } message ContainerCommandResponseProto { From c4e8b2b8968a34f436631479e24eb158ab72c076 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 27 May 2024 17:36:45 +0100 Subject: [PATCH 034/114] rename NewBlockInputStream --- .../dev-support/findbugsExcludeFile.xml | 2 +- ...InputStream.java => StreamBlockInput.java} | 6 ++--- .../io/BlockInputStreamFactoryImpl.java | 4 ++-- ...Stream.java => DummyStreamBlockInput.java} | 6 ++--- ...tStream.java => TestStreamBlockInput.java} | 8 +++---- .../io/TestBlockInputStreamFactoryImpl.java | 4 ++-- .../client/rpc/read/TestBlockInputStream.java | 22 +++++++++---------- .../client/rpc/read/TestKeyInputStream.java | 4 ++-- 8 files changed, 28 insertions(+), 28 deletions(-) rename hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/{NewBlockInputStream.java => StreamBlockInput.java} (99%) rename hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/{DummyNewBlockInputStream.java => DummyStreamBlockInput.java} (97%) rename hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/{TestNewBlockInputStream.java => TestStreamBlockInput.java} (97%) diff --git a/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml b/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml index a24100620acc..cf13ea967f67 100644 --- a/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml @@ -33,7 +33,7 @@ - + diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java similarity index 99% rename from hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java rename to hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 2490fc32ed49..40c736ad13dd 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/NewBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -66,10 +66,10 @@ * An {@link java.io.InputStream} called from KeyInputStream to read a block from the * container. */ -public class NewBlockInputStream extends BlockExtendedInputStream +public class StreamBlockInput extends BlockExtendedInputStream implements Seekable, CanUnbuffer, ByteBufferReadable { private static final Logger LOG = - LoggerFactory.getLogger(NewBlockInputStream.class); + LoggerFactory.getLogger(StreamBlockInput.class); private final BlockID blockID; private final long length; private final AtomicReference pipelineRef = @@ -94,7 +94,7 @@ public class NewBlockInputStream extends BlockExtendedInputStream private int retries; - public NewBlockInputStream( + public StreamBlockInput( BlockID blockID, long length, Pipeline pipeline, Token token, XceiverClientFactory xceiverClientFactory, diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 91dd6057659f..d7e27adbb6a6 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; -import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; +import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.io.ElasticByteBufferPool; @@ -86,7 +86,7 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, blockInfo, xceiverFactory, refreshFunction, ecBlockStreamFactory, config); } else { - return new NewBlockInputStream( + return new StreamBlockInput( blockInfo.getBlockID(), blockInfo.getLength(), pipeline, token, xceiverFactory, refreshFunction, config); } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java similarity index 97% rename from hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java rename to hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java index 22399b555f33..3a092f238309 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyNewBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java @@ -36,9 +36,9 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; /** - * A dummy NewBlockInputStream to mock read block call to DN. + * A dummy StreamBlockInput to mock read block call to DN. */ -class DummyNewBlockInputStream extends NewBlockInputStream { +class DummyStreamBlockInput extends StreamBlockInput { private final List readByteBuffers = new ArrayList<>(); private final List chunks; @@ -46,7 +46,7 @@ class DummyNewBlockInputStream extends NewBlockInputStream { private final Map chunkDataMap; @SuppressWarnings("parameternumber") - DummyNewBlockInputStream( + DummyStreamBlockInput( BlockID blockId, long blockLen, Pipeline pipeline, diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java similarity index 97% rename from hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java rename to hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java index 07c8b19cd4ee..a143d4f8ee36 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestNewBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java @@ -49,15 +49,15 @@ import static org.mockito.Mockito.mock; /** - * Tests for {@link TestNewBlockInputStream}'s functionality. + * Tests for {@link TestStreamBlockInput}'s functionality. */ -public class TestNewBlockInputStream { +public class TestStreamBlockInput { private int blockSize; private static final int CHUNK_SIZE = 100; private static final int BYTES_PER_CHECKSUM = 20; private static final Random RANDOM = new Random(); private static final AtomicLong CONTAINER_ID = new AtomicLong(); - private DummyNewBlockInputStream blockStream; + private DummyStreamBlockInput blockStream; private byte[] blockData; private List chunks; private Map chunkDataMap; @@ -77,7 +77,7 @@ public void setup() throws Exception { createChunkList(5); Pipeline pipeline = MockPipeline.createSingleNodePipeline(); - blockStream = new DummyNewBlockInputStream(blockID, blockSize, pipeline, + blockStream = new DummyStreamBlockInput(blockID, blockSize, pipeline, null, null, refreshFunction, clientConfig, chunks, chunkDataMap); } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java index a0138e4e89cb..faf0151bf529 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; -import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; +import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -62,7 +62,7 @@ public void testNonECGivesBlockInputStream() { factory.create(repConfig, blockInfo, blockInfo.getPipeline(), blockInfo.getToken(), null, null, clientConfig); - assertInstanceOf(NewBlockInputStream.class, stream); + assertInstanceOf(StreamBlockInput.class, stream); assertEquals(stream.getBlockID(), blockInfo.getBlockID()); assertEquals(stream.getLength(), blockInfo.getLength()); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java index dab080bda606..09eb4f81d672 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java @@ -21,7 +21,7 @@ import java.nio.ByteBuffer; import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; -import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; +import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.io.KeyInputStream; @@ -69,8 +69,8 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - NewBlockInputStream block0Stream = - (NewBlockInputStream)keyInputStream.getPartStreams().get(0); + StreamBlockInput block0Stream = + (StreamBlockInput)keyInputStream.getPartStreams().get(0); // To read 1 byte of chunk data, ChunkInputStream should get one full // checksum boundary worth of data from Container and store it in buffers. @@ -126,8 +126,8 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { bucket.writeRandomBytes(keyName, CHUNK_SIZE); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - NewBlockInputStream block0Stream = - (NewBlockInputStream) keyInputStream.getPartStreams().get(0); + StreamBlockInput block0Stream = + (StreamBlockInput) keyInputStream.getPartStreams().get(0); readDataFromChunk(block0Stream, 0, 1); assertNotNull(block0Stream.getCachedBuffers()); @@ -148,8 +148,8 @@ private void testBufferRelease(TestBucket bucket) throws Exception { try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - NewBlockInputStream block0Stream = - (NewBlockInputStream)keyInputStream.getPartStreams().get(0); + StreamBlockInput block0Stream = + (StreamBlockInput)keyInputStream.getPartStreams().get(0); // Read checksum boundary - 1 bytes of data int readDataLen = BYTES_PER_CHECKSUM - 1; @@ -197,16 +197,16 @@ private void testBufferRelease(TestBucket bucket) throws Exception { } } - private byte[] readDataFromChunk(NewBlockInputStream blockInputStream, - int offset, int readDataLength) throws IOException { + private byte[] readDataFromChunk(StreamBlockInput blockInputStream, + int offset, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; blockInputStream.seek(offset); blockInputStream.read(readData, 0, readDataLength); return readData; } - private byte[] readDataFromChunk(NewBlockInputStream blockInputStream, - int readDataLength) throws IOException { + private byte[] readDataFromChunk(StreamBlockInput blockInputStream, + int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; blockInputStream.read(readData, 0, readDataLength); return readData; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java index b2b48a126f6f..6d79977b5260 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientMetrics; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; -import org.apache.hadoop.hdds.scm.storage.NewBlockInputStream; +import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.io.KeyInputStream; @@ -156,7 +156,7 @@ private void testInputStreams(TestBucket bucket) throws Exception { int readBlockLength = 0; for (BlockExtendedInputStream stream : blockStreams) { - NewBlockInputStream blockStream = (NewBlockInputStream) stream; + StreamBlockInput blockStream = (StreamBlockInput) stream; int blockStreamLength = Math.min(BLOCK_SIZE, dataLength - readBlockLength); assertEquals(blockStreamLength, blockStream.getLength()); From 56d8ca97bf18876fd996b1e5849c1e0387b653e3 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 18 Jul 2024 23:20:10 +0800 Subject: [PATCH 035/114] revert sendCommandWithRetry --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 71 +++++++++---------- 1 file changed, 33 insertions(+), 38 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 7fdd63b42344..8b56124a6d1d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -356,7 +356,39 @@ private XceiverClientReply sendCommandWithRetry( // In case of an exception or an error, we will try to read from the // datanodes in the pipeline in a round-robin fashion. XceiverClientReply reply = new XceiverClientReply(null); - List datanodeList = getDatanodeList(request); + List datanodeList = null; + + DatanodeBlockID blockID = null; + if (request.getCmdType() == ContainerProtos.Type.GetBlock) { + blockID = request.getGetBlock().getBlockID(); + } else if (request.getCmdType() == ContainerProtos.Type.ReadChunk) { + blockID = request.getReadChunk().getBlockID(); + } else if (request.getCmdType() == ContainerProtos.Type.GetSmallFile) { + blockID = request.getGetSmallFile().getBlock().getBlockID(); + } + + if (blockID != null) { + // Check if the DN to which the GetBlock command was sent has been cached. + DatanodeDetails cachedDN = getBlockDNcache.get(blockID); + if (cachedDN != null) { + datanodeList = pipeline.getNodes(); + int getBlockDNCacheIndex = datanodeList.indexOf(cachedDN); + if (getBlockDNCacheIndex > 0) { + // Pull the Cached DN to the top of the DN list + Collections.swap(datanodeList, 0, getBlockDNCacheIndex); + } + } + } + if (datanodeList == null) { + if (topologyAwareRead) { + datanodeList = pipeline.getNodesInOrder(); + } else { + datanodeList = pipeline.getNodes(); + // Shuffle datanode list so that clients do not read in the same order + // every time. + Collections.shuffle(datanodeList); + } + } boolean allInService = datanodeList.stream() .allMatch(dn -> dn.getPersistedOpState() == NodeOperationalState.IN_SERVICE); @@ -685,41 +717,4 @@ public static Logger getLogger() { public void setTimeout(long timeout) { this.timeout = timeout; } - - private List getDatanodeList( - ContainerCommandRequestProto request) { - List datanodeList = null; - DatanodeBlockID blockID = null; - if (request.getCmdType() == ContainerProtos.Type.GetBlock) { - blockID = request.getGetBlock().getBlockID(); - } else if (request.getCmdType() == ContainerProtos.Type.ReadChunk) { - blockID = request.getReadChunk().getBlockID(); - } else if (request.getCmdType() == ContainerProtos.Type.GetSmallFile) { - blockID = request.getGetSmallFile().getBlock().getBlockID(); - } - - if (blockID != null) { - // Check if the DN to which the GetBlock command was sent has been cached. - DatanodeDetails cachedDN = getBlockDNcache.get(blockID); - if (cachedDN != null) { - datanodeList = pipeline.getNodes(); - int getBlockDNCacheIndex = datanodeList.indexOf(cachedDN); - if (getBlockDNCacheIndex > 0) { - // Pull the Cached DN to the top of the DN list - Collections.swap(datanodeList, 0, getBlockDNCacheIndex); - } - } - } - if (datanodeList == null) { - if (topologyAwareRead) { - datanodeList = pipeline.getNodesInOrder(); - } else { - datanodeList = pipeline.getNodes(); - // Shuffle datanode list so that clients do not read in the same order - // every time. - Collections.shuffle(datanodeList); - } - } - return datanodeList; - } } From 57b874c153e36e367259986f6ddad16bd676a4d9 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 19 Jul 2024 04:01:50 +0800 Subject: [PATCH 036/114] make strea block configurable --- .../hadoop/hdds/scm/OzoneClientConfig.java | 15 ++ .../io/BlockInputStreamFactoryImpl.java | 7 +- .../scm/storage/TestStreamBlockInput.java | 1 + .../io/TestBlockInputStreamFactoryImpl.java | 15 +- .../scm/storage/ContainerProtocolCalls.java | 2 +- .../client/rpc/read/TestChunkInputStream.java | 249 ++++++++++++++++++ .../client/rpc/read/TestInputStreamBase.java | 3 +- .../client/rpc/read/TestKeyInputStream.java | 36 ++- ...tStream.java => TestStreamBlockInput.java} | 41 +-- .../om/TestOmContainerLocationCache.java | 101 +++++-- 10 files changed, 406 insertions(+), 64 deletions(-) create mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/{TestBlockInputStream.java => TestStreamBlockInput.java} (88%) 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 fb184e4a6eb9..653c39509475 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 @@ -128,6 +128,13 @@ public enum ChecksumCombineMode { tags = ConfigTag.CLIENT) private long streamBufferMaxSize = 32 * 1024 * 1024; + @Config(key = "stream.readblock.enable", + defaultValue = "false", + type = ConfigType.BOOLEAN, + description = "Allow ReadBlock to stream all the readChunk in one request.", + tags = ConfigTag.CLIENT) + private boolean streamReadBlock = false; + @Config(key = "max.retries", defaultValue = "5", description = "Maximum number of retries by Ozone Client on " @@ -452,4 +459,12 @@ public boolean isDatastreamPipelineMode() { public void setDatastreamPipelineMode(boolean datastreamPipelineMode) { this.datastreamPipelineMode = datastreamPipelineMode; } + + public boolean isStreamReadBlock() { + return streamReadBlock; + } + + public void setStreamReadBlock(boolean streamReadBlock) { + this.streamReadBlock = streamReadBlock; + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 135e1e949a02..f5efe693f2cd 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; @@ -86,10 +87,14 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, return new ECBlockInputStreamProxy((ECReplicationConfig)repConfig, blockInfo, xceiverFactory, refreshFunction, ecBlockStreamFactory, config); - } else { + } else if (config.isStreamReadBlock()) { return new StreamBlockInput( blockInfo.getBlockID(), blockInfo.getLength(), pipeline, token, xceiverFactory, refreshFunction, config); + } else { + return new BlockInputStream( + blockInfo.getBlockID(), blockInfo.getLength(), + pipeline, token, xceiverFactory, refreshFunction, config); } } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java index a143d4f8ee36..f93250e82c79 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java @@ -71,6 +71,7 @@ public class TestStreamBlockInput { @BeforeEach public void setup() throws Exception { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setStreamReadBlock(true); refreshFunction = mock(Function.class); blockID = new BlockID(new ContainerBlockID(1, 1)); checksum = new Checksum(ChecksumType.CRC32, BYTES_PER_CHECKSUM); diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java index c1969efdd847..dbb2689b76cd 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java @@ -29,9 +29,12 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; import java.io.IOException; @@ -50,8 +53,9 @@ public class TestBlockInputStreamFactoryImpl { private OzoneConfiguration conf = new OzoneConfiguration(); - @Test - public void testNonECGivesBlockInputStream() throws IOException { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testNonECGivesBlockInputStream(boolean streamReadBlockEnabled) throws IOException { BlockInputStreamFactory factory = new BlockInputStreamFactoryImpl(); ReplicationConfig repConfig = RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE); @@ -63,11 +67,16 @@ public void testNonECGivesBlockInputStream() throws IOException { Mockito.when(pipeline.getReplicaIndex(any(DatanodeDetails.class))).thenReturn(1); OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setChecksumVerify(true); + clientConfig.setStreamReadBlock(streamReadBlockEnabled); BlockExtendedInputStream stream = factory.create(repConfig, blockInfo, blockInfo.getPipeline(), blockInfo.getToken(), null, null, clientConfig); - assertInstanceOf(StreamBlockInput.class, stream); + if (streamReadBlockEnabled) { + assertInstanceOf(StreamBlockInput.class, stream); + } else { + assertInstanceOf(BlockInputStream.class, stream); + } assertEquals(stream.getBlockID(), blockInfo.getBlockID()); assertEquals(stream.getLength(), blockInfo.getLength()); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index e42c869adcda..a411d70726a9 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -870,7 +870,7 @@ public static ContainerProtos.StreamDataResponseProto readBlock( return tryEachDatanode(xceiverClient.getPipeline(), d -> readBlock(xceiverClient, validators, builder, d), - d -> toErrorMessage(blockID, d)); + d -> toErrorMessage(BlockID.getFromProtobuf(blockID), d)); } private static StreamDataResponseProto readBlock(XceiverClientSpi xceiverClient, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java new file mode 100644 index 000000000000..c5301ba4194f --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java @@ -0,0 +1,249 @@ +/** + * 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.read; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; +import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.io.KeyInputStream; +import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; +import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; +import org.apache.hadoop.ozone.om.TestBucket; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +/** + * Tests {@link ChunkInputStream}. + */ +class TestChunkInputStream extends TestInputStreamBase { + + /** + * Run the tests as a single test method to avoid needing a new mini-cluster + * for each test. + */ + @ContainerLayoutTestInfo.ContainerTest + void testAll(ContainerLayoutVersion layout) throws Exception { + try (MiniOzoneCluster cluster = newCluster(layout)) { + cluster.waitForClusterToBeReady(); + + try (OzoneClient client = cluster.newClient()) { + TestBucket bucket = TestBucket.newBuilder(client).build(); + + testChunkReadBuffers(bucket); + testBufferRelease(bucket); + testCloseReleasesBuffers(bucket); + } + } + } + + + /** + * Test to verify that data read from chunks is stored in a list of buffers + * with max capacity equal to the bytes per checksum. + */ + private void testChunkReadBuffers(TestBucket bucket) throws Exception { + String keyName = getNewKeyName(); + int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); + byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); + + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + + BlockInputStream block0Stream = + (BlockInputStream)keyInputStream.getPartStreams().get(0); + block0Stream.initialize(); + + ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0); + + // To read 1 byte of chunk data, ChunkInputStream should get one full + // checksum boundary worth of data from Container and store it in buffers. + chunk0Stream.read(new byte[1]); + checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 1, 0, + BYTES_PER_CHECKSUM); + + // Read > checksum boundary of data from chunk0 + int readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); + byte[] readData = readDataFromChunk(chunk0Stream, 0, readDataLen); + bucket.validateData(inputData, 0, readData); + + // The first checksum boundary size of data was already existing in the + // ChunkStream buffers. Once that data is read, the next checksum + // boundary size of data will be fetched again to read the remaining data. + // Hence there should be 1 checksum boundary size of data stored in the + // ChunkStreams buffers at the end of the read. + checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 1, 0, + BYTES_PER_CHECKSUM); + + // Seek to a position in the third checksum boundary (so that current + // buffers do not have the seeked position) and read > BYTES_PER_CHECKSUM + // bytes of data. This should result in 2 * BYTES_PER_CHECKSUM amount of + // data being read into the buffers. There should be 2 buffers in the + // stream but the the first buffer should be released after it is read + // and the second buffer should have BYTES_PER_CHECKSUM capacity. + readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); + int offset = 2 * BYTES_PER_CHECKSUM + 1; + readData = readDataFromChunk(chunk0Stream, offset, readDataLen); + bucket.validateData(inputData, offset, readData); + checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 2, 1, + BYTES_PER_CHECKSUM); + + + // Read the full chunk data -1 and verify that all chunk data is read into + // buffers. We read CHUNK_SIZE - 1 as otherwise all the buffers will be + // released once all chunk data is read. + readData = readDataFromChunk(chunk0Stream, 0, CHUNK_SIZE - 1); + bucket.validateData(inputData, 0, readData); + int expectedNumBuffers = CHUNK_SIZE / BYTES_PER_CHECKSUM; + checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), + expectedNumBuffers, expectedNumBuffers - 1, BYTES_PER_CHECKSUM); + + // Read the last byte of chunk and verify that the buffers are released. + chunk0Stream.read(new byte[1]); + assertNull(chunk0Stream.getCachedBuffers(), + "ChunkInputStream did not release buffers after reaching EOF."); + } + } + + private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { + String keyName = getNewKeyName(); + bucket.writeRandomBytes(keyName, CHUNK_SIZE); + + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + BlockInputStream block0Stream = + (BlockInputStream) keyInputStream.getPartStreams().get(0); + block0Stream.initialize(); + + ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0); + readDataFromChunk(chunk0Stream, 0, 1); + assertNotNull(chunk0Stream.getCachedBuffers()); + + chunk0Stream.close(); + + assertNull(chunk0Stream.getCachedBuffers()); + } + } + + /** + * Test that ChunkInputStream buffers are released as soon as the last byte + * of the buffer is read. + */ + private void testBufferRelease(TestBucket bucket) throws Exception { + String keyName = getNewKeyName(); + byte[] inputData = bucket.writeRandomBytes(keyName, CHUNK_SIZE); + + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + + BlockInputStream block0Stream = + (BlockInputStream)keyInputStream.getPartStreams().get(0); + block0Stream.initialize(); + + ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0); + + // Read checksum boundary - 1 bytes of data + int readDataLen = BYTES_PER_CHECKSUM - 1; + byte[] readData = readDataFromChunk(chunk0Stream, 0, readDataLen); + bucket.validateData(inputData, 0, readData); + + // There should be 1 byte of data remaining in the buffer which is not + // yet read. Hence, the buffer should not be released. + checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), + 1, 0, BYTES_PER_CHECKSUM); + assertEquals(1, chunk0Stream.getCachedBuffers()[0].remaining()); + + // Reading the last byte in the buffer should result in all the buffers + // being released. + readData = readDataFromChunk(chunk0Stream, 1); + bucket.validateData(inputData, readDataLen, readData); + assertNull(chunk0Stream.getCachedBuffers(), + "Chunk stream buffers not released after last byte is read"); + + // Read more data to get the data till the next checksum boundary. + readDataLen = BYTES_PER_CHECKSUM / 2; + readData = readDataFromChunk(chunk0Stream, readDataLen); + // There should be one buffer and the buffer should not be released as + // there is data pending to be read from the buffer + checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 1, 0, + BYTES_PER_CHECKSUM); + ByteBuffer lastCachedBuffer = chunk0Stream.getCachedBuffers()[0]; + assertEquals(BYTES_PER_CHECKSUM - readDataLen, + lastCachedBuffer.remaining()); + + // Read more than the remaining data in buffer (but less than the next + // checksum boundary). + int position = (int) chunk0Stream.getPos(); + readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; + readData = readDataFromChunk(chunk0Stream, readDataLen); + bucket.validateData(inputData, position, readData); + // After reading the remaining data in the buffer, the buffer should be + // released and next checksum size of data must be read into the buffers + checkBufferSizeAndCapacity(chunk0Stream.getCachedBuffers(), 1, 0, + BYTES_PER_CHECKSUM); + // Verify that the previously cached buffer is released by comparing it + // with the current cached buffer + assertNotEquals(lastCachedBuffer, + chunk0Stream.getCachedBuffers()[0]); + } + } + + private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, + int offset, int readDataLength) throws IOException { + byte[] readData = new byte[readDataLength]; + chunkInputStream.seek(offset); + chunkInputStream.read(readData, 0, readDataLength); + return readData; + } + + private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, + int readDataLength) throws IOException { + byte[] readData = new byte[readDataLength]; + chunkInputStream.read(readData, 0, readDataLength); + return readData; + } + + /** + * Verify number of buffers and their capacities. + * @param buffers chunk stream buffers + * @param expectedNumBuffers expected number of buffers + * @param numReleasedBuffers first numReleasedBuffers are expected to + * be released and hence null + * @param expectedBufferCapacity expected buffer capacity of unreleased + * buffers + */ + private void checkBufferSizeAndCapacity(ByteBuffer[] buffers, + int expectedNumBuffers, int numReleasedBuffers, + long expectedBufferCapacity) { + assertEquals(expectedNumBuffers, buffers.length, + "ChunkInputStream does not have expected number of " + + "ByteBuffers"); + for (int i = 0; i < buffers.length; i++) { + if (i <= numReleasedBuffers - 1) { + // This buffer should have been released and hence null + assertNull(buffers[i], + "ChunkInputStream Buffer not released after being read"); + } else { + assertEquals(expectedBufferCapacity, buffers[i].capacity(), + "ChunkInputStream ByteBuffer capacity is wrong"); + } + } + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java index 256148dfb8de..bc50aaf7e94e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java @@ -41,10 +41,11 @@ abstract class TestInputStreamBase { static final int MAX_FLUSH_SIZE = 2 * FLUSH_SIZE; // 4MB static final int BLOCK_SIZE = 2 * MAX_FLUSH_SIZE; // 8MB static final int BYTES_PER_CHECKSUM = 256 * 1024; // 256KB + static OzoneConfiguration conf; protected static MiniOzoneCluster newCluster( ContainerLayoutVersion containerLayout) throws Exception { - OzoneConfiguration conf = new OzoneConfiguration(); + conf = new OzoneConfiguration(); OzoneClientConfig config = conf.getObject(OzoneClientConfig.class); config.setBytesPerChecksum(BYTES_PER_CHECKSUM); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java index 6d79977b5260..3ab8ae31188c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java @@ -29,11 +29,11 @@ import org.apache.hadoop.hdds.client.RatisReplicationConfig; 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.Type; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientMetrics; import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; -import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; +import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.io.KeyInputStream; @@ -156,10 +156,26 @@ private void testInputStreams(TestBucket bucket) throws Exception { int readBlockLength = 0; for (BlockExtendedInputStream stream : blockStreams) { - StreamBlockInput blockStream = (StreamBlockInput) stream; + BlockInputStream blockStream = (BlockInputStream) stream; int blockStreamLength = Math.min(BLOCK_SIZE, dataLength - readBlockLength); assertEquals(blockStreamLength, blockStream.getLength()); + + int expectedNumChunkStreams = + BufferUtils.getNumberOfBins(blockStreamLength, CHUNK_SIZE); + blockStream.initialize(); + List chunkStreams = blockStream.getChunkStreams(); + assertEquals(expectedNumChunkStreams, chunkStreams.size()); + + int readChunkLength = 0; + for (ChunkInputStream chunkStream : chunkStreams) { + int chunkStreamLength = Math.min(CHUNK_SIZE, + blockStreamLength - readChunkLength); + assertEquals(chunkStreamLength, chunkStream.getRemaining()); + + readChunkLength += chunkStreamLength; + } + readBlockLength += blockStreamLength; } } @@ -232,7 +248,7 @@ public void testSeek(TestBucket bucket) throws Exception { long writeChunkCount = metrics.getContainerOpCountMetrics( ContainerProtos.Type.WriteChunk); long readChunkCount = metrics.getContainerOpCountMetrics( - Type.ReadBlock); + ContainerProtos.Type.ReadChunk); String keyName = getNewKeyName(); // write data spanning 3 chunks @@ -251,15 +267,15 @@ public void testSeek(TestBucket bucket) throws Exception { // Seek operation should not result in any readChunk operation. assertEquals(readChunkCount, metrics - .getContainerOpCountMetrics(Type.ReadBlock)); + .getContainerOpCountMetrics(ContainerProtos.Type.ReadChunk)); byte[] readData = new byte[CHUNK_SIZE]; keyInputStream.read(readData, 0, CHUNK_SIZE); // Since we read data from index 150 to 250 and the chunk boundary is // 100 bytes, we need to read 2 chunks. - assertEquals(readChunkCount + 1, - metrics.getContainerOpCountMetrics(Type.ReadBlock)); + assertEquals(readChunkCount + 2, + metrics.getContainerOpCountMetrics(ContainerProtos.Type.ReadChunk)); keyInputStream.close(); @@ -344,15 +360,15 @@ private void testSkip(TestBucket bucket) throws Exception { // Skip operation should not result in any readChunk operation. assertEquals(readChunkCount, metrics - .getContainerOpCountMetrics(Type.ReadBlock)); + .getContainerOpCountMetrics(ContainerProtos.Type.ReadChunk)); byte[] readData = new byte[CHUNK_SIZE]; keyInputStream.read(readData, 0, CHUNK_SIZE); // Since we reading data from index 150 to 250 and the chunk boundary is // 100 bytes, we need to read 2 chunks. - assertEquals(readChunkCount + 1, - metrics.getContainerOpCountMetrics(Type.ReadBlock)); + assertEquals(readChunkCount + 2, + metrics.getContainerOpCountMetrics(ContainerProtos.Type.ReadChunk)); keyInputStream.close(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java similarity index 88% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java index 09eb4f81d672..9c563c1df9f0 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java @@ -20,7 +20,8 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; @@ -34,9 +35,9 @@ import static org.junit.jupiter.api.Assertions.assertNull; /** - * Tests {@link ChunkInputStream}. + * Tests {@link StreamBlockInput}. */ -class TestBlockInputStream extends TestInputStreamBase { +class TestStreamBlockInput extends TestInputStreamBase { /** * Run the tests as a single test method to avoid needing a new mini-cluster @@ -46,7 +47,7 @@ class TestBlockInputStream extends TestInputStreamBase { void testAll(ContainerLayoutVersion layout) throws Exception { try (MiniOzoneCluster cluster = newCluster(layout)) { cluster.waitForClusterToBeReady(); - + conf.setBoolean("ozone.client.stream.readblock.enable", true); try (OzoneClient client = cluster.newClient()) { TestBucket bucket = TestBucket.newBuilder(client).build(); @@ -72,7 +73,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { StreamBlockInput block0Stream = (StreamBlockInput)keyInputStream.getPartStreams().get(0); - // To read 1 byte of chunk data, ChunkInputStream should get one full + // To read 1 byte of chunk data, StreamBlockInput should get one full // checksum boundary worth of data from Container and store it in buffers. block0Stream.read(new byte[1]); checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, @@ -80,14 +81,14 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // Read > checksum boundary of data from chunk0 int readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); - byte[] readData = readDataFromChunk(block0Stream, 0, readDataLen); + byte[] readData = readDataFromBlock(block0Stream, 0, readDataLen); bucket.validateData(inputData, 0, readData); // The first checksum boundary size of data was already existing in the - // ChunkStream buffers. Once that data is read, the next checksum + // StreamBlockInput buffers. Once that data is read, the next checksum // boundary size of data will be fetched again to read the remaining data. // Hence there should be 1 checksum boundary size of data stored in the - // ChunkStreams buffers at the end of the read. + // StreamBlockInput buffers at the end of the read. checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, BYTES_PER_CHECKSUM); @@ -99,7 +100,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // and the second buffer should have BYTES_PER_CHECKSUM capacity. readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); int offset = 2 * BYTES_PER_CHECKSUM + 1; - readData = readDataFromChunk(block0Stream, offset, readDataLen); + readData = readDataFromBlock(block0Stream, offset, readDataLen); bucket.validateData(inputData, offset, readData); checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 2, 1, BYTES_PER_CHECKSUM); @@ -108,7 +109,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // Read the full chunk data -1 and verify that all chunk data is read into // buffers. We read CHUNK_SIZE - 1 as otherwise all the buffers will be // released once all chunk data is read. - readData = readDataFromChunk(block0Stream, 0, CHUNK_SIZE - 1); + readData = readDataFromBlock(block0Stream, 0, CHUNK_SIZE - 1); bucket.validateData(inputData, 0, readData); int expectedNumBuffers = CHUNK_SIZE / BYTES_PER_CHECKSUM; checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), @@ -129,7 +130,7 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { StreamBlockInput block0Stream = (StreamBlockInput) keyInputStream.getPartStreams().get(0); - readDataFromChunk(block0Stream, 0, 1); + readDataFromBlock(block0Stream, 0, 1); assertNotNull(block0Stream.getCachedBuffers()); block0Stream.close(); @@ -139,7 +140,7 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { } /** - * Test that ChunkInputStream buffers are released as soon as the last byte + * Test that StreamBlockInput buffers are released as soon as the last byte * of the buffer is read. */ private void testBufferRelease(TestBucket bucket) throws Exception { @@ -153,7 +154,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // Read checksum boundary - 1 bytes of data int readDataLen = BYTES_PER_CHECKSUM - 1; - byte[] readData = readDataFromChunk(block0Stream, 0, readDataLen); + byte[] readData = readDataFromBlock(block0Stream, 0, readDataLen); bucket.validateData(inputData, 0, readData); // There should be 1 byte of data remaining in the buffer which is not @@ -164,14 +165,14 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // Reading the last byte in the buffer should result in all the buffers // being released. - readData = readDataFromChunk(block0Stream, 1); + readData = readDataFromBlock(block0Stream, 1); bucket.validateData(inputData, readDataLen, readData); assertNull(block0Stream.getCachedBuffers(), - "Chunk stream buffers not released after last byte is read"); + "Stream block buffers not released after last byte is read"); // Read more data to get the data till the next checksum boundary. readDataLen = BYTES_PER_CHECKSUM / 2; - readData = readDataFromChunk(block0Stream, readDataLen); + readData = readDataFromBlock(block0Stream, readDataLen); // There should be one buffer and the buffer should not be released as // there is data pending to be read from the buffer checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, @@ -184,7 +185,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // checksum boundary). int position = (int) block0Stream.getPos(); readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; - readData = readDataFromChunk(block0Stream, readDataLen); + readData = readDataFromBlock(block0Stream, readDataLen); bucket.validateData(inputData, position, readData); // After reading the remaining data in the buffer, the buffer should be // released and next checksum size of data must be read into the buffers @@ -197,7 +198,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { } } - private byte[] readDataFromChunk(StreamBlockInput blockInputStream, + private byte[] readDataFromBlock(StreamBlockInput blockInputStream, int offset, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; blockInputStream.seek(offset); @@ -205,7 +206,7 @@ private byte[] readDataFromChunk(StreamBlockInput blockInputStream, return readData; } - private byte[] readDataFromChunk(StreamBlockInput blockInputStream, + private byte[] readDataFromBlock(StreamBlockInput blockInputStream, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; blockInputStream.read(readData, 0, readDataLength); @@ -214,7 +215,7 @@ private byte[] readDataFromChunk(StreamBlockInput blockInputStream, /** * Verify number of buffers and their capacities. - * @param buffers chunk stream buffers + * @param buffers block stream buffers * @param expectedNumBuffers expected number of buffers * @param numReleasedBuffers first numReleasedBuffers are expected to * be released and hence null diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java index 3139b573024f..2ae69dc3c96f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java @@ -28,15 +28,16 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; 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.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetCommittedBlockLengthResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; @@ -291,7 +292,8 @@ public void containerCachedInHappyCase() throws Exception { verify(mockScmContainerClient, times(1)) .getContainerWithPipelineBatch(newHashSet(CONTAINER_ID.get())); - mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadChunk(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); try (InputStream is = key1.getContent()) { byte[] read = new byte[(int) key1.getDataSize()]; IOUtils.read(is, read); @@ -365,8 +367,11 @@ public void containerRefreshedAfterDatanodeGetBlockError( try (InputStream is = key1.getContent()) { // Simulate dn1 got errors, and the container's moved to dn2. + mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, + dnException, dnResponseCode); mockScmGetContainerPipeline(CONTAINER_ID.get(), DN2); - mockReadBlock(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockGetBlock(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadChunk(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); byte[] read = new byte[(int) key1.getDataSize()]; IOUtils.read(is, read); @@ -410,10 +415,12 @@ public void containerRefreshedAfterDatanodeReadChunkError( try (InputStream is = key1.getContent()) { // simulate dn1 goes down, the container's to dn2. - mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, + mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadChunk(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, dnException, dnResponseCode); mockScmGetContainerPipeline(CONTAINER_ID.get(), DN2); - mockReadBlock(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockGetBlock(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadChunk(mockDn2Protocol, CONTAINER_ID.get(), 1L, data, null, null); byte[] read = new byte[(int) key1.getDataSize()]; IOUtils.read(is, read); @@ -457,7 +464,7 @@ public void containerNotRefreshedAfterDatanodeGetBlockError( try (InputStream is = key1.getContent()) { // simulate dn1 got errors, and the container's moved to dn2. - mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, ex, + mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, ex, errorCode); assertThrows(expectedEx, @@ -501,7 +508,8 @@ public void containerNotRefreshedAfterDatanodeReadChunkError( try (InputStream is = key1.getContent()) { // simulate dn1 got errors, and the container's moved to dn2. - mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, + mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadChunk(mockDn1Protocol, CONTAINER_ID.get(), 1L, null, dnException, dnResponseCode); assertThrows(expectedEx, @@ -553,7 +561,8 @@ public void containerRefreshedOnEmptyPipelines() throws Exception { // but the empty pipeline is not cached, and when some data node is back. mockScmGetContainerPipeline(CONTAINER_ID.get(), DN1); - mockReadBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockGetBlock(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); + mockReadChunk(mockDn1Protocol, CONTAINER_ID.get(), 1L, data, null, null); // the subsequent effort to read the key is success. OzoneKeyDetails updatedKey1 = bucket.getKey(keyName); try (InputStream is = updatedKey1.getContent()) { @@ -659,42 +668,41 @@ private void mockScmGetContainerPipelineEmpty(long containerId) newHashSet(containerId))).thenReturn(containerWithPipelines); } - private void mockReadBlock(XceiverClientGrpc mockDnProtocol, - long containerId, long localId, - byte[] data, - Exception exception, - Result errorCode) throws Exception { + private void mockGetBlock(XceiverClientGrpc mockDnProtocol, + long containerId, long localId, + byte[] data, + Exception exception, + Result errorCode) throws Exception { + final CompletableFuture response; if (exception != null) { response = new CompletableFuture<>(); response.completeExceptionally(exception); } else if (errorCode != null) { - ContainerCommandResponseProto readBlock = + ContainerCommandResponseProto getBlockResp = ContainerCommandResponseProto.newBuilder() .setResult(errorCode) - .setCmdType(Type.ReadBlock) + .setCmdType(Type.GetBlock) .build(); - response = completedFuture(readBlock); + response = completedFuture(getBlockResp); } else { - ContainerCommandResponseProto readBlock = + ContainerCommandResponseProto getBlockResp = ContainerCommandResponseProto.newBuilder() - .setStreamData(StreamDataResponseProto.newBuilder() - .addReadBlock(ReadBlockResponseProto.newBuilder() + .setGetBlock(GetBlockResponseProto.newBuilder() + .setBlockData(BlockData.newBuilder() + .addChunks(createChunkInfo(data)) .setBlockID(createBlockId(containerId, localId)) - .setChunkData(createChunkInfo(data)) - .setData(ByteString.copyFrom(data)) .build()) - .build()) + .build() + ) .setResult(Result.SUCCESS) - .setCmdType(Type.StreamRead) + .setCmdType(Type.GetBlock) .build(); - response = completedFuture(readBlock); + response = completedFuture(getBlockResp); } - doAnswer(invocation -> new XceiverClientReply(response)) .when(mockDnProtocol) - .sendCommandAsyncReadOnly(argThat(matchCmd(Type.ReadBlock)), any()); - + .sendCommandAsync(argThat(matchCmd(Type.GetBlock)), any()); } @Nonnull @@ -708,6 +716,43 @@ private ChunkInfo createChunkInfo(byte[] data) throws Exception { .build(); } + private void mockReadChunk(XceiverClientGrpc mockDnProtocol, + long containerId, long localId, + byte[] data, + Exception exception, + Result errorCode) throws Exception { + final CompletableFuture response; + if (exception != null) { + response = new CompletableFuture<>(); + response.completeExceptionally(exception); + } else if (errorCode != null) { + ContainerCommandResponseProto readChunkResp = + ContainerCommandResponseProto.newBuilder() + .setResult(errorCode) + .setCmdType(Type.ReadChunk) + .build(); + response = completedFuture(readChunkResp); + } else { + ContainerCommandResponseProto readChunkResp = + ContainerCommandResponseProto.newBuilder() + .setReadChunk(ReadChunkResponseProto.newBuilder() + .setBlockID(createBlockId(containerId, localId)) + .setChunkData(createChunkInfo(data)) + .setData(ByteString.copyFrom(data)) + .build() + ) + .setResult(Result.SUCCESS) + .setCmdType(Type.ReadChunk) + .build(); + response = completedFuture(readChunkResp); + } + + doAnswer(invocation -> new XceiverClientReply(response)) + .when(mockDnProtocol) + .sendCommandAsync(argThat(matchCmd(Type.ReadChunk)), any()); + + } + private static Pipeline createPipeline(DatanodeDetails dn) { return createPipeline(Collections.singletonList(dn)); } From 7b98108a2a421cf1a53cef9d0845b5d7be41c392 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 19 Jul 2024 04:09:21 +0800 Subject: [PATCH 037/114] fix checkstyle --- .../hadoop/ozone/client/rpc/read/TestInputStreamBase.java | 2 +- .../hadoop/ozone/client/rpc/read/TestStreamBlockInput.java | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java index bc50aaf7e94e..ab601793bd10 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java @@ -41,7 +41,7 @@ abstract class TestInputStreamBase { static final int MAX_FLUSH_SIZE = 2 * FLUSH_SIZE; // 4MB static final int BLOCK_SIZE = 2 * MAX_FLUSH_SIZE; // 8MB static final int BYTES_PER_CHECKSUM = 256 * 1024; // 256KB - static OzoneConfiguration conf; + protected static OzoneConfiguration conf; protected static MiniOzoneCluster newCluster( ContainerLayoutVersion containerLayout) throws Exception { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java index 9c563c1df9f0..908315dd39ac 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java @@ -20,8 +20,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; From db82e3bedeba5c07c6fa11dc7cb6af180f325447 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 19 Jul 2024 04:17:04 +0800 Subject: [PATCH 038/114] remove TestStreamBlockInput --- .../client/rpc/read/TestInputStreamBase.java | 3 +- .../client/rpc/read/TestStreamBlockInput.java | 235 ------------------ 2 files changed, 1 insertion(+), 237 deletions(-) delete mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java index ab601793bd10..256148dfb8de 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java @@ -41,11 +41,10 @@ abstract class TestInputStreamBase { static final int MAX_FLUSH_SIZE = 2 * FLUSH_SIZE; // 4MB static final int BLOCK_SIZE = 2 * MAX_FLUSH_SIZE; // 8MB static final int BYTES_PER_CHECKSUM = 256 * 1024; // 256KB - protected static OzoneConfiguration conf; protected static MiniOzoneCluster newCluster( ContainerLayoutVersion containerLayout) throws Exception { - conf = new OzoneConfiguration(); + OzoneConfiguration conf = new OzoneConfiguration(); OzoneClientConfig config = conf.getObject(OzoneClientConfig.class); config.setBytesPerChecksum(BYTES_PER_CHECKSUM); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java deleted file mode 100644 index 908315dd39ac..000000000000 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java +++ /dev/null @@ -1,235 +0,0 @@ -/** - * 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.read; - -import java.io.IOException; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; -import org.apache.hadoop.ozone.MiniOzoneCluster; -import org.apache.hadoop.ozone.client.OzoneClient; -import org.apache.hadoop.ozone.client.io.KeyInputStream; -import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; -import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; -import org.apache.hadoop.ozone.om.TestBucket; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; - -/** - * Tests {@link StreamBlockInput}. - */ -class TestStreamBlockInput extends TestInputStreamBase { - - /** - * Run the tests as a single test method to avoid needing a new mini-cluster - * for each test. - */ - @ContainerLayoutTestInfo.ContainerTest - void testAll(ContainerLayoutVersion layout) throws Exception { - try (MiniOzoneCluster cluster = newCluster(layout)) { - cluster.waitForClusterToBeReady(); - conf.setBoolean("ozone.client.stream.readblock.enable", true); - try (OzoneClient client = cluster.newClient()) { - TestBucket bucket = TestBucket.newBuilder(client).build(); - - testBlockReadBuffers(bucket); - testBufferRelease(bucket); - testCloseReleasesBuffers(bucket); - } - } - } - - - /** - * Test to verify that data read from chunks is stored in a list of buffers - * with max capacity equal to the bytes per checksum. - */ - private void testBlockReadBuffers(TestBucket bucket) throws Exception { - String keyName = getNewKeyName(); - int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); - byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); - - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - - StreamBlockInput block0Stream = - (StreamBlockInput)keyInputStream.getPartStreams().get(0); - - // To read 1 byte of chunk data, StreamBlockInput should get one full - // checksum boundary worth of data from Container and store it in buffers. - block0Stream.read(new byte[1]); - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, - BYTES_PER_CHECKSUM); - - // Read > checksum boundary of data from chunk0 - int readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); - byte[] readData = readDataFromBlock(block0Stream, 0, readDataLen); - bucket.validateData(inputData, 0, readData); - - // The first checksum boundary size of data was already existing in the - // StreamBlockInput buffers. Once that data is read, the next checksum - // boundary size of data will be fetched again to read the remaining data. - // Hence there should be 1 checksum boundary size of data stored in the - // StreamBlockInput buffers at the end of the read. - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, - BYTES_PER_CHECKSUM); - - // Seek to a position in the third checksum boundary (so that current - // buffers do not have the seeked position) and read > BYTES_PER_CHECKSUM - // bytes of data. This should result in 2 * BYTES_PER_CHECKSUM amount of - // data being read into the buffers. There should be 2 buffers in the - // stream but the the first buffer should be released after it is read - // and the second buffer should have BYTES_PER_CHECKSUM capacity. - readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); - int offset = 2 * BYTES_PER_CHECKSUM + 1; - readData = readDataFromBlock(block0Stream, offset, readDataLen); - bucket.validateData(inputData, offset, readData); - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 2, 1, - BYTES_PER_CHECKSUM); - - - // Read the full chunk data -1 and verify that all chunk data is read into - // buffers. We read CHUNK_SIZE - 1 as otherwise all the buffers will be - // released once all chunk data is read. - readData = readDataFromBlock(block0Stream, 0, CHUNK_SIZE - 1); - bucket.validateData(inputData, 0, readData); - int expectedNumBuffers = CHUNK_SIZE / BYTES_PER_CHECKSUM; - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), - expectedNumBuffers, expectedNumBuffers - 1, BYTES_PER_CHECKSUM); - - // Read the last byte of chunk and verify that the buffers are released. - block0Stream.read(new byte[1]); - assertNull(block0Stream.getCachedBuffers(), - "ChunkInputStream did not release buffers after reaching EOF."); - } - } - - private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { - String keyName = getNewKeyName(); - bucket.writeRandomBytes(keyName, CHUNK_SIZE); - - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - StreamBlockInput block0Stream = - (StreamBlockInput) keyInputStream.getPartStreams().get(0); - - readDataFromBlock(block0Stream, 0, 1); - assertNotNull(block0Stream.getCachedBuffers()); - - block0Stream.close(); - - assertNull(block0Stream.getCachedBuffers()); - } - } - - /** - * Test that StreamBlockInput buffers are released as soon as the last byte - * of the buffer is read. - */ - private void testBufferRelease(TestBucket bucket) throws Exception { - String keyName = getNewKeyName(); - byte[] inputData = bucket.writeRandomBytes(keyName, CHUNK_SIZE); - - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - - StreamBlockInput block0Stream = - (StreamBlockInput)keyInputStream.getPartStreams().get(0); - - // Read checksum boundary - 1 bytes of data - int readDataLen = BYTES_PER_CHECKSUM - 1; - byte[] readData = readDataFromBlock(block0Stream, 0, readDataLen); - bucket.validateData(inputData, 0, readData); - - // There should be 1 byte of data remaining in the buffer which is not - // yet read. Hence, the buffer should not be released. - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), - 1, 0, BYTES_PER_CHECKSUM); - assertEquals(1, block0Stream.getCachedBuffers()[0].remaining()); - - // Reading the last byte in the buffer should result in all the buffers - // being released. - readData = readDataFromBlock(block0Stream, 1); - bucket.validateData(inputData, readDataLen, readData); - assertNull(block0Stream.getCachedBuffers(), - "Stream block buffers not released after last byte is read"); - - // Read more data to get the data till the next checksum boundary. - readDataLen = BYTES_PER_CHECKSUM / 2; - readData = readDataFromBlock(block0Stream, readDataLen); - // There should be one buffer and the buffer should not be released as - // there is data pending to be read from the buffer - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, - BYTES_PER_CHECKSUM); - ByteBuffer lastCachedBuffer = block0Stream.getCachedBuffers()[0]; - assertEquals(BYTES_PER_CHECKSUM - readDataLen, - lastCachedBuffer.remaining()); - - // Read more than the remaining data in buffer (but less than the next - // checksum boundary). - int position = (int) block0Stream.getPos(); - readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; - readData = readDataFromBlock(block0Stream, readDataLen); - bucket.validateData(inputData, position, readData); - // After reading the remaining data in the buffer, the buffer should be - // released and next checksum size of data must be read into the buffers - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, 0, - BYTES_PER_CHECKSUM); - // Verify that the previously cached buffer is released by comparing it - // with the current cached buffer - assertNotEquals(lastCachedBuffer, - block0Stream.getCachedBuffers()[0]); - } - } - - private byte[] readDataFromBlock(StreamBlockInput blockInputStream, - int offset, int readDataLength) throws IOException { - byte[] readData = new byte[readDataLength]; - blockInputStream.seek(offset); - blockInputStream.read(readData, 0, readDataLength); - return readData; - } - - private byte[] readDataFromBlock(StreamBlockInput blockInputStream, - int readDataLength) throws IOException { - byte[] readData = new byte[readDataLength]; - blockInputStream.read(readData, 0, readDataLength); - return readData; - } - - /** - * Verify number of buffers and their capacities. - * @param buffers block stream buffers - * @param expectedNumBuffers expected number of buffers - * @param numReleasedBuffers first numReleasedBuffers are expected to - * be released and hence null - * @param expectedBufferCapacity expected buffer capacity of unreleased - * buffers - */ - private void checkBufferSizeAndCapacity(ByteBuffer[] buffers, - int expectedNumBuffers, int numReleasedBuffers, - long expectedBufferCapacity) { - assertEquals(expectedNumBuffers - numReleasedBuffers, buffers.length, - "BlockInputStream does not have expected number of " + - "ByteBuffers"); - for (int i = 0; i < buffers.length; i++) { - assertEquals(expectedBufferCapacity, buffers[i].capacity(), - "BlockInputStream ByteBuffer capacity is wrong"); - - } - } -} From 158d2e4c3b1084db1e018751801ecf6ce1d37f28 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 19 Jul 2024 04:21:25 +0800 Subject: [PATCH 039/114] fix checkstyle --- .../container/common/transport/server/ratis/CSMMetrics.java | 1 - .../common/transport/server/ratis/DispatcherContext.java | 1 - 2 files changed, 2 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java index a0da62e54281..3634ae34ac8d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java @@ -98,7 +98,6 @@ public String getRaftGroupId() { } public void incNumWriteStateMachineOps() { - numWriteStateMachineOps.incr(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java index 603a3d82fb87..2b2947a52276 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java @@ -34,7 +34,6 @@ public final class DispatcherContext { private static final DispatcherContext HANDLE_READ_CHUNK = newBuilder(Op.HANDLE_READ_CHUNK).build(); - private static final DispatcherContext HANDLE_READ_BLOCK = newBuilder(Op.HANDLE_READ_BLOCK).build(); private static final DispatcherContext HANDLE_WRITE_CHUNK From 4b5e14c3f74712a94622cad474b51a136f0b9ac9 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 5 Sep 2024 02:44:03 +0800 Subject: [PATCH 040/114] Merge branch 'master' into HDDS-10338 --- .../main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java | 1 + 1 file changed, 1 insertion(+) 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 0ccdb6918f80..7ee36ba249ab 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 @@ -507,6 +507,7 @@ public void setMaxConcurrentWritePerKey(int maxConcurrentWritePerKey) { public int getMaxConcurrentWritePerKey() { return this.maxConcurrentWritePerKey; + } public boolean isStreamReadBlock() { return streamReadBlock; From 0d0e31c6c5272675ccd2e2d78567234fa5b986f9 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 5 Sep 2024 17:07:04 +0800 Subject: [PATCH 041/114] Merge branch 'master' into HDDS-10338 --- .../ozone/container/common/impl/HddsDispatcher.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 61faee34fd70..768ad84f362e 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 @@ -864,18 +864,18 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, long oPLatencyMS = Time.monotonicNow() - startTime; metrics.incContainerOpsLatencies(cmdType, oPLatencyMS); if (responseProto == null) { - audit(action, eventType, msg, AuditEventStatus.SUCCESS, null); + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.SUCCESS, null); } else { OnDemandContainerDataScanner.scanContainer(container); - audit(action, eventType, msg, AuditEventStatus.FAILURE, + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, new Exception(responseProto.getMessage())); streamObserver.onNext(responseProto); } perf.appendOpLatencyMs(oPLatencyMS); - performanceAudit(action, msg, perf, oPLatencyMS); + performanceAudit(action, msg, dispatcherContext, perf, oPLatencyMS); } catch (StorageContainerException sce) { - audit(action, eventType, msg, AuditEventStatus.FAILURE, sce); + audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, sce); streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, sce, msg)); } catch (IOException ioe) { final String s = ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED From 3cefe36d91adea10b4b239d92420f4298e04ee67 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 25 Sep 2024 17:35:42 +0800 Subject: [PATCH 042/114] update for compabatbilities and add some tests --- .../hadoop/hdds/scm/OzoneClientConfig.java | 2 +- .../hdds/scm/storage/StreamBlockInput.java | 49 ++-- .../io/BlockInputStreamFactoryImpl.java | 18 +- .../scm/storage/DummyStreamBlockInput.java | 2 +- .../apache/hadoop/hdds/DatanodeVersion.java | 2 + .../scm/storage/ContainerProtocolCalls.java | 34 +-- .../container/keyvalue/KeyValueHandler.java | 3 + .../apache/hadoop/ozone/MiniOzoneCluster.java | 2 +- .../client/rpc/read/TestInputStreamBase.java | 5 + .../client/rpc/read/TestStreamBlockInput.java | 237 ++++++++++++++++++ 10 files changed, 316 insertions(+), 38 deletions(-) create mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java 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 328b5101a919..738df5a62b15 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 @@ -133,7 +133,7 @@ public enum ChecksumCombineMode { type = ConfigType.BOOLEAN, description = "Allow ReadBlock to stream all the readChunk in one request.", tags = ConfigTag.CLIENT) - private boolean streamReadBlock = false; + private boolean streamReadBlock = true; @Config(key = "max.retries", defaultValue = "5", diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 40c736ad13dd..d6c85742242b 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -53,6 +52,7 @@ import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; +import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -99,7 +99,7 @@ public StreamBlockInput( Token token, XceiverClientFactory xceiverClientFactory, Function refreshFunction, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { this.blockID = blockID; this.length = length; setPipeline(pipeline); @@ -380,10 +380,16 @@ public synchronized void unbuffer() { releaseBuffers(); } - private void setPipeline(Pipeline pipeline) { + private void setPipeline(Pipeline pipeline) throws IOException { if (pipeline == null) { return; } + long replicaIndexes = pipeline.getNodes().stream().mapToInt(pipeline::getReplicaIndex).distinct().count(); + + if (replicaIndexes > 1) { + throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", + pipeline)); + } // irrespective of the container state, we will always read via Standalone // protocol. @@ -405,8 +411,14 @@ protected synchronized void checkOpen() throws IOException { protected synchronized void acquireClient() throws IOException { if (xceiverClientFactory != null && xceiverClient == null) { - xceiverClient = xceiverClientFactory.acquireClientForReadData( - pipelineRef.get()); + 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; + } } } @@ -547,19 +559,11 @@ private synchronized void readDataFromContainer(int len) throws IOException { @VisibleForTesting protected long readData(long startByteIndex, long len) throws IOException { - Pipeline pipeline = xceiverClient.getPipeline(); + Pipeline pipeline = pipelineRef.get(); buffers = new ArrayList<>(); - DatanodeBlockID.Builder blockBuilder = DatanodeBlockID - .newBuilder().setContainerID(blockID.getContainerID()) - .setLocalID(blockID.getLocalID()) - .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId()); - int replicaIndex = pipeline.getReplicaIndex(pipeline.getClosestNode()); - if (replicaIndex > 0) { - blockBuilder.setReplicaIndex(replicaIndex); - } StreamDataResponseProto response = ContainerProtocolCalls.readBlock(xceiverClient, startByteIndex, - len, blockBuilder.build(), validators, tokenRef.get(), verifyChecksum); + len, blockID, validators, tokenRef.get(), pipeline.getReplicaIndexes(), verifyChecksum); List readBlocks = response.getReadBlockList(); for (ReadBlockResponseProto readBlock : readBlocks) { @@ -747,18 +751,25 @@ public boolean isVerifyChecksum() { } private void refreshBlockInfo(IOException cause) throws IOException { - LOG.info("Unable to read information for block {} from pipeline {}: {}", + LOG.info("Attempting to update pipeline and block token for block {} from pipeline {}: {}", blockID, pipelineRef.get().getId(), cause.getMessage()); if (refreshFunction != null) { LOG.debug("Re-fetching pipeline and block token for block {}", blockID); BlockLocationInfo blockLocationInfo = refreshFunction.apply(blockID); if (blockLocationInfo == null) { - LOG.debug("No new block location info for block {}", blockID); + LOG.warn("No new block location info for block {}", blockID); } else { - LOG.debug("New pipeline for block {}: {}", blockID, - blockLocationInfo.getPipeline()); setPipeline(blockLocationInfo.getPipeline()); + LOG.info("New pipeline for block {}: {}", blockID, + blockLocationInfo.getPipeline()); + tokenRef.set(blockLocationInfo.getToken()); + if (blockLocationInfo.getToken() != null) { + OzoneBlockTokenIdentifier tokenId = new OzoneBlockTokenIdentifier(); + tokenId.readFromByteArray(tokenRef.get().getIdentifier()); + LOG.info("A new token is added for block {}. Expiry: {}", + blockID, Instant.ofEpochMilli(tokenId.getExpiryDate())); + } } } else { throw cause; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index aa85e0a4109c..679135d552bb 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -20,6 +20,7 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; @@ -39,6 +40,8 @@ import java.util.function.Function; import java.util.function.Supplier; +import static org.apache.hadoop.hdds.DatanodeVersion.STEAM_BLOCK_SUPPORT; + /** * Factory class to create various BlockStream instances. */ @@ -72,7 +75,6 @@ public BlockInputStreamFactoryImpl(ByteBufferPool byteBufferPool, * @param blockInfo The blockInfo representing the block. * @param pipeline The pipeline to be used for reading the block * @param token The block Access Token - * @param verifyChecksum Whether to verify checksums or not. * @param xceiverFactory Factory to create the xceiver in the client * @param refreshFunction Function to refresh the pipeline if needed * @return BlockExtendedInputStream of the correct type. @@ -87,7 +89,7 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, return new ECBlockInputStreamProxy((ECReplicationConfig)repConfig, blockInfo, xceiverFactory, refreshFunction, ecBlockStreamFactory, config); - } else if (config.isStreamReadBlock()) { + } else if (config.isStreamReadBlock() && allDataNodesSupportStreamBlock(pipeline)) { return new StreamBlockInput( blockInfo.getBlockID(), blockInfo.getLength(), pipeline, token, xceiverFactory, refreshFunction, config); @@ -98,4 +100,16 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, } } + private boolean allDataNodesSupportStreamBlock(Pipeline pipeline) { + // return true only if all DataNodes in the pipeline are on a version + // that supports for reading a block by streaming chunks.. + for (DatanodeDetails dn : pipeline.getNodes()) { + if (dn.getCurrentVersion() < + STEAM_BLOCK_SUPPORT.toProtoValue()) { + return false; + } + } + return true; + } + } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java index 3a092f238309..596a15f7b6e0 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java @@ -55,7 +55,7 @@ class DummyStreamBlockInput extends StreamBlockInput { Function refreshFunction, OzoneClientConfig config, List chunks, - Map chunkDataMap) { + Map chunkDataMap) throws IOException { super(blockId, blockLen, pipeline, token, xceiverClientManager, refreshFunction, config); this.chunks = chunks; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java index e35d20d53e15..e6f95fdd7237 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java @@ -33,6 +33,8 @@ public enum DatanodeVersion implements ComponentVersion { SEPARATE_RATIS_PORTS_AVAILABLE(1, "Version with separated Ratis port."), COMBINED_PUTBLOCK_WRITECHUNK_RPC(2, "WriteChunk can optionally support " + "a PutBlock request"), + STEAM_BLOCK_SUPPORT(3, + "This version has support for reading a block by streaming chunks."), FUTURE_VERSION(-1, "Used internally in the client when the server side is " + " newer and an unknown server version has arrived to the client."); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index b8206a29b3a7..cf7773386672 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -879,30 +879,27 @@ public static List toValidatorList(Validator validator) { /** * Calls the container protocol to read a chunk. * - * @param chunk information about chunk to read * @param xceiverClient client to perform call * @param blockID ID of the block * @param validators functions to validate the response * @param token a token for this block (may be null) - * @param verifyChecksum * @return container protocol read chunk response * @throws IOException if there is an I/O error while performing the call */ + @SuppressWarnings("checkstyle:ParameterNumber") public static ContainerProtos.StreamDataResponseProto readBlock( - XceiverClientSpi xceiverClient, long offset, long len, DatanodeBlockID blockID, + XceiverClientSpi xceiverClient, long offset, long len, BlockID blockID, List validators, Token token, - boolean verifyChecksum) throws IOException { - ReadBlockRequestProto.Builder readBlockRequest = + Map replicaIndexes, boolean verifyChecksum) throws IOException { + final ReadBlockRequestProto.Builder readBlockRequest = ReadBlockRequestProto.newBuilder() - .setBlockID(blockID) .setOffset(offset) .setVerifyChecksum(verifyChecksum) .setLen(len) .setVersion(ContainerProtos.ReadChunkVersion.V1); - ContainerCommandRequestProto.Builder builder = + final ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto.newBuilder().setCmdType(Type.ReadBlock) - .setContainerID(blockID.getContainerID()) - .setReadBlock(readBlockRequest); + .setContainerID(blockID.getContainerID()); if (token != null) { builder.setEncodedToken(token.encodeToUrlString()); } @@ -910,16 +907,25 @@ public static ContainerProtos.StreamDataResponseProto readBlock( return tryEachDatanode(xceiverClient.getPipeline(), d -> readBlock(xceiverClient, - validators, builder, d), - d -> toErrorMessage(BlockID.getFromProtobuf(blockID), d)); + validators, blockID, builder, readBlockRequest, d, replicaIndexes), + d -> toErrorMessage(blockID, d)); } private static StreamDataResponseProto readBlock(XceiverClientSpi xceiverClient, - List validators, + List validators, BlockID blockID, ContainerCommandRequestProto.Builder builder, - DatanodeDetails datanode) throws IOException { + ReadBlockRequestProto.Builder readBlockBuilder, + DatanodeDetails datanode, + Map replicaIndexes) throws IOException { + final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); + int replicaIndex = replicaIndexes.getOrDefault(datanode, 0); + if (replicaIndex > 0) { + datanodeBlockID.setReplicaIndex(replicaIndex); + } + readBlockBuilder.setBlockID(datanodeBlockID); final ContainerCommandRequestProto request = builder - .setDatanodeUuid(datanode.getUuidString()).build(); + .setDatanodeUuid(datanode.getUuidString()) + .setReadBlock(readBlockBuilder).build(); ContainerCommandResponseProto response = xceiverClient.sendCommand(request, validators); return response.getStreamData(); 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 045389d4764a..fc41e03951a5 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 @@ -1352,6 +1352,9 @@ public ContainerCommandResponseProto streamDataReadOnly( BlockID blockID = BlockID.getFromProtobuf( readBlock.getBlockID()); + if (replicaIndexCheckRequired(request)) { + BlockUtils.verifyReplicaIdx(kvContainer, blockID); + } BlockData blockData = blockManager.getBlock(kvContainer, blockID); List chunkInfos = blockData.getChunks(); long blockOffset = 0; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java index 9c76c0ec0c79..af3b70a5cea9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java @@ -275,7 +275,7 @@ abstract class Builder { protected boolean includeRecon = false; protected int dnInitialVersion = DatanodeVersion.FUTURE_VERSION.toProtoValue(); - protected int dnCurrentVersion = DatanodeVersion.COMBINED_PUTBLOCK_WRITECHUNK_RPC.toProtoValue(); + protected int dnCurrentVersion = DatanodeVersion.STEAM_BLOCK_SUPPORT.toProtoValue(); protected int numOfDatanodes = 3; protected boolean startDataNodes = true; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java index 256148dfb8de..ffd3bb821fbe 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java @@ -44,10 +44,15 @@ abstract class TestInputStreamBase { protected static MiniOzoneCluster newCluster( ContainerLayoutVersion containerLayout) throws Exception { + return newCluster(containerLayout, false); + } + protected static MiniOzoneCluster newCluster( + ContainerLayoutVersion containerLayout, boolean streamReadBlock) throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); OzoneClientConfig config = conf.getObject(OzoneClientConfig.class); config.setBytesPerChecksum(BYTES_PER_CHECKSUM); + config.setStreamReadBlock(streamReadBlock); conf.setFromObject(config); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java new file mode 100644 index 000000000000..9243c5780ace --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java @@ -0,0 +1,237 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.ozone.client.rpc.read; + +import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.io.KeyInputStream; +import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; +import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; +import org.apache.hadoop.ozone.om.TestBucket; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.BLOCK_SIZE; +import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.BYTES_PER_CHECKSUM; +import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.CHUNK_SIZE; +import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.getNewKeyName; +import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.newCluster; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +/** + * Tests {@link StreamBlockInput}. + */ +public class TestStreamBlockInput { + /** + * Run the tests as a single test method to avoid needing a new mini-cluster + * for each test. + */ + @ContainerLayoutTestInfo.ContainerTest + void testAll(ContainerLayoutVersion layout) throws Exception { + try (MiniOzoneCluster cluster = newCluster(layout, true)) { + cluster.waitForClusterToBeReady(); + + try (OzoneClient client = cluster.newClient()) { + TestBucket bucket = TestBucket.newBuilder(client).build(); + + testChunkReadBuffers(bucket); + testBufferRelease(bucket); + testCloseReleasesBuffers(bucket); + } + } + } + + + /** + * Test to verify that data read from chunks is stored in a list of buffers + * with max capacity equal to the bytes per checksum. + */ + private void testChunkReadBuffers(TestBucket bucket) throws Exception { + String keyName = getNewKeyName(); + int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); + byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); + + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + + StreamBlockInput block0Stream = + (StreamBlockInput) keyInputStream.getPartStreams().get(0); + + + // To read 1 byte of chunk data, ChunkInputStream should get one full + // checksum boundary worth of data from Container and store it in buffers. + block0Stream.read(new byte[1]); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, + BYTES_PER_CHECKSUM); + + // Read > checksum boundary of data from chunk0 + int readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); + byte[] readData = readDataFromChunk(block0Stream, 0, readDataLen); + bucket.validateData(inputData, 0, readData); + + // The first checksum boundary size of data was already existing in the + // ChunkStream buffers. Once that data is read, the next checksum + // boundary size of data will be fetched again to read the remaining data. + // Hence there should be 1 checksum boundary size of data stored in the + // ChunkStreams buffers at the end of the read. + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, + BYTES_PER_CHECKSUM); + + // Seek to a position in the third checksum boundary (so that current + // buffers do not have the seeked position) and read > BYTES_PER_CHECKSUM + // bytes of data. This should result in 2 * BYTES_PER_CHECKSUM amount of + // data being read into the buffers. There should be 2 buffers in the + // stream but the the first buffer should be released after it is read + // and the second buffer should have BYTES_PER_CHECKSUM capacity. + readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); + int offset = 2 * BYTES_PER_CHECKSUM + 1; + readData = readDataFromChunk(block0Stream, offset, readDataLen); + bucket.validateData(inputData, offset, readData); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, + BYTES_PER_CHECKSUM); + + + // Read the full chunk data -1 and verify that all chunk data is read into + // buffers. We read CHUNK_SIZE - 1 as otherwise all the buffers will be + // released once all chunk data is read. + readData = readDataFromChunk(block0Stream, 0, CHUNK_SIZE - 1); + bucket.validateData(inputData, 0, readData); + int expectedNumBuffers = CHUNK_SIZE / BYTES_PER_CHECKSUM; + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, BYTES_PER_CHECKSUM); + + // Read the last byte of chunk and verify that the buffers are released. + block0Stream.read(new byte[1]); + assertNull(block0Stream.getCachedBuffers(), + "ChunkInputStream did not release buffers after reaching EOF."); + } + } + + private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { + String keyName = getNewKeyName(); + bucket.writeRandomBytes(keyName, CHUNK_SIZE); + + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + StreamBlockInput block0Stream = + (StreamBlockInput) keyInputStream.getPartStreams().get(0); + + readDataFromChunk(block0Stream, 0, 1); + assertNotNull(block0Stream.getCachedBuffers()); + + block0Stream.close(); + + assertNull(block0Stream.getCachedBuffers()); + } + } + + /** + * Test that ChunkInputStream buffers are released as soon as the last byte + * of the buffer is read. + */ + private void testBufferRelease(TestBucket bucket) throws Exception { + String keyName = getNewKeyName(); + byte[] inputData = bucket.writeRandomBytes(keyName, CHUNK_SIZE); + + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + + StreamBlockInput block0Stream = + (StreamBlockInput) keyInputStream.getPartStreams().get(0); + + // Read checksum boundary - 1 bytes of data + int readDataLen = BYTES_PER_CHECKSUM - 1; + byte[] readData = readDataFromChunk(block0Stream, 0, readDataLen); + bucket.validateData(inputData, 0, readData); + + // There should be 1 byte of data remaining in the buffer which is not + // yet read. Hence, the buffer should not be released. + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), + 1, BYTES_PER_CHECKSUM); + assertEquals(1, block0Stream.getCachedBuffers()[0].remaining()); + + // Reading the last byte in the buffer should result in all the buffers + // being released. + readData = readDataFromChunk(block0Stream, 1); + bucket.validateData(inputData, readDataLen, readData); + assertNull(block0Stream.getCachedBuffers(), + "Chunk stream buffers not released after last byte is read"); + + // Read more data to get the data till the next checksum boundary. + readDataLen = BYTES_PER_CHECKSUM / 2; + readData = readDataFromChunk(block0Stream, readDataLen); + // There should be one buffer and the buffer should not be released as + // there is data pending to be read from the buffer + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, + BYTES_PER_CHECKSUM); + ByteBuffer lastCachedBuffer = block0Stream.getCachedBuffers()[0]; + assertEquals(BYTES_PER_CHECKSUM - readDataLen, + lastCachedBuffer.remaining()); + + // Read more than the remaining data in buffer (but less than the next + // checksum boundary). + int position = (int) block0Stream.getPos(); + readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; + readData = readDataFromChunk(block0Stream, readDataLen); + bucket.validateData(inputData, position, readData); + // After reading the remaining data in the buffer, the buffer should be + // released and next checksum size of data must be read into the buffers + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, + BYTES_PER_CHECKSUM); + // Verify that the previously cached buffer is released by comparing it + // with the current cached buffer + assertNotEquals(lastCachedBuffer, + block0Stream.getCachedBuffers()[0]); + } + } + + private byte[] readDataFromChunk(StreamBlockInput streamBlockInput, + int offset, int readDataLength) throws IOException { + byte[] readData = new byte[readDataLength]; + streamBlockInput.seek(offset); + streamBlockInput.read(readData, 0, readDataLength); + return readData; + } + + private byte[] readDataFromChunk(StreamBlockInput streamBlockInput, + int readDataLength) throws IOException { + byte[] readData = new byte[readDataLength]; + streamBlockInput.read(readData, 0, readDataLength); + return readData; + } + + /** + * Verify number of buffers and their capacities. + * @param buffers chunk stream buffers + * @param expectedNumBuffers expected number of buffers + * @param expectedBufferCapacity expected buffer capacity of unreleased + * buffers + */ + private void checkBufferSizeAndCapacity(ByteBuffer[] buffers, + int expectedNumBuffers, + long expectedBufferCapacity) { + assertEquals(expectedNumBuffers, buffers.length, + "ChunkInputStream does not have expected number of " + + "ByteBuffers"); + for (int i = 0; i < buffers.length; i++) { + assertEquals(expectedBufferCapacity, buffers[i].capacity(), + "ChunkInputStream ByteBuffer capacity is wrong"); + } + } +} From 61b79ae23ea63226958361c24bef49929a418055 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 26 Sep 2024 06:27:13 +0800 Subject: [PATCH 043/114] default streamReadBlock as false --- .../main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 738df5a62b15..328b5101a919 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 @@ -133,7 +133,7 @@ public enum ChecksumCombineMode { type = ConfigType.BOOLEAN, description = "Allow ReadBlock to stream all the readChunk in one request.", tags = ConfigTag.CLIENT) - private boolean streamReadBlock = true; + private boolean streamReadBlock = false; @Config(key = "max.retries", defaultValue = "5", From 5fb147818f18c0041cbeee0bbdf3cd42f56f860c Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 30 Sep 2024 21:40:56 +0800 Subject: [PATCH 044/114] log BlockInputStream Initializing --- .../org/apache/hadoop/hdds/scm/storage/BlockInputStream.java | 1 + .../org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java | 1 + 2 files changed, 2 insertions(+) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index f792a678dad4..c0027ab87e86 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -124,6 +124,7 @@ public BlockInputStream( OzoneClientConfig config) throws IOException { this.blockInfo = blockInfo; this.blockID = blockInfo.getBlockID(); + LOG.debug("Initializing BlockInputStream for block {}", blockID); this.length = blockInfo.getLength(); setPipeline(pipeline); tokenRef.set(token); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index d6c85742242b..f253c0a5ac84 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -101,6 +101,7 @@ public StreamBlockInput( Function refreshFunction, OzoneClientConfig config) throws IOException { this.blockID = blockID; + LOG.debug("Initializing StreamBlockInput for block {}", blockID); this.length = length; setPipeline(pipeline); tokenRef.set(token); From b857fbafeba68245c92d5c8b88a166e4d2286cd5 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 3 Oct 2024 15:41:53 +0800 Subject: [PATCH 045/114] remove ReadBlockResponse --- .../hadoop/hdds/scm/OzoneClientConfig.java | 2 +- .../hadoop/hdds/scm/XceiverClientGrpc.java | 8 ++++---- .../hdds/scm/storage/StreamBlockInput.java | 19 +++++++++--------- .../ContainerCommandResponseBuilders.java | 12 ++++------- .../main/proto/DatanodeClientProtocol.proto | 13 +----------- .../ozone/client/MockXceiverClientSpi.java | 20 ------------------- 6 files changed, 20 insertions(+), 54 deletions(-) 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 328b5101a919..738df5a62b15 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 @@ -133,7 +133,7 @@ public enum ChecksumCombineMode { type = ConfigType.BOOLEAN, description = "Allow ReadBlock to stream all the readChunk in one request.", tags = ConfigTag.CLIENT) - private boolean streamReadBlock = false; + private boolean streamReadBlock = true; @Config(key = "max.retries", defaultValue = "5", diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index d5ffda08ec03..ff568d675320 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc; @@ -648,10 +648,10 @@ public XceiverClientReply sendCommandAsyncReadOnly( @Override public void onNext( ContainerCommandResponseProto responseProto) { - ReadBlockResponseProto readBlock = - responseProto.getReadBlock(); + ReadChunkRequestProto readChunk = + responseProto.getReadChunk(); if (responseProto.getResult() == Result.SUCCESS) { - streamData.addReadBlock(readBlock); + streamData.addReadChunk(readChunk); } else { future.complete( ContainerCommandResponseProto.newBuilder(responseProto) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index f253c0a5ac84..2bb336fe8d9d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; @@ -565,9 +565,9 @@ protected long readData(long startByteIndex, long len) StreamDataResponseProto response = ContainerProtocolCalls.readBlock(xceiverClient, startByteIndex, len, blockID, validators, tokenRef.get(), pipeline.getReplicaIndexes(), verifyChecksum); - List readBlocks = response.getReadBlockList(); + List readBlocks = response.getReadChunkList(); - for (ReadBlockResponseProto readBlock : readBlocks) { + for (ReadChunkResponseProto readBlock : readBlocks) { if (readBlock.hasData()) { buffers.add(readBlock.getData().asReadOnlyByteBuffer()); } else if (readBlock.hasDataBuffers()) { @@ -681,18 +681,18 @@ private void validateBlock( ) throws IOException { StreamDataResponseProto streamData = response.getStreamData(); - for (ReadBlockResponseProto readBlock : streamData.getReadBlockList()) { + for (ReadChunkResponseProto readChunk : streamData.getReadChunkList()) { List byteStrings; boolean isV0 = false; ContainerProtos.ChunkInfo chunkInfo = - readBlock.getChunkData(); + readChunk.getChunkData(); if (chunkInfo.getLen() <= 0) { throw new IOException("Failed to get chunk: chunkName == " + chunkInfo.getChunkName() + "len == " + chunkInfo.getLen()); } - if (readBlock.hasData()) { - ByteString byteString = readBlock.getData(); + if (readChunk.hasData()) { + ByteString byteString = readChunk.getData(); if (byteString.size() != chunkInfo.getLen()) { // Bytes read from chunk should be equal to chunk size. throw new OzoneChecksumException(String.format( @@ -704,7 +704,7 @@ private void validateBlock( byteStrings.add(byteString); isV0 = true; } else { - byteStrings = readBlock.getDataBuffers().getBuffersList(); + byteStrings = readChunk.getDataBuffers().getBuffersList(); long buffersLen = BufferUtils.getBuffersLen(byteStrings); if (buffersLen != chunkInfo.getLen()) { // Bytes read from chunk should be equal to chunk size. @@ -718,12 +718,13 @@ private void validateBlock( if (verifyChecksum) { ChecksumData checksumData = ChecksumData.getFromProtoBuf( chunkInfo.getChecksumData()); + int startIndex = (int) readChunk.getChunkData().getOffset() / checksumData.getBytesPerChecksum(); // ChecksumData stores checksum for each 'numBytesPerChecksum' // number of bytes in a list. Compute the index of the first // checksum to match with the read data - Checksum.verifyChecksum(byteStrings, checksumData, readBlock.getStartIndex(), + Checksum.verifyChecksum(byteStrings, checksumData, startIndex, isV0); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index 2eed8cfce59f..8dc8eac7c083 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetSmallFileResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ListBlockResponseProto; @@ -340,28 +339,25 @@ public static ContainerCommandResponseProto getReadBlockResponse( Function byteBufferToByteString, int startIndex) { - ReadBlockResponseProto.Builder response; + ReadChunkResponseProto.Builder response; if (isReadChunkV0) { // V0 has all response data in a single ByteBuffer - response = ReadBlockResponseProto.newBuilder() + response = ReadChunkResponseProto.newBuilder() .setChunkData(chunkInfo) .setData(data.toByteString(byteBufferToByteString)) .setBlockID(blockID); } else { // V1 splits response data into a list of ByteBuffers - response = ReadBlockResponseProto.newBuilder() + response = ReadChunkResponseProto.newBuilder() .setChunkData(chunkInfo) .setDataBuffers(DataBuffers.newBuilder() .addAllBuffers(data.toByteStringList(byteBufferToByteString)) .build()) .setBlockID(blockID); } - if (startIndex >= 0) { - response.setStartIndex(startIndex); - } return getSuccessResponseBuilder(request) - .setReadBlock(response) + .setReadChunk(response) .build(); } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 412a1650e8cd..25144c6eb80d 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -405,19 +405,8 @@ message ReadBlockRequestProto { optional ReadChunkVersion version = 5; } -message ReadBlockResponseProto { - required DatanodeBlockID blockID = 1; - required ChunkInfo chunkData = 2; - optional uint32 startIndex = 3; - // Chunk data should be returned in one of the two for - oneof responseData { - bytes data = 4; // Chunk data is returned as single buffer for V0 - DataBuffers dataBuffers = 5; // Chunk data is returned as a list of buffers - } -} - message StreamDataResponseProto { - repeated ReadBlockResponseProto readBlock = 1; + repeated ReadChunkResponseProto readChunk = 1; } message EchoRequestProto { diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java index b2a2b88de1a1..91bcab671dfb 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java @@ -30,8 +30,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetCommittedBlockLengthResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; @@ -106,9 +104,6 @@ public XceiverClientReply sendCommandAsync( case ListBlock: return result(request, r -> r.setListBlock(listBlock(request.getContainerID()))); - case ReadBlock: - return result(request, - r -> r.setStreamData(readBlock(request.getReadBlock()))); default: throw new IllegalArgumentException( "Mock version of datanode call " + request.getCmdType() @@ -136,21 +131,6 @@ private ContainerProtos.ListBlockResponseProto listBlock(long containerID) { .addAllBlockData(datanodeStorage.listBlock(containerID)).build(); } - private ContainerProtos.StreamDataResponseProto readBlock( - ReadBlockRequestProto readBlock) { - BlockData blockData = datanodeStorage.getBlock(readBlock.getBlockID()); - List chunkInfos = blockData.getChunksList(); - StreamDataResponseProto.Builder builder = StreamDataResponseProto.newBuilder(); - for (ChunkInfo chunkInfo : chunkInfos) { - builder.addReadBlock(ReadBlockResponseProto.newBuilder() - .setChunkData(chunkInfo) - .setData(datanodeStorage - .readChunkData(blockData.getBlockID(), chunkInfo)) - .setBlockID(blockData.getBlockID()).build()); - } - return builder.build(); - } - private PutBlockResponseProto putBlock(PutBlockRequestProto putBlock) { return PutBlockResponseProto.newBuilder() .setCommittedBlockLength( From 92505bc22477750728c0a77d39e15e2ffed15861 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 3 Oct 2024 17:54:42 +0800 Subject: [PATCH 046/114] remove ReadBlockResponse --- .../java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java | 5 +++-- .../org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java | 2 +- .../src/main/proto/DatanodeClientProtocol.proto | 3 +-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index ff568d675320..bc626354a023 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc; @@ -648,7 +649,7 @@ public XceiverClientReply sendCommandAsyncReadOnly( @Override public void onNext( ContainerCommandResponseProto responseProto) { - ReadChunkRequestProto readChunk = + ReadChunkResponseProto readChunk = responseProto.getReadChunk(); if (responseProto.getResult() == Result.SUCCESS) { streamData.addReadChunk(readChunk); @@ -670,7 +671,7 @@ public void onError(Throwable t) { @Override public void onCompleted() { - if (streamData.getReadBlockCount() > 0) { + if (streamData.getReadChunkCount() > 0) { future.complete(response.setStreamData(streamData) .setCmdType(Type.StreamRead).setResult(Result.SUCCESS).build()); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 2bb336fe8d9d..60e092243cb3 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -578,7 +578,7 @@ protected long readData(long startByteIndex, long len) "from container. No data returned."); } } - return response.getReadBlock(0) + return response.getReadChunk(0) .getChunkData().getOffset(); } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 25144c6eb80d..e66dc48242d4 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -253,8 +253,7 @@ message ContainerCommandResponseProto { optional FinalizeBlockResponseProto finalizeBlock = 22; optional EchoResponseProto echo = 23; - optional ReadBlockResponseProto readBlock = 24; - optional StreamDataResponseProto streamData = 25;} + optional StreamDataResponseProto streamData = 24;} message ContainerDataProto { enum State { From 8202468c3dfefebf38dc29a6485dd3785ef9bd7a Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 4 Oct 2024 04:16:22 +0800 Subject: [PATCH 047/114] remove V0 --- .../hdds/scm/storage/StreamBlockInput.java | 34 ++++++------------- .../ContainerCommandResponseBuilders.java | 27 +++++---------- .../container/keyvalue/KeyValueHandler.java | 17 ++-------- 3 files changed, 20 insertions(+), 58 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 60e092243cb3..1ea6e69bef03 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -683,7 +683,6 @@ private void validateBlock( StreamDataResponseProto streamData = response.getStreamData(); for (ReadChunkResponseProto readChunk : streamData.getReadChunkList()) { List byteStrings; - boolean isV0 = false; ContainerProtos.ChunkInfo chunkInfo = readChunk.getChunkData(); @@ -691,30 +690,17 @@ private void validateBlock( throw new IOException("Failed to get chunk: chunkName == " + chunkInfo.getChunkName() + "len == " + chunkInfo.getLen()); } - if (readChunk.hasData()) { - ByteString byteString = readChunk.getData(); - if (byteString.size() != chunkInfo.getLen()) { - // Bytes read from chunk should be equal to chunk size. - throw new OzoneChecksumException(String.format( - "Inconsistent read for chunk=%s len=%d bytesRead=%d", - chunkInfo.getChunkName(), chunkInfo.getLen(), - byteString.size())); - } - byteStrings = new ArrayList<>(); - byteStrings.add(byteString); - isV0 = true; - } else { - byteStrings = readChunk.getDataBuffers().getBuffersList(); - long buffersLen = BufferUtils.getBuffersLen(byteStrings); - if (buffersLen != chunkInfo.getLen()) { - // Bytes read from chunk should be equal to chunk size. - throw new OzoneChecksumException(String.format( - "Inconsistent read for chunk=%s len=%d bytesRead=%d", - chunkInfo.getChunkName(), chunkInfo.getLen(), - buffersLen)); - } + byteStrings = readChunk.getDataBuffers().getBuffersList(); + long buffersLen = BufferUtils.getBuffersLen(byteStrings); + if (buffersLen != chunkInfo.getLen()) { + // Bytes read from chunk should be equal to chunk size. + throw new OzoneChecksumException(String.format( + "Inconsistent read for chunk=%s len=%d bytesRead=%d", + chunkInfo.getChunkName(), chunkInfo.getLen(), + buffersLen)); } + if (verifyChecksum) { ChecksumData checksumData = ChecksumData.getFromProtoBuf( chunkInfo.getChecksumData()); @@ -725,7 +711,7 @@ private void validateBlock( // checksum to match with the read data Checksum.verifyChecksum(byteStrings, checksumData, startIndex, - isV0); + false); } } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index 8dc8eac7c083..874bfcac8135 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -335,27 +335,16 @@ public static ContainerCommandResponseProto getReadChunkResponse( public static ContainerCommandResponseProto getReadBlockResponse( ContainerCommandRequestProto request, DatanodeBlockID blockID, - ChunkInfo chunkInfo, boolean isReadChunkV0, ChunkBuffer data, - Function byteBufferToByteString, - int startIndex) { + ChunkInfo chunkInfo, ChunkBuffer data, + Function byteBufferToByteString) { ReadChunkResponseProto.Builder response; - - if (isReadChunkV0) { - // V0 has all response data in a single ByteBuffer - response = ReadChunkResponseProto.newBuilder() - .setChunkData(chunkInfo) - .setData(data.toByteString(byteBufferToByteString)) - .setBlockID(blockID); - } else { - // V1 splits response data into a list of ByteBuffers - response = ReadChunkResponseProto.newBuilder() - .setChunkData(chunkInfo) - .setDataBuffers(DataBuffers.newBuilder() - .addAllBuffers(data.toByteStringList(byteBufferToByteString)) - .build()) - .setBlockID(blockID); - } + response = ReadChunkResponseProto.newBuilder() + .setChunkData(chunkInfo) + .setDataBuffers(DataBuffers.newBuilder() + .addAllBuffers(data.toByteStringList(byteBufferToByteString)) + .build()) + .setBlockID(blockID); return getSuccessResponseBuilder(request) .setReadChunk(response) .build(); 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 fc41e03951a5..409eac8e3ccc 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 @@ -1372,22 +1372,16 @@ public ContainerCommandResponseProto streamDataReadOnly( dispatcherContext = DispatcherContext.getHandleReadBlock(); } - boolean isReadChunkV0 = readBlock.getVersion() - .equals(ContainerProtos.ReadChunkVersion.V0); - long offset = readBlock.getOffset(); long len = readBlock.getLen(); long adjustedChunkOffset, adjustedChunkLen; do { - int startIndex = -1; ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); if (readBlock.getVerifyChecksum()) { Pair adjustedOffsetAndLength = computeChecksumBoundaries(chunk, offset, len); adjustedChunkOffset = adjustedOffsetAndLength.getLeft(); adjustedChunkLen = adjustedOffsetAndLength.getRight(); - startIndex = (int) adjustedChunkOffset / - chunk.getChecksumData().getBytesPerChecksum(); adjustedChunkOffset += chunk.getOffset(); } else { adjustedChunkOffset = offset; @@ -1399,13 +1393,6 @@ public ContainerCommandResponseProto streamDataReadOnly( ContainerProtos.ChunkInfo.newBuilder(chunk) .setOffset(adjustedChunkOffset) .setLen(adjustedChunkLen).build()); - // For older clients, set ReadDataIntoSingleBuffer to true so that - // all the data read from chunk file is returned as a single - // ByteString. Older clients cannot process data returned as a list - // of ByteStrings. - if (isReadChunkV0) { - chunkInfo.setReadDataIntoSingleBuffer(isReadChunkV0); - } data = chunkManager.readChunk( kvContainer, blockID, chunkInfo, dispatcherContext); @@ -1418,8 +1405,8 @@ public ContainerCommandResponseProto streamDataReadOnly( streamObserver.onNext( getReadBlockResponse(request, blockData.getProtoBufMessage().getBlockID(), - chunkInfo.getProtoBufMessage(), isReadChunkV0, - data, byteBufferToByteString, startIndex)); + chunkInfo.getProtoBufMessage(), + data, byteBufferToByteString)); len -= adjustedChunkLen + adjustedChunkOffset - offset; offset = adjustedChunkOffset + adjustedChunkLen; chunkIndex++; From a1925c4b8a1ec06150425a0a47a5a65ee7513a1f Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 4 Oct 2024 04:47:13 +0800 Subject: [PATCH 048/114] fix checkstyle --- .../java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java | 1 - .../org/apache/hadoop/ozone/client/MockXceiverClientSpi.java | 3 --- 2 files changed, 4 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index bc626354a023..b45b9dd0a798 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -41,7 +41,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java index 91bcab671dfb..b14582c8ea97 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java @@ -20,7 +20,6 @@ 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.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; @@ -33,7 +32,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; @@ -43,7 +41,6 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import java.io.IOException; -import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.function.Function; From ef2345542bff08b07815ada0ad6ace7806fc5bb6 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 4 Oct 2024 05:37:59 +0800 Subject: [PATCH 049/114] remove V0 --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 10 ++++----- .../hdds/scm/storage/StreamBlockInput.java | 8 +++---- .../scm/storage/ContainerProtocolCalls.java | 21 +++++++++---------- .../hdds/scm/utils/ClientCommandsUtils.java | 9 -------- .../container/keyvalue/KeyValueHandler.java | 2 +- .../main/proto/DatanodeClientProtocol.proto | 5 ++--- 6 files changed, 22 insertions(+), 33 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index b45b9dd0a798..ca5326178e37 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -633,8 +633,8 @@ public XceiverClientReply sendCommandAsyncReadOnly( new CompletableFuture<>(); ContainerCommandResponseProto.Builder response = ContainerCommandResponseProto.newBuilder(); - ContainerProtos.StreamDataResponseProto.Builder streamData = - ContainerProtos.StreamDataResponseProto.newBuilder(); + ContainerProtos.ReadBlockResponseProto.Builder readBlock = + ContainerProtos.ReadBlockResponseProto.newBuilder(); checkOpen(dn); UUID dnID = dn.getUuid(); Type cmdType = request.getCmdType(); @@ -651,7 +651,7 @@ public void onNext( ReadChunkResponseProto readChunk = responseProto.getReadChunk(); if (responseProto.getResult() == Result.SUCCESS) { - streamData.addReadChunk(readChunk); + readBlock.addReadChunk(readChunk); } else { future.complete( ContainerCommandResponseProto.newBuilder(responseProto) @@ -670,8 +670,8 @@ public void onError(Throwable t) { @Override public void onCompleted() { - if (streamData.getReadChunkCount() > 0) { - future.complete(response.setStreamData(streamData) + if (readBlock.getReadChunkCount() > 0) { + future.complete(response.setReadBlock(readBlock) .setCmdType(Type.StreamRead).setResult(Result.SUCCESS).build()); } if (!future.isDone()) { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 1ea6e69bef03..299360ff5750 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; @@ -562,7 +562,7 @@ protected long readData(long startByteIndex, long len) throws IOException { Pipeline pipeline = pipelineRef.get(); buffers = new ArrayList<>(); - StreamDataResponseProto response = + ReadBlockResponseProto response = ContainerProtocolCalls.readBlock(xceiverClient, startByteIndex, len, blockID, validators, tokenRef.get(), pipeline.getReplicaIndexes(), verifyChecksum); List readBlocks = response.getReadChunkList(); @@ -680,8 +680,8 @@ private void validateBlock( ContainerProtos.ContainerCommandResponseProto response ) throws IOException { - StreamDataResponseProto streamData = response.getStreamData(); - for (ReadChunkResponseProto readChunk : streamData.getReadChunkList()) { + ReadBlockResponseProto readBlock = response.getReadBlock(); + for (ReadChunkResponseProto readChunk : readBlock.getReadChunkList()) { List byteStrings; ContainerProtos.ChunkInfo chunkInfo = diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index cf7773386672..5bc7a3459e65 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -54,11 +54,11 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StreamDataResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.FinalizeBlockRequestProto; @@ -887,7 +887,7 @@ public static List toValidatorList(Validator validator) { * @throws IOException if there is an I/O error while performing the call */ @SuppressWarnings("checkstyle:ParameterNumber") - public static ContainerProtos.StreamDataResponseProto readBlock( + public static ContainerProtos.ReadBlockResponseProto readBlock( XceiverClientSpi xceiverClient, long offset, long len, BlockID blockID, List validators, Token token, Map replicaIndexes, boolean verifyChecksum) throws IOException { @@ -895,8 +895,7 @@ public static ContainerProtos.StreamDataResponseProto readBlock( ReadBlockRequestProto.newBuilder() .setOffset(offset) .setVerifyChecksum(verifyChecksum) - .setLen(len) - .setVersion(ContainerProtos.ReadChunkVersion.V1); + .setLen(len); final ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto.newBuilder().setCmdType(Type.ReadBlock) .setContainerID(blockID.getContainerID()); @@ -911,12 +910,12 @@ public static ContainerProtos.StreamDataResponseProto readBlock( d -> toErrorMessage(blockID, d)); } - private static StreamDataResponseProto readBlock(XceiverClientSpi xceiverClient, - List validators, BlockID blockID, - ContainerCommandRequestProto.Builder builder, - ReadBlockRequestProto.Builder readBlockBuilder, - DatanodeDetails datanode, - Map replicaIndexes) throws IOException { + private static ReadBlockResponseProto readBlock(XceiverClientSpi xceiverClient, + List validators, BlockID blockID, + ContainerCommandRequestProto.Builder builder, + ReadBlockRequestProto.Builder readBlockBuilder, + DatanodeDetails datanode, + Map replicaIndexes) throws IOException { final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); int replicaIndex = replicaIndexes.getOrDefault(datanode, 0); if (replicaIndex > 0) { @@ -928,6 +927,6 @@ private static StreamDataResponseProto readBlock(XceiverClientSpi xceiverClient, .setReadBlock(readBlockBuilder).build(); ContainerCommandResponseProto response = xceiverClient.sendCommand(request, validators); - return response.getStreamData(); + return response.getReadBlock(); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java index 97911526210b..4b36db1dd44e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java @@ -48,13 +48,4 @@ public static ContainerProtos.ReadChunkVersion getReadChunkVersion( return ContainerProtos.ReadChunkVersion.V0; } } - - public static ContainerProtos.ReadChunkVersion getReadChunkVersion( - ReadBlockRequestProto readBlock) { - if (readBlock.hasVersion()) { - return readBlock.getVersion(); - } else { - return ContainerProtos.ReadChunkVersion.V0; - } - } } 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 409eac8e3ccc..928796073dd8 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 @@ -121,9 +121,9 @@ 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 static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos .ContainerDataProto.State.RECOVERING; +import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion; import static org.apache.hadoop.ozone.ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST; import static org.apache.hadoop.ozone.OzoneConsts.INCREMENTAL_CHUNK_LIST; import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index e66dc48242d4..a085b4e2229d 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -253,7 +253,7 @@ message ContainerCommandResponseProto { optional FinalizeBlockResponseProto finalizeBlock = 22; optional EchoResponseProto echo = 23; - optional StreamDataResponseProto streamData = 24;} + optional ReadBlockResponseProto readBlock = 24;} message ContainerDataProto { enum State { @@ -401,10 +401,9 @@ message ReadBlockRequestProto { required uint64 offset = 2; required uint64 len = 3; required bool verifyChecksum = 4; - optional ReadChunkVersion version = 5; } -message StreamDataResponseProto { +message ReadBlockResponseProto { repeated ReadChunkResponseProto readChunk = 1; } From 7a7388f3da939f550eaa9be339cc1c1bf72916af Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 4 Oct 2024 14:41:49 +0800 Subject: [PATCH 050/114] fix checkstyle --- .../org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java index 4b36db1dd44e..c4202c9cc6aa 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/utils/ClientCommandsUtils.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hdds.scm.utils; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; /** * These methods should be merged with other similar utility classes. From 7bbe3d976fd5d334dadda1ae9a0aa55f5e2facad Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Sun, 6 Oct 2024 03:35:11 +0800 Subject: [PATCH 051/114] adress comments --- .../apache/hadoop/hdds/scm/storage/StreamBlockInput.java | 4 +--- .../hadoop/ozone/container/keyvalue/KeyValueHandler.java | 6 +++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 299360ff5750..5135bcdf44b9 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -568,9 +568,7 @@ protected long readData(long startByteIndex, long len) List readBlocks = response.getReadChunkList(); for (ReadChunkResponseProto readBlock : readBlocks) { - if (readBlock.hasData()) { - buffers.add(readBlock.getData().asReadOnlyByteBuffer()); - } else if (readBlock.hasDataBuffers()) { + if (readBlock.hasDataBuffers()) { buffers.addAll(BufferUtils.getReadOnlyByteBuffers( readBlock.getDataBuffers().getBuffersList())); } else { 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 928796073dd8..bc9f4eea4168 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 @@ -1352,9 +1352,9 @@ public ContainerCommandResponseProto streamDataReadOnly( BlockID blockID = BlockID.getFromProtobuf( readBlock.getBlockID()); - if (replicaIndexCheckRequired(request)) { - BlockUtils.verifyReplicaIdx(kvContainer, blockID); - } + // This is a new api the block should always be checked. + BlockUtils.verifyReplicaIdx(kvContainer, blockID); + BlockData blockData = blockManager.getBlock(kvContainer, blockID); List chunkInfos = blockData.getChunks(); long blockOffset = 0; From 71094bde359227e334ce8db313cb3a5fce3271d4 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 9 Oct 2024 02:13:06 +0800 Subject: [PATCH 052/114] add testReadBlock in TestKeyValueHandler and rename variable --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 12 +- .../hdds/scm/storage/StreamBlockInput.java | 52 ++++----- .../io/BlockInputStreamFactoryImpl.java | 4 +- .../apache/hadoop/hdds/DatanodeVersion.java | 2 +- .../container/keyvalue/KeyValueHandler.java | 16 +-- .../keyvalue/TestKeyValueHandler.java | 104 +++++++++++++++++- .../apache/hadoop/ozone/MiniOzoneCluster.java | 2 +- 7 files changed, 145 insertions(+), 47 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index ca5326178e37..da9d3d9d3477 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -648,10 +648,8 @@ public XceiverClientReply sendCommandAsyncReadOnly( @Override public void onNext( ContainerCommandResponseProto responseProto) { - ReadChunkResponseProto readChunk = - responseProto.getReadChunk(); if (responseProto.getResult() == Result.SUCCESS) { - readBlock.addReadChunk(readChunk); + readBlock.addReadChunk(responseProto.getReadChunk()); } else { future.complete( ContainerCommandResponseProto.newBuilder(responseProto) @@ -670,15 +668,15 @@ public void onError(Throwable t) { @Override public void onCompleted() { - if (readBlock.getReadChunkCount() > 0) { - future.complete(response.setReadBlock(readBlock) - .setCmdType(Type.StreamRead).setResult(Result.SUCCESS).build()); - } if (!future.isDone()) { future.completeExceptionally(new IOException( "Stream completed but no reply for request " + processForDebug(request))); } + if (readBlock.getReadChunkCount() > 0) { + future.complete(response.setReadBlock(readBlock) + .setCmdType(Type.StreamRead).setResult(Result.SUCCESS).build()); + } metrics.decrPendingContainerOpsMetrics(cmdType); metrics.addContainerOpsLatency( cmdType, System.currentTimeMillis() - requestTime); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 5135bcdf44b9..2650ce6da143 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -71,7 +71,7 @@ public class StreamBlockInput extends BlockExtendedInputStream private static final Logger LOG = LoggerFactory.getLogger(StreamBlockInput.class); private final BlockID blockID; - private final long length; + private final long blockLength; private final AtomicReference pipelineRef = new AtomicReference<>(); private final AtomicReference> tokenRef = @@ -84,7 +84,7 @@ public class StreamBlockInput extends BlockExtendedInputStream private long blockPosition = -1; private List buffers; private boolean allocated = false; - private long bufferOffsetWrtBlockDataData; + private long bufferOffsetWrtBlockData; private long buffersSize; private static final int EOF = -1; private final List validators; @@ -102,7 +102,7 @@ public StreamBlockInput( OzoneClientConfig config) throws IOException { this.blockID = blockID; LOG.debug("Initializing StreamBlockInput for block {}", blockID); - this.length = length; + this.blockLength = length; setPipeline(pipeline); tokenRef.set(token); this.xceiverClientFactory = xceiverClientFactory; @@ -122,12 +122,12 @@ public BlockID getBlockID() { } public long getLength() { - return length; + return blockLength; } @Override public synchronized long getPos() { - if (length == 0) { + if (blockLength == 0) { return 0; } if (blockPosition >= 0) { @@ -136,26 +136,24 @@ public synchronized long getPos() { if (allocated && !buffersHaveData() && !dataRemainingInBlock()) { Preconditions.checkState( - bufferOffsetWrtBlockDataData + buffersSize == length, + bufferOffsetWrtBlockData + buffersSize == blockLength, "EOF detected but not at the last byte of the chunk"); - return length; + return blockLength; } if (buffersHaveData()) { // BufferOffset w.r.t to BlockData + BufferOffset w.r.t buffers + // Position of current Buffer - return bufferOffsetWrtBlockDataData + bufferoffsets.get(bufferIndex) + + return bufferOffsetWrtBlockData + bufferoffsets.get(bufferIndex) + buffers.get(bufferIndex).position(); } if (buffersAllocated()) { - return bufferOffsetWrtBlockDataData + buffersSize; + return bufferOffsetWrtBlockData + buffersSize; } return 0; } @Override public synchronized int read() throws IOException { - checkOpen(); - int dataout = EOF; int len = 1; int available; @@ -351,7 +349,7 @@ protected int readWithStrategy(ByteReaderStrategy strategy) throws IOException { @Override public synchronized void seek(long pos) throws IOException { - if (pos < 0 || pos > length) { + if (pos < 0 || pos > blockLength) { if (pos == 0) { // It is possible for length and pos to be zero in which case // seek should return instead of throwing exception @@ -363,7 +361,7 @@ public synchronized void seek(long pos) throws IOException { if (buffersHavePosition(pos)) { // The bufferPosition is w.r.t the current block. // Adjust the bufferIndex and position to the seeked position. - adjustBufferPosition(pos - bufferOffsetWrtBlockDataData); + adjustBufferPosition(pos - bufferOffsetWrtBlockData); } else { blockPosition = pos; } @@ -411,7 +409,8 @@ protected synchronized void checkOpen() throws IOException { } protected synchronized void acquireClient() throws IOException { - if (xceiverClientFactory != null && xceiverClient == null) { + checkOpen(); + if (xceiverClient == null) { final Pipeline pipeline = pipelineRef.get(); try { xceiverClient = xceiverClientFactory.acquireClientForReadData(pipeline); @@ -429,7 +428,7 @@ private synchronized int prepareRead(int len) throws IOException { if (buffersHavePosition(blockPosition)) { // The current buffers have the seeked position. Adjust the buffer // index and position to point to the buffer position. - adjustBufferPosition(blockPosition - bufferOffsetWrtBlockDataData); + adjustBufferPosition(blockPosition - bufferOffsetWrtBlockData); } else { // Read a required block data to fill the buffers with seeked // position data @@ -459,9 +458,9 @@ private boolean buffersHavePosition(long pos) { // Check if the current buffers cover the input position // Released buffers should not be considered when checking if position // is available - return pos >= bufferOffsetWrtBlockDataData + + return pos >= bufferOffsetWrtBlockData + bufferoffsets.get(0) && - pos < bufferOffsetWrtBlockDataData + buffersSize; + pos < bufferOffsetWrtBlockData + buffersSize; } return false; } @@ -492,6 +491,8 @@ private void adjustBufferPosition(long bufferPosition) { bufferIndex = Collections.binarySearch(bufferoffsets.subList( bufferIndex + 1, buffers.size()), bufferPosition); } + // bufferIndex is negative if bufferPosition isn't found in bufferOffsets + // count (bufferIndex = -bufferIndex - 2) to get bufferPosition is between which offsets. if (bufferIndex < 0) { bufferIndex = -bufferIndex - 2; } @@ -533,7 +534,7 @@ private synchronized void readDataFromContainer(int len) throws IOException { startByteIndex = blockPosition; } else { // Start reading the block from the last blockPosition onwards. - startByteIndex = bufferOffsetWrtBlockDataData + buffersSize; + startByteIndex = bufferOffsetWrtBlockData + buffersSize; } // bufferOffsetWrtChunkData and buffersSize are updated after the data @@ -541,7 +542,7 @@ private synchronized void readDataFromContainer(int len) throws IOException { // and is retried, we need the previous position. Position is reset after // successful read in adjustBufferPosition() blockPosition = getPos(); - bufferOffsetWrtBlockDataData = readData(startByteIndex, len); + bufferOffsetWrtBlockData = readData(startByteIndex, len); long tempOffset = 0L; buffersSize = 0L; bufferoffsets = new ArrayList<>(buffers.size()); @@ -553,7 +554,7 @@ private synchronized void readDataFromContainer(int len) throws IOException { } bufferIndex = 0; allocated = true; - adjustBufferPosition(startByteIndex - bufferOffsetWrtBlockDataData); + adjustBufferPosition(startByteIndex - bufferOffsetWrtBlockData); } @@ -619,17 +620,17 @@ private boolean dataRemainingInBlock() { if (blockPosition >= 0) { bufferPos = blockPosition; } else { - bufferPos = bufferOffsetWrtBlockDataData + buffersSize; + bufferPos = bufferOffsetWrtBlockData + buffersSize; } - return bufferPos < length; + return bufferPos < blockLength; } /** * Check if current buffer had been read till the end. */ private boolean bufferEOF() { - return allocated && !buffers.get(bufferIndex).hasRemaining(); + return allocated && buffersAllocated() && !buffers.get(bufferIndex).hasRemaining(); } /** @@ -644,7 +645,7 @@ private void releaseBuffers(int releaseUptoBufferIndex) { // blockPosition should be set to point to the last position of the // buffers. This should be done so that getPos() can return the current // block position - blockPosition = bufferOffsetWrtBlockDataData + + blockPosition = bufferOffsetWrtBlockData + bufferoffsets.get(releaseUptoBufferIndex) + buffers.get(releaseUptoBufferIndex).capacity(); // Release all the buffers @@ -653,6 +654,7 @@ private void releaseBuffers(int releaseUptoBufferIndex) { buffers = buffers.subList(releaseUptoBufferIndex + 1, buffersLen); bufferoffsets = bufferoffsets.subList( releaseUptoBufferIndex + 1, buffersLen); + bufferIndex = 0; } } @@ -662,7 +664,7 @@ private void releaseBuffers(int releaseUptoBufferIndex) { private void releaseBuffers() { buffers = null; bufferIndex = 0; - // We should not reset bufferOffsetWrtChunkData and buffersSize here + // We should not reset bufferOffsetWrtBlockData and buffersSize here // because when getPos() is called we use these // values and determine whether chunk is read completely or not. } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 679135d552bb..3b1eaf328bcb 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -40,7 +40,7 @@ import java.util.function.Function; import java.util.function.Supplier; -import static org.apache.hadoop.hdds.DatanodeVersion.STEAM_BLOCK_SUPPORT; +import static org.apache.hadoop.hdds.DatanodeVersion.STREAM_BLOCK_SUPPORT; /** * Factory class to create various BlockStream instances. @@ -105,7 +105,7 @@ private boolean allDataNodesSupportStreamBlock(Pipeline pipeline) { // that supports for reading a block by streaming chunks.. for (DatanodeDetails dn : pipeline.getNodes()) { if (dn.getCurrentVersion() < - STEAM_BLOCK_SUPPORT.toProtoValue()) { + STREAM_BLOCK_SUPPORT.toProtoValue()) { return false; } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java index e6f95fdd7237..2556174bab0c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java @@ -33,7 +33,7 @@ public enum DatanodeVersion implements ComponentVersion { SEPARATE_RATIS_PORTS_AVAILABLE(1, "Version with separated Ratis port."), COMBINED_PUTBLOCK_WRITECHUNK_RPC(2, "WriteChunk can optionally support " + "a PutBlock request"), - STEAM_BLOCK_SUPPORT(3, + STREAM_BLOCK_SUPPORT(3, "This version has support for reading a block by streaming chunks."), FUTURE_VERSION(-1, "Used internally in the client when the server side is " 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 bc9f4eea4168..12e986126bdc 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 @@ -1348,14 +1348,14 @@ public ContainerCommandResponseProto streamDataReadOnly( return malformedRequest(request); } ReadBlockRequestProto readBlock = request.getReadBlock(); - ChunkBuffer data; 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); - BlockData blockData = blockManager.getBlock(kvContainer, blockID); + BlockData blockData = getBlockManager().getBlock(kvContainer, blockID); List chunkInfos = blockData.getChunks(); long blockOffset = 0; int chunkIndex = -1; @@ -1366,12 +1366,13 @@ public ContainerCommandResponseProto streamDataReadOnly( break; } } + Preconditions.checkState(chunkIndex >= 0); - BlockUtils.verifyBCSId(kvContainer, blockID); if (dispatcherContext == null) { dispatcherContext = DispatcherContext.getHandleReadBlock(); } + ChunkBuffer data; long offset = readBlock.getOffset(); long len = readBlock.getLen(); long adjustedChunkOffset, adjustedChunkLen; @@ -1393,15 +1394,10 @@ public ContainerCommandResponseProto streamDataReadOnly( ContainerProtos.ChunkInfo.newBuilder(chunk) .setOffset(adjustedChunkOffset) .setLen(adjustedChunkLen).build()); - data = chunkManager.readChunk( + data = getChunkManager().readChunk( kvContainer, blockID, chunkInfo, dispatcherContext); Preconditions.checkNotNull(data, "Chunk data is null"); - if (DispatcherContext.op(dispatcherContext).readFromTmpFile()) { - validateChunkChecksumData(data, chunkInfo); - metrics.incBytesReadStateMachine(chunkInfo.getLen()); - metrics.incNumReadStateMachine(); - } streamObserver.onNext( getReadBlockResponse(request, blockData.getProtoBufMessage().getBlockID(), @@ -1476,7 +1472,7 @@ private boolean logBlocksIfNonZero(Container container) } if (nonZero) { LOG.error("blocks in rocksDB on container delete: {}", - stringBuilder.toString()); + stringBuilder); } } return nonZero; diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index 2637f1922c68..a422c8d2c5cf 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -20,38 +20,54 @@ import java.io.File; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.Path; +import java.util.ArrayList; import java.util.List; import java.util.Collections; import java.util.HashMap; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumData; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; +import org.apache.hadoop.hdds.scm.ByteStringConversion; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.security.token.TokenVerifier; +import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; +import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.interfaces.Handler; +import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; +import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil; import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet; import org.apache.hadoop.ozone.container.common.volume.StorageVolume; import org.apache.hadoop.ozone.container.common.volume.VolumeSet; +import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; +import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; +import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; import org.apache.ozone.test.GenericTestUtils; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DATANODE_VOLUME_CHOOSING_POLICY; @@ -62,16 +78,23 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.io.TempDir; +import org.mockito.MockedStatic; +import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -89,7 +112,13 @@ public class TestKeyValueHandler { private static final String DATANODE_UUID = UUID.randomUUID().toString(); private static final long DUMMY_CONTAINER_ID = 9999; + private static final long LOCAL_ID = 1; private static final String DUMMY_PATH = "dummy/dir/doesnt/exist"; + private static final int CHUNK_SIZE = 1024 * 1024; // 1MB + private static final int BLOCK_SIZE = 4 * CHUNK_SIZE; + private static final int BYTES_PER_CHECKSUM = 256 * 1024; + private final Function byteBufferToByteString = + ByteStringConversion.createByteBufferConversion(true); private HddsDispatcher dispatcher; private KeyValueHandler handler; @@ -281,7 +310,7 @@ public void testVolumeSetInKeyValueHandler() throws Exception { volumeSet, metrics, c -> { }); assertEquals("org.apache.hadoop.ozone.container.common" + - ".volume.CapacityVolumeChoosingPolicy", + ".volume.CapacityVolumeChoosingPolicy", keyValueHandler.getVolumeChoosingPolicyForTesting() .getClass().getName()); @@ -451,4 +480,77 @@ private static ContainerCommandRequestProto createContainerRequest( .setContainerID(containerID).setPipelineID(UUID.randomUUID().toString()) .build(); } + + @Test + public void testReadBlock() throws IOException { + + StreamObserver streamObserver = mock(StreamObserver.class); + KeyValueContainer container = mock(KeyValueContainer.class); + final KeyValueHandler kvHandler = new KeyValueHandler(new OzoneConfiguration(), + UUID.randomUUID().toString(), mock(ContainerSet.class), mock(VolumeSet.class), mock(ContainerMetrics.class), + mock(IncrementalReportSender.class)); + final KeyValueHandler keyValueHandler = spy(kvHandler); + DispatcherContext dispatcherContext = mock(DispatcherContext.class); + + List chunkInfoList = new ArrayList<>(); + BlockData blockData = new BlockData(new BlockID(1, 1)); + for (int i = 0; i < 4; i++) { + chunkInfoList.add(ContainerProtos.ChunkInfo + .newBuilder() + .setOffset(CHUNK_SIZE * i) + .setLen(CHUNK_SIZE) + .setChecksumData( + ChecksumData.newBuilder().setBytesPerChecksum(BYTES_PER_CHECKSUM) + .setType(ChecksumType.CRC32).build()) + .setChunkName("chunkName" + i) + .build()); + } + blockData.setChunks(chunkInfoList); + + try (MockedStatic blockUtils = mockStatic(BlockUtils.class)) { + BlockManager blockManager = mock(BlockManager.class); + ChunkManager chunkManager = mock(ChunkManager.class); + when(keyValueHandler.getBlockManager()).thenReturn(blockManager); + when(keyValueHandler.getChunkManager()).thenReturn(chunkManager); + when(blockManager.getBlock(any(), any())).thenReturn(blockData); + ChunkBuffer data = ChunkBuffer.wrap(ByteBuffer.allocate(0)); + when(chunkManager.readChunk(any(), any(), + any(), any())) + .thenReturn(data); + testStreamDataReadOnly(0, 1, keyValueHandler, dispatcherContext, + streamObserver, container); + testStreamDataReadOnly(0, CHUNK_SIZE + 1, keyValueHandler, dispatcherContext, + streamObserver, container); + testStreamDataReadOnly(CHUNK_SIZE / 2, 2 * CHUNK_SIZE, keyValueHandler, dispatcherContext, + streamObserver, container); + } + } + + private static ContainerCommandRequestProto readBlockRequest( + long offset, long length) { + return ContainerCommandRequestProto.newBuilder() + .setCmdType(Type.ReadBlock) + .setReadBlock( + ContainerProtos.ReadBlockRequestProto.newBuilder() + .setBlockID( + ContainerProtos.DatanodeBlockID.newBuilder() + .setContainerID(DUMMY_CONTAINER_ID) + .setLocalID(LOCAL_ID)) + .setOffset(offset) + .setLen(length) + .setVerifyChecksum(true)) + .setContainerID(DUMMY_CONTAINER_ID) + .setDatanodeUuid(UUID.randomUUID().toString()) + .build(); + } + + private static void testStreamDataReadOnly( + long offset, long length, KeyValueHandler keyValueHandler, DispatcherContext dispatcherContext, + StreamObserver streamObserver, KeyValueContainer container) { + int responseCount = (int) (((offset + length - 1) / CHUNK_SIZE) + 1 - (offset / CHUNK_SIZE)); + ContainerCommandRequestProto requestProto = readBlockRequest(offset, length); + keyValueHandler.streamDataReadOnly(requestProto, container, dispatcherContext, streamObserver); + verify(streamObserver, times(responseCount)).onNext(any()); + clearInvocations(streamObserver); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java index af3b70a5cea9..9655cee1fd78 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java @@ -275,7 +275,7 @@ abstract class Builder { protected boolean includeRecon = false; protected int dnInitialVersion = DatanodeVersion.FUTURE_VERSION.toProtoValue(); - protected int dnCurrentVersion = DatanodeVersion.STEAM_BLOCK_SUPPORT.toProtoValue(); + protected int dnCurrentVersion = DatanodeVersion.STREAM_BLOCK_SUPPORT.toProtoValue(); protected int numOfDatanodes = 3; protected boolean startDataNodes = true; From 9e5f77c94f511b2dca01ea8c7129b79a16637671 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 9 Oct 2024 02:51:13 +0800 Subject: [PATCH 053/114] fix checkstyle and fix bug --- .../apache/hadoop/hdds/scm/XceiverClientGrpc.java | 1 - .../hadoop/hdds/scm/storage/StreamBlockInput.java | 14 +++++++------- .../container/keyvalue/TestKeyValueHandler.java | 8 +++----- 3 files changed, 10 insertions(+), 13 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index da9d3d9d3477..40a3eaa0d3da 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -41,7 +41,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 2650ce6da143..5e6c381a3b8c 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -274,13 +274,6 @@ public synchronized int read(byte[] b, int off, int len) throws IOException { } - @Override - public synchronized void close() throws IOException { - releaseClient(); - releaseBuffers(); - xceiverClientFactory = null; - } - @Override public synchronized int read(ByteBuffer byteBuffer) throws IOException { if (byteBuffer == null) { @@ -776,4 +769,11 @@ public synchronized ByteBuffer[] getCachedBuffers() { private boolean isConnectivityIssue(IOException ex) { return Status.fromThrowable(ex).getCode() == Status.UNAVAILABLE.getCode(); } + + @Override + public synchronized void close() throws IOException { + releaseClient(); + releaseBuffers(); + xceiverClientFactory = null; + } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index a422c8d2c5cf..166a159092e5 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -78,7 +78,6 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; @@ -114,9 +113,8 @@ public class TestKeyValueHandler { private static final long DUMMY_CONTAINER_ID = 9999; private static final long LOCAL_ID = 1; private static final String DUMMY_PATH = "dummy/dir/doesnt/exist"; - private static final int CHUNK_SIZE = 1024 * 1024; // 1MB - private static final int BLOCK_SIZE = 4 * CHUNK_SIZE; - private static final int BYTES_PER_CHECKSUM = 256 * 1024; + private static final long CHUNK_SIZE = 1024 * 1024; // 1MB + private static final long BYTES_PER_CHECKSUM = 256 * 1024; private final Function byteBufferToByteString = ByteStringConversion.createByteBufferConversion(true); @@ -500,7 +498,7 @@ public void testReadBlock() throws IOException { .setOffset(CHUNK_SIZE * i) .setLen(CHUNK_SIZE) .setChecksumData( - ChecksumData.newBuilder().setBytesPerChecksum(BYTES_PER_CHECKSUM) + ChecksumData.newBuilder().setBytesPerChecksum((int) BYTES_PER_CHECKSUM) .setType(ChecksumType.CRC32).build()) .setChunkName("chunkName" + i) .build()); From 281e91ebc88520a81771fe4ccbf8258a3b2d73d5 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 9 Oct 2024 04:23:35 +0800 Subject: [PATCH 054/114] revert StreamObserver.onComplete --- .../org/apache/hadoop/hdds/scm/XceiverClientGrpc.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 40a3eaa0d3da..5be0bc836fb1 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -667,15 +667,15 @@ public void onError(Throwable t) { @Override public void onCompleted() { + if (readBlock.getReadChunkCount() > 0) { + future.complete(response.setReadBlock(readBlock) + .setCmdType(Type.StreamRead).setResult(Result.SUCCESS).build()); + } if (!future.isDone()) { future.completeExceptionally(new IOException( "Stream completed but no reply for request " + processForDebug(request))); } - if (readBlock.getReadChunkCount() > 0) { - future.complete(response.setReadBlock(readBlock) - .setCmdType(Type.StreamRead).setResult(Result.SUCCESS).build()); - } metrics.decrPendingContainerOpsMetrics(cmdType); metrics.addContainerOpsLatency( cmdType, System.currentTimeMillis() - requestTime); From 397ef40ef43a5e8a5a645688f0a086d94144f428 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 9 Oct 2024 19:07:21 +0800 Subject: [PATCH 055/114] create functions to handle exception --- .../hdds/scm/storage/StreamBlockInput.java | 88 +++++++------------ 1 file changed, 34 insertions(+), 54 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 5e6c381a3b8c..16eb8a122668 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -165,25 +165,11 @@ public synchronized int read() throws IOException { } catch (SCMSecurityException ex) { throw ex; } catch (StorageContainerException e) { - if (shouldRetryRead(e)) { - releaseClient(); - refreshBlockInfo(e); - continue; - } else { - throw e; - } + handleStorageContainerException(e); + continue; } catch (IOException ioe) { - if (shouldRetryRead(ioe)) { - if (isConnectivityIssue(ioe)) { - releaseClient(); - refreshBlockInfo(ioe); - } else { - releaseClient(); - } - continue; - } else { - throw ioe; - } + handleIOException(ioe); + continue; } if (available == EOF) { // There is no more data in the chunk stream. The buffers should have @@ -236,25 +222,11 @@ public synchronized int read(byte[] b, int off, int len) throws IOException { } catch (SCMSecurityException ex) { throw ex; } catch (StorageContainerException e) { - if (shouldRetryRead(e)) { - releaseClient(); - refreshBlockInfo(e); - continue; - } else { - throw e; - } + handleStorageContainerException(e); + continue; } catch (IOException ioe) { - if (shouldRetryRead(ioe)) { - if (isConnectivityIssue(ioe)) { - releaseClient(); - refreshBlockInfo(ioe); - } else { - releaseClient(); - } - continue; - } else { - throw ioe; - } + handleIOException(ioe); + continue; } if (available == EOF) { // There is no more data in the block stream. The buffers should have @@ -293,25 +265,11 @@ public synchronized int read(ByteBuffer byteBuffer) throws IOException { } catch (SCMSecurityException ex) { throw ex; } catch (StorageContainerException e) { - if (shouldRetryRead(e)) { - releaseClient(); - refreshBlockInfo(e); - continue; - } else { - throw e; - } + handleStorageContainerException(e); + continue; } catch (IOException ioe) { - if (shouldRetryRead(ioe)) { - if (isConnectivityIssue(ioe)) { - releaseClient(); - refreshBlockInfo(ioe); - } else { - releaseClient(); - } - continue; - } else { - throw ioe; - } + handleIOException(ioe); + continue; } if (available == EOF) { // There is no more data in the block stream. The buffers should have @@ -776,4 +734,26 @@ public synchronized void close() throws IOException { releaseBuffers(); xceiverClientFactory = null; } + + private void handleStorageContainerException(StorageContainerException e) throws IOException { + if (shouldRetryRead(e)) { + releaseClient(); + refreshBlockInfo(e); + } else { + throw e; + } + } + + private void handleIOException(IOException ioe) throws IOException { + if (shouldRetryRead(ioe)) { + if (isConnectivityIssue(ioe)) { + releaseClient(); + refreshBlockInfo(ioe); + } else { + releaseClient(); + } + } else { + throw ioe; + } + } } From fe5f8ec130d875f0be9abae105d477586659cbe0 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 10 Oct 2024 01:51:06 +0800 Subject: [PATCH 056/114] address comments --- .../hdds/scm/storage/StreamBlockInput.java | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index 16eb8a122668..f19948a03feb 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -134,18 +134,18 @@ public synchronized long getPos() { return blockPosition; } - if (allocated && !buffersHaveData() && !dataRemainingInBlock()) { - Preconditions.checkState( - bufferOffsetWrtBlockData + buffersSize == blockLength, - "EOF detected but not at the last byte of the chunk"); - return blockLength; - } if (buffersHaveData()) { // BufferOffset w.r.t to BlockData + BufferOffset w.r.t buffers + // Position of current Buffer return bufferOffsetWrtBlockData + bufferoffsets.get(bufferIndex) + buffers.get(bufferIndex).position(); } + if (allocated && !dataRemainingInBlock()) { + Preconditions.checkState( + bufferOffsetWrtBlockData + buffersSize == blockLength, + "EOF detected but not at the last byte of the chunk"); + return blockLength; + } if (buffersAllocated()) { return bufferOffsetWrtBlockData + buffersSize; } @@ -300,12 +300,12 @@ protected int readWithStrategy(ByteReaderStrategy strategy) throws IOException { @Override public synchronized void seek(long pos) throws IOException { + if (pos == 0 && blockLength == 0) { + // It is possible for length and pos to be zero in which case + // seek should return instead of throwing exception + return; + } if (pos < 0 || pos > blockLength) { - if (pos == 0) { - // It is possible for length and pos to be zero in which case - // seek should return instead of throwing exception - return; - } throw new EOFException("EOF encountered at pos: " + pos + " for block: " + blockID); } From 9566b89a90a01a892838eb6790d525568516c185 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 17 Oct 2024 00:33:28 +0800 Subject: [PATCH 057/114] address comments --- .../dev-support/findbugsExcludeFile.xml | 4 --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 10 ++++---- .../hdds/scm/storage/StreamBlockInput.java | 1 + .../scm/storage/TestStreamBlockInput.java | 25 ++++++++++++------- .../keyvalue/TestKeyValueHandler.java | 2 +- 5 files changed, 23 insertions(+), 19 deletions(-) diff --git a/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml b/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml index cf13ea967f67..7b874905bf8a 100644 --- a/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdds/client/dev-support/findbugsExcludeFile.xml @@ -32,8 +32,4 @@ - - - - diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 5be0bc836fb1..21480efb51ac 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -110,12 +110,12 @@ public class XceiverClientGrpc extends XceiverClientSpi { * Constructs a client that can communicate with the Container framework on * data nodes via DatanodeClientProtocol. * - * @param pipeline - Pipeline that defines the machines. - * @param config -- Ozone Config + * @param pipeline - Pipeline that defines the machines. + * @param config -- Ozone Config * @param trustManager - a {@link ClientTrustManager} with proper CA handling. */ public XceiverClientGrpc(Pipeline pipeline, ConfigurationSource config, - ClientTrustManager trustManager) { + ClientTrustManager trustManager) { super(); Preconditions.checkNotNull(pipeline); Preconditions.checkNotNull(config); @@ -444,7 +444,7 @@ private XceiverClientReply sendCommandWithRetry( // in case these don't exist for the specific datanode. reply.addDatanode(dn); if (request.getCmdType() == ContainerProtos.Type.ReadBlock) { - responseProto = sendCommandAsyncReadOnly(request, dn).getResponse().get(); + responseProto = sendCommandReadBlock(request, dn).getResponse().get(); } else { responseProto = sendCommandAsync(request, dn).getResponse().get(); } @@ -624,7 +624,7 @@ private void decreasePendingMetricsAndReleaseSemaphore() { return new XceiverClientReply(replyFuture); } - public XceiverClientReply sendCommandAsyncReadOnly( + public XceiverClientReply sendCommandReadBlock( ContainerCommandRequestProto request, DatanodeDetails dn) throws IOException, InterruptedException { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java index f19948a03feb..1eab3d17ce7b 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java @@ -83,6 +83,7 @@ public class StreamBlockInput extends BlockExtendedInputStream private int bufferIndex; private long blockPosition = -1; private List buffers; + // Checks if the StreamBlockInput has already read data from the container. private boolean allocated = false; private long bufferOffsetWrtBlockData; private long buffersSize; diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java index f93250e82c79..7bbeb1fc6485 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java @@ -165,7 +165,8 @@ private void seekAndVerify(int pos) throws Exception { @Test public void testFullChunkRead() throws Exception { byte[] b = new byte[blockSize]; - blockStream.read(b, 0, blockSize); + int numBytesRead = blockStream.read(b, 0, blockSize); + assertEquals(blockSize, numBytesRead); matchWithInputData(b, 0, blockSize); } @@ -174,8 +175,8 @@ public void testPartialChunkRead() throws Exception { int len = blockSize / 2; byte[] b = new byte[len]; - blockStream.read(b, 0, len); - + int numBytesRead = blockStream.read(b, 0, len); + assertEquals(blockSize, numBytesRead); matchWithInputData(b, 0, len); // To read block data from index 0 to 225 (len = 225), we need to read @@ -200,7 +201,8 @@ public void testSeek() throws Exception { // copying chunk data from index 20 to 59 into the buffers (checksum // boundaries). byte[] b = new byte[30]; - blockStream.read(b, 0, 30); + int numBytesRead = blockStream.read(b, 0, 30); + assertEquals(blockSize, numBytesRead); matchWithInputData(b, 25, 30); matchWithInputData(blockStream.getReadByteBuffers(), 20, 40); @@ -222,7 +224,8 @@ public void testSeek() throws Exception { // released and hence chunkPosition updated with current position of chunk. seekAndVerify(25); b = new byte[15]; - blockStream.read(b, 0, 15); + numBytesRead = blockStream.read(b, 0, 15); + assertEquals(blockSize, numBytesRead); matchWithInputData(b, 25, 15); } @@ -231,19 +234,22 @@ public void testSeekAndRead() throws Exception { // Seek to a position and read data seekAndVerify(50); byte[] b1 = new byte[20]; - blockStream.read(b1, 0, 20); + int numBytesRead = blockStream.read(b1, 0, 20); + assertEquals(20, numBytesRead); matchWithInputData(b1, 50, 20); // Next read should start from the position of the last read + 1 i.e. 70 byte[] b2 = new byte[20]; - blockStream.read(b2, 0, 20); + numBytesRead = blockStream.read(b2, 0, 20); + assertEquals(20, numBytesRead); matchWithInputData(b2, 70, 20); } @Test public void testUnbuffered() throws Exception { byte[] b1 = new byte[20]; - blockStream.read(b1, 0, 20); + int numBytesRead = blockStream.read(b1, 0, 20); + assertEquals(20, numBytesRead); matchWithInputData(b1, 0, 20); blockStream.unbuffer(); @@ -252,7 +258,8 @@ public void testUnbuffered() throws Exception { // Next read should start from the position of the last read + 1 i.e. 20 byte[] b2 = new byte[20]; - blockStream.read(b2, 0, 20); + numBytesRead = blockStream.read(b2, 0, 20); + assertEquals(20, numBytesRead); matchWithInputData(b2, 20, 20); } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index 166a159092e5..d8753acd9880 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -308,7 +308,7 @@ public void testVolumeSetInKeyValueHandler() throws Exception { volumeSet, metrics, c -> { }); assertEquals("org.apache.hadoop.ozone.container.common" + - ".volume.CapacityVolumeChoosingPolicy", + ".volume.CapacityVolumeChoosingPolicy", keyValueHandler.getVolumeChoosingPolicyForTesting() .getClass().getName()); From 200da6de10534e6bd9d185ae9106d4c8875fd4fd Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 17 Oct 2024 01:57:07 +0800 Subject: [PATCH 058/114] address comments --- .../hadoop/hdds/scm/storage/TestStreamBlockInput.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java index 7bbeb1fc6485..8f80473dff8c 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java @@ -176,7 +176,7 @@ public void testPartialChunkRead() throws Exception { byte[] b = new byte[len]; int numBytesRead = blockStream.read(b, 0, len); - assertEquals(blockSize, numBytesRead); + assertEquals(len, numBytesRead); matchWithInputData(b, 0, len); // To read block data from index 0 to 225 (len = 225), we need to read @@ -202,7 +202,7 @@ public void testSeek() throws Exception { // boundaries). byte[] b = new byte[30]; int numBytesRead = blockStream.read(b, 0, 30); - assertEquals(blockSize, numBytesRead); + assertEquals(30, numBytesRead); matchWithInputData(b, 25, 30); matchWithInputData(blockStream.getReadByteBuffers(), 20, 40); @@ -225,7 +225,7 @@ public void testSeek() throws Exception { seekAndVerify(25); b = new byte[15]; numBytesRead = blockStream.read(b, 0, 15); - assertEquals(blockSize, numBytesRead); + assertEquals(15, numBytesRead); matchWithInputData(b, 25, 15); } From 0814a229cea3aaf836d7d858e9ce90dcbc3dcaac Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 18 Oct 2024 18:49:32 +0800 Subject: [PATCH 059/114] address comments --- .../apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java | 1 + 1 file changed, 1 insertion(+) 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 12e986126bdc..e9b159836c5b 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 @@ -1394,6 +1394,7 @@ public ContainerCommandResponseProto streamDataReadOnly( ContainerProtos.ChunkInfo.newBuilder(chunk) .setOffset(adjustedChunkOffset) .setLen(adjustedChunkLen).build()); + BlockUtils.verifyReplicaIdx(kvContainer, blockID); data = getChunkManager().readChunk( kvContainer, blockID, chunkInfo, dispatcherContext); From e040768e845a1272ec01be30615a23fa60a0c48f Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 18 Oct 2024 19:06:58 +0800 Subject: [PATCH 060/114] address comment --- .../apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java | 1 + 1 file changed, 1 insertion(+) 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 e9b159836c5b..a4aa6dce080d 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 @@ -1395,6 +1395,7 @@ public ContainerCommandResponseProto streamDataReadOnly( .setOffset(adjustedChunkOffset) .setLen(adjustedChunkLen).build()); BlockUtils.verifyReplicaIdx(kvContainer, blockID); + BlockUtils.verifyBCSId(kvContainer, blockID); data = getChunkManager().readChunk( kvContainer, blockID, chunkInfo, dispatcherContext); From 398692e51bc16b181ed61b4423e914c4fb4a7f06 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 15 Nov 2024 11:28:39 +0800 Subject: [PATCH 061/114] fix DummyStreamBlockInput --- .../hadoop/hdds/scm/storage/DummyStreamBlockInput.java | 8 ++++---- .../hadoop/hdds/scm/storage/TestStreamBlockInput.java | 6 ++++++ 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java index 596a15f7b6e0..6d007eb91108 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java @@ -106,10 +106,11 @@ protected long readData(long offset, long len) { long remainingToRead = Math.min(chunkLen, len); if (isVerifyChecksum()) { if (len < chunkLen) { - ChecksumData checksumData = ChecksumData.getFromProtoBuf( + final ChecksumData checksumData = ChecksumData.getFromProtoBuf( chunkInfo.getChecksumData()); - int bytesPerChecksum = checksumData.getBytesPerChecksum(); - remainingToRead = (len / bytesPerChecksum + 1) * bytesPerChecksum; + final long endByteIndex = len - 1; + final int bytesPerChecksum = checksumData.getBytesPerChecksum(); + remainingToRead = (endByteIndex / bytesPerChecksum + 1) * bytesPerChecksum; } else { remainingToRead = chunkLen; } @@ -122,7 +123,6 @@ protected long readData(long offset, long len) { } else { bufferLen = bufferCapacity; } - System.out.println(bufferLen); ByteString byteString = ByteString.copyFrom(chunkDataMap.get(chunks.get(chunkIndex).getChunkName()), (int) chunkOffset, (int) bufferLen); diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java index 8f80473dff8c..752fb37e5918 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java @@ -243,6 +243,12 @@ public void testSeekAndRead() throws Exception { numBytesRead = blockStream.read(b2, 0, 20); assertEquals(20, numBytesRead); matchWithInputData(b2, 70, 20); + + byte[] b3 = new byte[20]; + seekAndVerify(80); + numBytesRead = blockStream.read(b3, 0, 20); + assertEquals(20, numBytesRead); + matchWithInputData(b3, 80, 20); } @Test From fead0b7562d560737fa389f3e28309fbcec6bb63 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 15 Nov 2024 12:00:13 +0800 Subject: [PATCH 062/114] rmove StreamData type --- .../java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java | 4 ++-- .../src/main/proto/DatanodeClientProtocol.proto | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 21480efb51ac..74416992ab77 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -652,7 +652,7 @@ public void onNext( } else { future.complete( ContainerCommandResponseProto.newBuilder(responseProto) - .setCmdType(Type.StreamRead).build()); + .setCmdType(Type.ReadBlock).build()); } } @@ -669,7 +669,7 @@ public void onError(Throwable t) { public void onCompleted() { if (readBlock.getReadChunkCount() > 0) { future.complete(response.setReadBlock(readBlock) - .setCmdType(Type.StreamRead).setResult(Result.SUCCESS).build()); + .setCmdType(Type.ReadBlock).setResult(Result.SUCCESS).build()); } if (!future.isDone()) { future.completeExceptionally(new IOException( diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index a085b4e2229d..20a6a277dd2b 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -109,7 +109,6 @@ enum Type { FinalizeBlock = 21; Echo = 22; ReadBlock = 23; - StreamRead = 24; } From 2fb28519dc09052b69c184ee0459e71c64f39ddb Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 15 Nov 2024 12:16:02 +0800 Subject: [PATCH 063/114] fix checkstyle --- .../apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java index 6d007eb91108..894529c856ef 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java @@ -106,8 +106,7 @@ protected long readData(long offset, long len) { long remainingToRead = Math.min(chunkLen, len); if (isVerifyChecksum()) { if (len < chunkLen) { - final ChecksumData checksumData = ChecksumData.getFromProtoBuf( - chunkInfo.getChecksumData()); + final ChecksumData checksumData = ChecksumData.getFromProtoBuf(chunkInfo.getChecksumData()); final long endByteIndex = len - 1; final int bytesPerChecksum = checksumData.getBytesPerChecksum(); remainingToRead = (endByteIndex / bytesPerChecksum + 1) * bytesPerChecksum; From 1968756ec5a8e1d12e7da6c438a63cff75bb89ae Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 26 Nov 2024 02:46:27 +0800 Subject: [PATCH 064/114] Merge branch 'master' into HDDS-10338 --- .../hadoop/ozone/container/keyvalue/TestKeyValueHandler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index 26f57f866206..f6b36559a7b9 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -86,6 +86,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.io.TempDir; +import org.mockito.MockedStatic; import org.mockito.Mockito; import static org.mockito.Mockito.clearInvocations; From 5c70fd4847e485936ea323c08e02d3b93e889bd8 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 28 Nov 2024 03:52:34 +0800 Subject: [PATCH 065/114] fix verify checksum --- .../container/keyvalue/KeyValueHandler.java | 20 +++++++++++-------- .../keyvalue/TestKeyValueHandler.java | 2 +- .../client/rpc/read/TestStreamBlockInput.java | 6 +++--- 3 files changed, 16 insertions(+), 12 deletions(-) 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 e4d617f0d402..93274dee0d74 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 @@ -1375,10 +1375,14 @@ public ContainerCommandResponseProto streamDataReadOnly( List chunkInfos = blockData.getChunks(); long blockOffset = 0; int chunkIndex = -1; + long chunkOffset = 0; + long offset = readBlock.getOffset(); for (int i = 0; i < chunkInfos.size(); i++) { - blockOffset += chunkInfos.get(i).getLen(); - if (blockOffset > readBlock.getOffset()) { + final long chunkLen = chunkInfos.get(i).getLen(); + blockOffset += chunkLen; + if (blockOffset > offset) { chunkIndex = i; + chunkOffset = offset - blockOffset + chunkLen; break; } } @@ -1389,21 +1393,21 @@ public ContainerCommandResponseProto streamDataReadOnly( } ChunkBuffer data; - long offset = readBlock.getOffset(); + long len = readBlock.getLen(); long adjustedChunkOffset, adjustedChunkLen; do { ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); if (readBlock.getVerifyChecksum()) { Pair adjustedOffsetAndLength = - computeChecksumBoundaries(chunk, offset, len); + computeChecksumBoundaries(chunk, chunkOffset, len); adjustedChunkOffset = adjustedOffsetAndLength.getLeft(); adjustedChunkLen = adjustedOffsetAndLength.getRight(); adjustedChunkOffset += chunk.getOffset(); } else { - adjustedChunkOffset = offset; + adjustedChunkOffset = chunkOffset; adjustedChunkLen = Math.min( - chunk.getLen() + chunk.getOffset() - offset, len); + chunk.getLen() + chunk.getOffset() - chunkOffset, len); } ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf( @@ -1421,8 +1425,8 @@ public ContainerCommandResponseProto streamDataReadOnly( blockData.getProtoBufMessage().getBlockID(), chunkInfo.getProtoBufMessage(), data, byteBufferToByteString)); - len -= adjustedChunkLen + adjustedChunkOffset - offset; - offset = adjustedChunkOffset + adjustedChunkLen; + len -= adjustedChunkLen + adjustedChunkOffset - chunkOffset; + chunkOffset = 0; chunkIndex++; } while (len > 0); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index f6b36559a7b9..fceefa388587 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -502,7 +502,7 @@ public void testReadBlock() throws IOException { for (int i = 0; i < 4; i++) { chunkInfoList.add(ContainerProtos.ChunkInfo .newBuilder() - .setOffset(CHUNK_SIZE * i) + .setOffset(0) .setLen(CHUNK_SIZE) .setChecksumData( ChecksumData.newBuilder().setBytesPerChecksum((int) BYTES_PER_CHECKSUM) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java index 9243c5780ace..282c9804174a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java @@ -54,7 +54,7 @@ void testAll(ContainerLayoutVersion layout) throws Exception { try (OzoneClient client = cluster.newClient()) { TestBucket bucket = TestBucket.newBuilder(client).build(); - testChunkReadBuffers(bucket); + testBlockReadBuffers(bucket); testBufferRelease(bucket); testCloseReleasesBuffers(bucket); } @@ -63,10 +63,10 @@ void testAll(ContainerLayoutVersion layout) throws Exception { /** - * Test to verify that data read from chunks is stored in a list of buffers + * Test to verify that data read from blocks is stored in a list of buffers * with max capacity equal to the bytes per checksum. */ - private void testChunkReadBuffers(TestBucket bucket) throws Exception { + private void testBlockReadBuffers(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); From bc804ffecdc5233a8440ade168928187beb47f1f Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 28 Nov 2024 20:54:21 +0800 Subject: [PATCH 066/114] no need to compute startByteIndex --- .../hadoop/ozone/container/keyvalue/KeyValueHandler.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 93274dee0d74..cc7e1a8b3212 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 @@ -1445,8 +1445,7 @@ private Pair computeChecksumBoundaries( ContainerProtos.ChunkInfo chunkInfo, long startByteIndex, long dataLen) { int bytesPerChecksum = chunkInfo.getChecksumData().getBytesPerChecksum(); - long chunkOffset = chunkInfo.getOffset(); - startByteIndex = startByteIndex - chunkOffset; + // index of the last byte to be read from chunk, inclusively. final long endByteIndex = startByteIndex + dataLen - 1; From 0e4e41e032540c3d48888198fff7612ca6fa0e92 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 3 Dec 2024 18:40:54 +0800 Subject: [PATCH 067/114] address comments --- .../apache/hadoop/hdds/scm/OzoneClientConfig.java | 2 +- .../apache/hadoop/hdds/scm/XceiverClientGrpc.java | 4 ++-- .../hadoop/hdds/scm/storage/BlockInputStream.java | 1 - .../hdds/scm/storage/ContainerProtocolCalls.java | 3 ++- .../transport/server/GrpcXceiverService.java | 2 +- .../ozone/container/keyvalue/KeyValueHandler.java | 14 +++++++------- .../src/main/proto/DatanodeClientProtocol.proto | 2 ++ 7 files changed, 15 insertions(+), 13 deletions(-) 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 738df5a62b15..328b5101a919 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 @@ -133,7 +133,7 @@ public enum ChecksumCombineMode { type = ConfigType.BOOLEAN, description = "Allow ReadBlock to stream all the readChunk in one request.", tags = ConfigTag.CLIENT) - private boolean streamReadBlock = true; + private boolean streamReadBlock = false; @Config(key = "max.retries", defaultValue = "5", diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index d615f9eed1bc..61e2cd8f5953 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -662,7 +662,7 @@ public void onError(Throwable t) { metrics.decrPendingContainerOpsMetrics(cmdType); metrics.addContainerOpsLatency( cmdType, System.currentTimeMillis() - requestTime); - + semaphore.release(); } @Override @@ -679,11 +679,11 @@ public void onCompleted() { metrics.decrPendingContainerOpsMetrics(cmdType); metrics.addContainerOpsLatency( cmdType, System.currentTimeMillis() - requestTime); + semaphore.release(); } }); requestObserver.onNext(request); requestObserver.onCompleted(); - semaphore.release(); return new XceiverClientReply(future); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index 447f2baedbf2..d6353be9d22d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -124,7 +124,6 @@ public BlockInputStream( OzoneClientConfig config) throws IOException { this.blockInfo = blockInfo; this.blockID = blockInfo.getBlockID(); - LOG.debug("Initializing BlockInputStream for block {}", blockID); this.length = blockInfo.getLength(); setPipeline(pipeline); tokenRef.set(token); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 5bc7a3459e65..822c23e18017 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -880,6 +880,8 @@ public static List toValidatorList(Validator validator) { * Calls the container protocol to read a chunk. * * @param xceiverClient client to perform call + * @param offset offset where block starts + * @param len length of data to read * @param blockID ID of the block * @param validators functions to validate the response * @param token a token for this block (may be null) @@ -903,7 +905,6 @@ public static ContainerProtos.ReadBlockResponseProto readBlock( builder.setEncodedToken(token.encodeToUrlString()); } - return tryEachDatanode(xceiverClient.getPipeline(), d -> readBlock(xceiverClient, validators, blockID, builder, readBlockRequest, d, replicaIndexes), diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java index b89ec410f98c..3913d724665d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java @@ -125,7 +125,7 @@ public void onError(Throwable t) { @Override public void onCompleted() { if (isClosed.compareAndSet(false, true)) { - LOG.info("ContainerCommand send completed"); + LOG.debug("ContainerCommand send completed"); responseObserver.onCompleted(); } } 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 cc7e1a8b3212..bdf060619cad 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 @@ -1355,14 +1355,14 @@ public ContainerCommandResponseProto streamDataReadOnly( DispatcherContext dispatcherContext, StreamObserver streamObserver) { ContainerCommandResponseProto responseProto = null; - try { - if (!request.hasReadBlock()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Malformed Read Block request. trace ID: {}", - request.getTraceID()); - } - return malformedRequest(request); + 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( diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 20a6a277dd2b..4d578fb78851 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -77,6 +77,8 @@ package hadoop.hdds.datanode; * 18. CopyContainer - Copies a container from a remote machine. * * 19. FinalizeBlock - Finalize block request from client. + * + * 20. ReadBlock - Allows us to read a block. */ enum Type { From f8e5f2818f3eb320d1e78e315c39b5cc0a6aef90 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 3 Dec 2024 18:57:37 +0800 Subject: [PATCH 068/114] address comments --- ...Input.java => StreamBlockInputStream.java} | 36 +++++++++---------- .../io/BlockInputStreamFactoryImpl.java | 4 +-- ....java => DummyStreamBlockInputStream.java} | 6 ++-- ...t.java => TestStreamBlockInputStream.java} | 8 ++--- .../io/TestBlockInputStreamFactoryImpl.java | 4 +-- .../container/common/impl/HddsDispatcher.java | 2 +- .../interfaces/ContainerDispatcher.java | 4 +-- .../container/common/interfaces/Handler.java | 2 +- .../container/keyvalue/KeyValueHandler.java | 2 +- .../keyvalue/TestKeyValueHandler.java | 10 +++--- ...t.java => TestStreamBlockInputStream.java} | 28 +++++++-------- 11 files changed, 53 insertions(+), 53 deletions(-) rename hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/{StreamBlockInput.java => StreamBlockInputStream.java} (96%) rename hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/{DummyStreamBlockInput.java => DummyStreamBlockInputStream.java} (96%) rename hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/{TestStreamBlockInput.java => TestStreamBlockInputStream.java} (97%) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/{TestStreamBlockInput.java => TestStreamBlockInputStream.java} (92%) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java similarity index 96% rename from hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java rename to hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index 1eab3d17ce7b..61907d7a256e 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -66,10 +66,10 @@ * An {@link java.io.InputStream} called from KeyInputStream to read a block from the * container. */ -public class StreamBlockInput extends BlockExtendedInputStream +public class StreamBlockInputStream extends BlockExtendedInputStream implements Seekable, CanUnbuffer, ByteBufferReadable { private static final Logger LOG = - LoggerFactory.getLogger(StreamBlockInput.class); + LoggerFactory.getLogger(StreamBlockInputStream.class); private final BlockID blockID; private final long blockLength; private final AtomicReference pipelineRef = @@ -79,11 +79,11 @@ public class StreamBlockInput extends BlockExtendedInputStream private XceiverClientFactory xceiverClientFactory; private XceiverClientSpi xceiverClient; - private List bufferoffsets; + private List bufferOffsets; private int bufferIndex; private long blockPosition = -1; private List buffers; - // Checks if the StreamBlockInput has already read data from the container. + // Checks if the StreamBlockInputStream has already read data from the container. private boolean allocated = false; private long bufferOffsetWrtBlockData; private long buffersSize; @@ -95,14 +95,14 @@ public class StreamBlockInput extends BlockExtendedInputStream private int retries; - public StreamBlockInput( + public StreamBlockInputStream( BlockID blockID, long length, Pipeline pipeline, Token token, XceiverClientFactory xceiverClientFactory, Function refreshFunction, OzoneClientConfig config) throws IOException { this.blockID = blockID; - LOG.debug("Initializing StreamBlockInput for block {}", blockID); + LOG.debug("Initializing StreamBlockInputStream for block {}", blockID); this.blockLength = length; setPipeline(pipeline); tokenRef.set(token); @@ -138,7 +138,7 @@ public synchronized long getPos() { if (buffersHaveData()) { // BufferOffset w.r.t to BlockData + BufferOffset w.r.t buffers + // Position of current Buffer - return bufferOffsetWrtBlockData + bufferoffsets.get(bufferIndex) + + return bufferOffsetWrtBlockData + bufferOffsets.get(bufferIndex) + buffers.get(bufferIndex).position(); } if (allocated && !dataRemainingInBlock()) { @@ -411,7 +411,7 @@ private boolean buffersHavePosition(long pos) { // Released buffers should not be considered when checking if position // is available return pos >= bufferOffsetWrtBlockData + - bufferoffsets.get(0) && + bufferOffsets.get(0) && pos < bufferOffsetWrtBlockData + buffersSize; } return false; @@ -434,13 +434,13 @@ private void adjustBufferPosition(long bufferPosition) { // The bufferPosition is w.r.t the current buffers. // Adjust the bufferIndex and position to the seeked bufferPosition. if (bufferIndex >= buffers.size()) { - bufferIndex = Collections.binarySearch(bufferoffsets, bufferPosition); - } else if (bufferPosition < bufferoffsets.get(bufferIndex)) { + bufferIndex = Collections.binarySearch(bufferOffsets, bufferPosition); + } else if (bufferPosition < bufferOffsets.get(bufferIndex)) { bufferIndex = Collections.binarySearch( - bufferoffsets.subList(0, bufferIndex), bufferPosition); - } else if (bufferPosition >= bufferoffsets.get(bufferIndex) + + bufferOffsets.subList(0, bufferIndex), bufferPosition); + } else if (bufferPosition >= bufferOffsets.get(bufferIndex) + buffers.get(bufferIndex).capacity()) { - bufferIndex = Collections.binarySearch(bufferoffsets.subList( + bufferIndex = Collections.binarySearch(bufferOffsets.subList( bufferIndex + 1, buffers.size()), bufferPosition); } // bufferIndex is negative if bufferPosition isn't found in bufferOffsets @@ -450,7 +450,7 @@ private void adjustBufferPosition(long bufferPosition) { } buffers.get(bufferIndex).position( - (int) (bufferPosition - bufferoffsets.get(bufferIndex))); + (int) (bufferPosition - bufferOffsets.get(bufferIndex))); // Reset buffers > bufferIndex to position 0. We do this to reset any // previous reads/ seeks which might have updated any buffer position. @@ -497,9 +497,9 @@ private synchronized void readDataFromContainer(int len) throws IOException { bufferOffsetWrtBlockData = readData(startByteIndex, len); long tempOffset = 0L; buffersSize = 0L; - bufferoffsets = new ArrayList<>(buffers.size()); + bufferOffsets = new ArrayList<>(buffers.size()); for (ByteBuffer buffer : buffers) { - bufferoffsets.add(tempOffset); + bufferOffsets.add(tempOffset); tempOffset += buffer.limit(); buffersSize += buffer.limit(); @@ -598,13 +598,13 @@ private void releaseBuffers(int releaseUptoBufferIndex) { // buffers. This should be done so that getPos() can return the current // block position blockPosition = bufferOffsetWrtBlockData + - bufferoffsets.get(releaseUptoBufferIndex) + + bufferOffsets.get(releaseUptoBufferIndex) + buffers.get(releaseUptoBufferIndex).capacity(); // Release all the buffers releaseBuffers(); } else { buffers = buffers.subList(releaseUptoBufferIndex + 1, buffersLen); - bufferoffsets = bufferoffsets.subList( + bufferOffsets = bufferOffsets.subList( releaseUptoBufferIndex + 1, buffersLen); bufferIndex = 0; } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 3b1eaf328bcb..ac7e80d56873 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; -import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; +import org.apache.hadoop.hdds.scm.storage.StreamBlockInputStream; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.io.ElasticByteBufferPool; @@ -90,7 +90,7 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, blockInfo, xceiverFactory, refreshFunction, ecBlockStreamFactory, config); } else if (config.isStreamReadBlock() && allDataNodesSupportStreamBlock(pipeline)) { - return new StreamBlockInput( + return new StreamBlockInputStream( blockInfo.getBlockID(), blockInfo.getLength(), pipeline, token, xceiverFactory, refreshFunction, config); } else { diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInputStream.java similarity index 96% rename from hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java rename to hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInputStream.java index 894529c856ef..dfd1cdc0992a 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInputStream.java @@ -36,9 +36,9 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; /** - * A dummy StreamBlockInput to mock read block call to DN. + * A dummy StreamBlockInputStream to mock read block call to DN. */ -class DummyStreamBlockInput extends StreamBlockInput { +class DummyStreamBlockInputStream extends StreamBlockInputStream { private final List readByteBuffers = new ArrayList<>(); private final List chunks; @@ -46,7 +46,7 @@ class DummyStreamBlockInput extends StreamBlockInput { private final Map chunkDataMap; @SuppressWarnings("parameternumber") - DummyStreamBlockInput( + DummyStreamBlockInputStream( BlockID blockId, long blockLen, Pipeline pipeline, diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java similarity index 97% rename from hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java rename to hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 752fb37e5918..03009f7a4619 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInput.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -49,15 +49,15 @@ import static org.mockito.Mockito.mock; /** - * Tests for {@link TestStreamBlockInput}'s functionality. + * Tests for {@link TestStreamBlockInputStream}'s functionality. */ -public class TestStreamBlockInput { +public class TestStreamBlockInputStream { private int blockSize; private static final int CHUNK_SIZE = 100; private static final int BYTES_PER_CHECKSUM = 20; private static final Random RANDOM = new Random(); private static final AtomicLong CONTAINER_ID = new AtomicLong(); - private DummyStreamBlockInput blockStream; + private DummyStreamBlockInputStream blockStream; private byte[] blockData; private List chunks; private Map chunkDataMap; @@ -78,7 +78,7 @@ public void setup() throws Exception { createChunkList(5); Pipeline pipeline = MockPipeline.createSingleNodePipeline(); - blockStream = new DummyStreamBlockInput(blockID, blockSize, pipeline, + blockStream = new DummyStreamBlockInputStream(blockID, blockSize, pipeline, null, null, refreshFunction, clientConfig, chunks, chunkDataMap); } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java index dbb2689b76cd..804ba1c0c51c 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream; import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; -import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; +import org.apache.hadoop.hdds.scm.storage.StreamBlockInputStream; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -73,7 +73,7 @@ public void testNonECGivesBlockInputStream(boolean streamReadBlockEnabled) throw blockInfo.getToken(), null, null, clientConfig); if (streamReadBlockEnabled) { - assertInstanceOf(StreamBlockInput.class, stream); + assertInstanceOf(StreamBlockInputStream.class, stream); } else { assertInstanceOf(BlockInputStream.class, stream); } 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 a81b10934a27..c31a7117a380 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 @@ -861,7 +861,7 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, ContainerProtos.Result.CONTAINER_INTERNAL_ERROR); } perf.appendPreOpLatencyMs(Time.monotonicNow() - startTime); - responseProto = handler.streamDataReadOnly( + responseProto = handler.readBlock( msg, (KeyValueContainer) container, dispatcherContext, streamObserver); long oPLatencyMS = Time.monotonicNow() - startTime; metrics.incContainerOpsLatencies(cmdType, oPLatencyMS); 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 4664cf4f365d..cfd2a452a001 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 @@ -96,13 +96,13 @@ default StateMachine.DataChannel getStreamDataChannel( } /** - * When reading data form client using stream, get StreamDataChannel. + * When reading data form client by streaming chunks. */ default void streamDataReadOnly( ContainerCommandRequestProto msg, StreamObserver streamObserver, DispatcherContext dispatcherContext) { throw new UnsupportedOperationException( - "getStreamDataChannel not supported."); + "streamDataReadOnly 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 f80a26d8df73..8f5e83fdd60a 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 @@ -225,7 +225,7 @@ public void setClusterID(String clusterID) { this.clusterId = clusterID; } - public abstract ContainerCommandResponseProto streamDataReadOnly( + public abstract ContainerCommandResponseProto readBlock( ContainerCommandRequestProto msg, KeyValueContainer container, DispatcherContext dispatcherContext, StreamObserver streamObserver); 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 bdf060619cad..fb897cba0c82 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 @@ -1350,7 +1350,7 @@ public void deleteUnreferenced(Container container, long localID) } @Override - public ContainerCommandResponseProto streamDataReadOnly( + public ContainerCommandResponseProto readBlock( ContainerCommandRequestProto request, KeyValueContainer kvContainer, DispatcherContext dispatcherContext, StreamObserver streamObserver) { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index fceefa388587..2c0fb0ec4ce8 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -522,11 +522,11 @@ public void testReadBlock() throws IOException { when(chunkManager.readChunk(any(), any(), any(), any())) .thenReturn(data); - testStreamDataReadOnly(0, 1, keyValueHandler, dispatcherContext, + testReadBlock(0, 1, keyValueHandler, dispatcherContext, streamObserver, container); - testStreamDataReadOnly(0, CHUNK_SIZE + 1, keyValueHandler, dispatcherContext, + testReadBlock(0, CHUNK_SIZE + 1, keyValueHandler, dispatcherContext, streamObserver, container); - testStreamDataReadOnly(CHUNK_SIZE / 2, 2 * CHUNK_SIZE, keyValueHandler, dispatcherContext, + testReadBlock(CHUNK_SIZE / 2, 2 * CHUNK_SIZE, keyValueHandler, dispatcherContext, streamObserver, container); } } @@ -549,12 +549,12 @@ private static ContainerCommandRequestProto readBlockRequest( .build(); } - private static void testStreamDataReadOnly( + private static void testReadBlock( long offset, long length, KeyValueHandler keyValueHandler, DispatcherContext dispatcherContext, StreamObserver streamObserver, KeyValueContainer container) { int responseCount = (int) (((offset + length - 1) / CHUNK_SIZE) + 1 - (offset / CHUNK_SIZE)); ContainerCommandRequestProto requestProto = readBlockRequest(offset, length); - keyValueHandler.streamDataReadOnly(requestProto, container, dispatcherContext, streamObserver); + keyValueHandler.readBlock(requestProto, container, dispatcherContext, streamObserver); verify(streamObserver, times(responseCount)).onNext(any()); clearInvocations(streamObserver); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java similarity index 92% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 282c9804174a..d63c0664b726 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInput.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -17,7 +17,7 @@ package org.apache.hadoop.ozone.client.rpc.read; -import org.apache.hadoop.hdds.scm.storage.StreamBlockInput; +import org.apache.hadoop.hdds.scm.storage.StreamBlockInputStream; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.io.KeyInputStream; @@ -39,9 +39,9 @@ import static org.junit.jupiter.api.Assertions.assertNull; /** - * Tests {@link StreamBlockInput}. + * Tests {@link StreamBlockInputStream}. */ -public class TestStreamBlockInput { +public class TestStreamBlockInputStream { /** * Run the tests as a single test method to avoid needing a new mini-cluster * for each test. @@ -73,8 +73,8 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - StreamBlockInput block0Stream = - (StreamBlockInput) keyInputStream.getPartStreams().get(0); + StreamBlockInputStream block0Stream = + (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); // To read 1 byte of chunk data, ChunkInputStream should get one full @@ -130,8 +130,8 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { bucket.writeRandomBytes(keyName, CHUNK_SIZE); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - StreamBlockInput block0Stream = - (StreamBlockInput) keyInputStream.getPartStreams().get(0); + StreamBlockInputStream block0Stream = + (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); readDataFromChunk(block0Stream, 0, 1); assertNotNull(block0Stream.getCachedBuffers()); @@ -152,8 +152,8 @@ private void testBufferRelease(TestBucket bucket) throws Exception { try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - StreamBlockInput block0Stream = - (StreamBlockInput) keyInputStream.getPartStreams().get(0); + StreamBlockInputStream block0Stream = + (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); // Read checksum boundary - 1 bytes of data int readDataLen = BYTES_PER_CHECKSUM - 1; @@ -201,18 +201,18 @@ private void testBufferRelease(TestBucket bucket) throws Exception { } } - private byte[] readDataFromChunk(StreamBlockInput streamBlockInput, + private byte[] readDataFromChunk(StreamBlockInputStream streamBlockInputStream, int offset, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; - streamBlockInput.seek(offset); - streamBlockInput.read(readData, 0, readDataLength); + streamBlockInputStream.seek(offset); + streamBlockInputStream.read(readData, 0, readDataLength); return readData; } - private byte[] readDataFromChunk(StreamBlockInput streamBlockInput, + private byte[] readDataFromChunk(StreamBlockInputStream streamBlockInputStream, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; - streamBlockInput.read(readData, 0, readDataLength); + streamBlockInputStream.read(readData, 0, readDataLength); return readData; } From 228de8d5d6b1e4113f9ecaeb857e4995c14428af Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Wed, 4 Dec 2024 01:02:22 +0800 Subject: [PATCH 069/114] add read empty block in TestStreamBlockInputStream.java --- .../rpc/read/TestStreamBlockInputStream.java | 38 ++++++++++++++----- 1 file changed, 28 insertions(+), 10 deletions(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index d63c0664b726..4935f76c8899 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -37,6 +37,7 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests {@link StreamBlockInputStream}. @@ -57,6 +58,7 @@ void testAll(ContainerLayoutVersion layout) throws Exception { testBlockReadBuffers(bucket); testBufferRelease(bucket); testCloseReleasesBuffers(bucket); + testReadEmptyBlock(bucket); } } } @@ -85,7 +87,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // Read > checksum boundary of data from chunk0 int readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); - byte[] readData = readDataFromChunk(block0Stream, 0, readDataLen); + byte[] readData = readDataFromBlock(block0Stream, 0, readDataLen); bucket.validateData(inputData, 0, readData); // The first checksum boundary size of data was already existing in the @@ -104,7 +106,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // and the second buffer should have BYTES_PER_CHECKSUM capacity. readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); int offset = 2 * BYTES_PER_CHECKSUM + 1; - readData = readDataFromChunk(block0Stream, offset, readDataLen); + readData = readDataFromBlock(block0Stream, offset, readDataLen); bucket.validateData(inputData, offset, readData); checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, BYTES_PER_CHECKSUM); @@ -113,7 +115,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // Read the full chunk data -1 and verify that all chunk data is read into // buffers. We read CHUNK_SIZE - 1 as otherwise all the buffers will be // released once all chunk data is read. - readData = readDataFromChunk(block0Stream, 0, CHUNK_SIZE - 1); + readData = readDataFromBlock(block0Stream, 0, CHUNK_SIZE - 1); bucket.validateData(inputData, 0, readData); int expectedNumBuffers = CHUNK_SIZE / BYTES_PER_CHECKSUM; checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, BYTES_PER_CHECKSUM); @@ -133,7 +135,7 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { StreamBlockInputStream block0Stream = (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); - readDataFromChunk(block0Stream, 0, 1); + readDataFromBlock(block0Stream, 0, 1); assertNotNull(block0Stream.getCachedBuffers()); block0Stream.close(); @@ -157,7 +159,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // Read checksum boundary - 1 bytes of data int readDataLen = BYTES_PER_CHECKSUM - 1; - byte[] readData = readDataFromChunk(block0Stream, 0, readDataLen); + byte[] readData = readDataFromBlock(block0Stream, 0, readDataLen); bucket.validateData(inputData, 0, readData); // There should be 1 byte of data remaining in the buffer which is not @@ -168,14 +170,14 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // Reading the last byte in the buffer should result in all the buffers // being released. - readData = readDataFromChunk(block0Stream, 1); + readData = readDataFromBlock(block0Stream, 1); bucket.validateData(inputData, readDataLen, readData); assertNull(block0Stream.getCachedBuffers(), "Chunk stream buffers not released after last byte is read"); // Read more data to get the data till the next checksum boundary. readDataLen = BYTES_PER_CHECKSUM / 2; - readData = readDataFromChunk(block0Stream, readDataLen); + readData = readDataFromBlock(block0Stream, readDataLen); // There should be one buffer and the buffer should not be released as // there is data pending to be read from the buffer checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, @@ -188,7 +190,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // checksum boundary). int position = (int) block0Stream.getPos(); readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; - readData = readDataFromChunk(block0Stream, readDataLen); + readData = readDataFromBlock(block0Stream, readDataLen); bucket.validateData(inputData, position, readData); // After reading the remaining data in the buffer, the buffer should be // released and next checksum size of data must be read into the buffers @@ -201,7 +203,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { } } - private byte[] readDataFromChunk(StreamBlockInputStream streamBlockInputStream, + private byte[] readDataFromBlock(StreamBlockInputStream streamBlockInputStream, int offset, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; streamBlockInputStream.seek(offset); @@ -209,7 +211,7 @@ private byte[] readDataFromChunk(StreamBlockInputStream streamBlockInputStream, return readData; } - private byte[] readDataFromChunk(StreamBlockInputStream streamBlockInputStream, + private byte[] readDataFromBlock(StreamBlockInputStream streamBlockInputStream, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; streamBlockInputStream.read(readData, 0, readDataLength); @@ -234,4 +236,20 @@ private void checkBufferSizeAndCapacity(ByteBuffer[] buffers, "ChunkInputStream ByteBuffer capacity is wrong"); } } + + private void testReadEmptyBlock(TestBucket bucket) throws Exception { + String keyName = getNewKeyName(); + int dataLength = 10; + bucket.writeRandomBytes(keyName, 0); + + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + + byte[] readData = new byte[dataLength]; + assertTrue(keyInputStream.getPartStreams().isEmpty()); + keyInputStream.read(readData); + for (int i = 0; i < readData.length; i++) { + assertEquals(readData[i], (byte) 0); + } + } + } } From c24da9b71fef5b8e39fe90b13423e94cd0dc1de6 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Sat, 7 Dec 2024 02:11:39 +0800 Subject: [PATCH 070/114] address comments --- .../apache/hadoop/hdds/scm/XceiverClientGrpc.java | 3 ++- .../hdds/scm/storage/StreamBlockInputStream.java | 11 +---------- .../ozone/client/rpc/read/TestInputStreamBase.java | 5 ----- .../client/rpc/read/TestStreamBlockInputStream.java | 13 +++++++++++-- 4 files changed, 14 insertions(+), 18 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 61e2cd8f5953..f8f99ee8f57f 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -63,6 +63,7 @@ import io.opentracing.Scope; import io.opentracing.Span; import io.opentracing.util.GlobalTracer; +import org.apache.hadoop.util.Time; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; @@ -661,7 +662,7 @@ public void onError(Throwable t) { future.completeExceptionally(t); metrics.decrPendingContainerOpsMetrics(cmdType); metrics.addContainerOpsLatency( - cmdType, System.currentTimeMillis() - requestTime); + cmdType, Time.monotonicNow() - requestTime); semaphore.release(); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index 61907d7a256e..d10b35e8d18e 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -433,16 +433,7 @@ protected boolean buffersAllocated() { private void adjustBufferPosition(long bufferPosition) { // The bufferPosition is w.r.t the current buffers. // Adjust the bufferIndex and position to the seeked bufferPosition. - if (bufferIndex >= buffers.size()) { - bufferIndex = Collections.binarySearch(bufferOffsets, bufferPosition); - } else if (bufferPosition < bufferOffsets.get(bufferIndex)) { - bufferIndex = Collections.binarySearch( - bufferOffsets.subList(0, bufferIndex), bufferPosition); - } else if (bufferPosition >= bufferOffsets.get(bufferIndex) + - buffers.get(bufferIndex).capacity()) { - bufferIndex = Collections.binarySearch(bufferOffsets.subList( - bufferIndex + 1, buffers.size()), bufferPosition); - } + bufferIndex = Collections.binarySearch(bufferOffsets, bufferPosition); // bufferIndex is negative if bufferPosition isn't found in bufferOffsets // count (bufferIndex = -bufferIndex - 2) to get bufferPosition is between which offsets. if (bufferIndex < 0) { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java index ffd3bb821fbe..256148dfb8de 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java @@ -44,15 +44,10 @@ abstract class TestInputStreamBase { protected static MiniOzoneCluster newCluster( ContainerLayoutVersion containerLayout) throws Exception { - return newCluster(containerLayout, false); - } - protected static MiniOzoneCluster newCluster( - ContainerLayoutVersion containerLayout, boolean streamReadBlock) throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); OzoneClientConfig config = conf.getObject(OzoneClientConfig.class); config.setBytesPerChecksum(BYTES_PER_CHECKSUM); - config.setStreamReadBlock(streamReadBlock); conf.setFromObject(config); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 4935f76c8899..e889a7953b69 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -17,9 +17,13 @@ package org.apache.hadoop.ozone.client.rpc.read; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.storage.StreamBlockInputStream; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.io.KeyInputStream; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; @@ -49,10 +53,15 @@ public class TestStreamBlockInputStream { */ @ContainerLayoutTestInfo.ContainerTest void testAll(ContainerLayoutVersion layout) throws Exception { - try (MiniOzoneCluster cluster = newCluster(layout, true)) { + try (MiniOzoneCluster cluster = newCluster(layout)) { cluster.waitForClusterToBeReady(); - try (OzoneClient client = cluster.newClient()) { + OzoneConfiguration conf = cluster.getConf(); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setStreamReadBlock(true); + OzoneConfiguration copy = new OzoneConfiguration(conf); + copy.setFromObject(clientConfig); + try (OzoneClient client = OzoneClientFactory.getRpcClient(copy)) { TestBucket bucket = TestBucket.newBuilder(client).build(); testBlockReadBuffers(bucket); From f457ac278c010777ec585309b1ff639f49352e9e Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Sat, 7 Dec 2024 02:46:35 +0800 Subject: [PATCH 071/114] fix checkstyle --- .../hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index e889a7953b69..1014f31777a8 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.client.rpc.read; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.storage.StreamBlockInputStream; import org.apache.hadoop.ozone.MiniOzoneCluster; From c1fbad2ee36e1e892bf2c5582fb732860fcf300c Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 13 Dec 2024 01:52:57 +0800 Subject: [PATCH 072/114] address comments --- .../apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index d10b35e8d18e..f704a9e864fc 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -102,7 +102,6 @@ public StreamBlockInputStream( Function refreshFunction, OzoneClientConfig config) throws IOException { this.blockID = blockID; - LOG.debug("Initializing StreamBlockInputStream for block {}", blockID); this.blockLength = length; setPipeline(pipeline); tokenRef.set(token); @@ -356,7 +355,7 @@ private void setPipeline(Pipeline pipeline) throws IOException { protected synchronized void checkOpen() throws IOException { if (xceiverClientFactory == null) { - throw new IOException("BlockInputStream has been closed."); + throw new IOException("StreamBlockInputStream has been closed."); } } From ba100c874a32d71cb283b219c97f4dadfd21cbb0 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Fri, 31 Jan 2025 03:04:25 +0800 Subject: [PATCH 073/114] fix conflict --- .../scm/protocolPB/ContainerCommandResponseBuilders.java | 2 +- .../hadoop/ozone/container/keyvalue/KeyValueHandler.java | 2 +- .../ozone/client/rpc/read/TestStreamBlockInputStream.java | 5 +++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index 69d395d4ff08..a4891a1429ea 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -335,7 +335,7 @@ public static ContainerCommandResponseProto getReadChunkResponse( public static ContainerCommandResponseProto getReadBlockResponse( ContainerCommandRequestProto request, DatanodeBlockID blockID, - ChunkInfo chunkInfo, ChunkBuffer data, + ChunkInfo chunkInfo, ChunkBufferToByteString data, Function byteBufferToByteString) { ReadChunkResponseProto.Builder response; 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 ae5c1934d717..c4de9887b5a5 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 @@ -1429,7 +1429,7 @@ public ContainerCommandResponseProto readBlock( dispatcherContext = DispatcherContext.getHandleReadBlock(); } - ChunkBuffer data; + ChunkBufferToByteString data; long len = readBlock.getLen(); long adjustedChunkOffset, adjustedChunkLen; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 1014f31777a8..e22a86100dc5 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -45,14 +45,14 @@ /** * Tests {@link StreamBlockInputStream}. */ -public class TestStreamBlockInputStream { +public class TestStreamBlockInputStream extends TestInputStreamBase { /** * Run the tests as a single test method to avoid needing a new mini-cluster * for each test. */ @ContainerLayoutTestInfo.ContainerTest void testAll(ContainerLayoutVersion layout) throws Exception { - try (MiniOzoneCluster cluster = newCluster(layout)) { + try (MiniOzoneCluster cluster = newCluster()) { cluster.waitForClusterToBeReady(); OzoneConfiguration conf = cluster.getConf(); @@ -61,6 +61,7 @@ void testAll(ContainerLayoutVersion layout) throws Exception { OzoneConfiguration copy = new OzoneConfiguration(conf); copy.setFromObject(clientConfig); try (OzoneClient client = OzoneClientFactory.getRpcClient(copy)) { + updateConfig(layout); TestBucket bucket = TestBucket.newBuilder(client).build(); testBlockReadBuffers(bucket); From 5db133a83b2481545c4138530d4b6fa6e51ce963 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 17 Mar 2025 02:05:48 +0800 Subject: [PATCH 074/114] fix checks --- .../scm/storage/StreamBlockInputStream.java | 2 + .../rpc/read/TestStreamBlockInputStream.java | 49 ++++++------------- 2 files changed, 17 insertions(+), 34 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index 0cf9267a4e78..e8187b158572 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -115,10 +115,12 @@ public StreamBlockInputStream( } + @Override public BlockID getBlockID() { return blockID; } + @Override public long getLength() { return blockLength; } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 08bb7e98a6ce..e7147086823b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -17,11 +17,6 @@ package org.apache.hadoop.ozone.client.rpc.read; -import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.BLOCK_SIZE; -import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.BYTES_PER_CHECKSUM; -import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.CHUNK_SIZE; -import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.getNewKeyName; -import static org.apache.hadoop.ozone.client.rpc.read.TestInputStreamBase.newCluster; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -90,8 +85,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // To read 1 byte of chunk data, ChunkInputStream should get one full // checksum boundary worth of data from Container and store it in buffers. block0Stream.read(new byte[1]); - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, - BYTES_PER_CHECKSUM); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); // Read > checksum boundary of data from chunk0 int readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); @@ -101,23 +95,20 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // The first checksum boundary size of data was already existing in the // ChunkStream buffers. Once that data is read, the next checksum // boundary size of data will be fetched again to read the remaining data. - // Hence there should be 1 checksum boundary size of data stored in the + // Hence, there should be 1 checksum boundary size of data stored in the // ChunkStreams buffers at the end of the read. - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, - BYTES_PER_CHECKSUM); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); // Seek to a position in the third checksum boundary (so that current // buffers do not have the seeked position) and read > BYTES_PER_CHECKSUM // bytes of data. This should result in 2 * BYTES_PER_CHECKSUM amount of // data being read into the buffers. There should be 2 buffers in the - // stream but the the first buffer should be released after it is read + // stream but the first buffer should be released after it is read // and the second buffer should have BYTES_PER_CHECKSUM capacity. - readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); int offset = 2 * BYTES_PER_CHECKSUM + 1; readData = readDataFromBlock(block0Stream, offset, readDataLen); bucket.validateData(inputData, offset, readData); - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, - BYTES_PER_CHECKSUM); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); // Read the full chunk data -1 and verify that all chunk data is read into @@ -125,8 +116,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // released once all chunk data is read. readData = readDataFromBlock(block0Stream, 0, CHUNK_SIZE - 1); bucket.validateData(inputData, 0, readData); - int expectedNumBuffers = CHUNK_SIZE / BYTES_PER_CHECKSUM; - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, BYTES_PER_CHECKSUM); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); // Read the last byte of chunk and verify that the buffers are released. block0Stream.read(new byte[1]); @@ -172,8 +162,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // There should be 1 byte of data remaining in the buffer which is not // yet read. Hence, the buffer should not be released. - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), - 1, BYTES_PER_CHECKSUM); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); assertEquals(1, block0Stream.getCachedBuffers()[0].remaining()); // Reading the last byte in the buffer should result in all the buffers @@ -185,11 +174,9 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // Read more data to get the data till the next checksum boundary. readDataLen = BYTES_PER_CHECKSUM / 2; - readData = readDataFromBlock(block0Stream, readDataLen); // There should be one buffer and the buffer should not be released as // there is data pending to be read from the buffer - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, - BYTES_PER_CHECKSUM); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); ByteBuffer lastCachedBuffer = block0Stream.getCachedBuffers()[0]; assertEquals(BYTES_PER_CHECKSUM - readDataLen, lastCachedBuffer.remaining()); @@ -202,8 +189,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { bucket.validateData(inputData, position, readData); // After reading the remaining data in the buffer, the buffer should be // released and next checksum size of data must be read into the buffers - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers(), 1, - BYTES_PER_CHECKSUM); + checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); // Verify that the previously cached buffer is released by comparing it // with the current cached buffer assertNotEquals(lastCachedBuffer, @@ -229,18 +215,13 @@ private byte[] readDataFromBlock(StreamBlockInputStream streamBlockInputStream, /** * Verify number of buffers and their capacities. * @param buffers chunk stream buffers - * @param expectedNumBuffers expected number of buffers - * @param expectedBufferCapacity expected buffer capacity of unreleased - * buffers */ - private void checkBufferSizeAndCapacity(ByteBuffer[] buffers, - int expectedNumBuffers, - long expectedBufferCapacity) { - assertEquals(expectedNumBuffers, buffers.length, + private void checkBufferSizeAndCapacity(ByteBuffer[] buffers) { + assertEquals(1, buffers.length, "ChunkInputStream does not have expected number of " + "ByteBuffers"); - for (int i = 0; i < buffers.length; i++) { - assertEquals(expectedBufferCapacity, buffers[i].capacity(), + for (ByteBuffer buffer : buffers) { + assertEquals(BYTES_PER_CHECKSUM, buffer.capacity(), "ChunkInputStream ByteBuffer capacity is wrong"); } } @@ -255,8 +236,8 @@ private void testReadEmptyBlock(TestBucket bucket) throws Exception { byte[] readData = new byte[dataLength]; assertTrue(keyInputStream.getPartStreams().isEmpty()); keyInputStream.read(readData); - for (int i = 0; i < readData.length; i++) { - assertEquals(readData[i], (byte) 0); + for (byte b : readData) { + assertEquals(b, (byte) 0); } } } From 2bf716db54cc33455731713746eb120967462287 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 17 Mar 2025 16:05:46 +0800 Subject: [PATCH 075/114] fix checks --- .../hadoop/hdds/scm/storage/TestStreamBlockInputStream.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 29aeeef30bdb..07daabde1833 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -53,7 +53,6 @@ public class TestStreamBlockInputStream { private static final int CHUNK_SIZE = 100; private static final int BYTES_PER_CHECKSUM = 20; private static final Random RANDOM = new Random(); - private static final AtomicLong CONTAINER_ID = new AtomicLong(); private DummyStreamBlockInputStream blockStream; private byte[] blockData; private List chunks; From d53bdcba0ad60f756e25decadf589fb21a84d1a0 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Mon, 17 Mar 2025 16:41:15 +0800 Subject: [PATCH 076/114] fix checkstyle --- .../hadoop/hdds/scm/storage/TestStreamBlockInputStream.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 07daabde1833..43421abdfed9 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -30,7 +30,6 @@ import java.util.List; import java.util.Map; import java.util.Random; -import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ContainerBlockID; From aa51cbb7f96f7bf70b3ee18d74d0d8a51f60239f Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 20 Mar 2025 00:04:20 +0800 Subject: [PATCH 077/114] fix test --- .../hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index e7147086823b..f53ec8783a67 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -174,6 +174,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // Read more data to get the data till the next checksum boundary. readDataLen = BYTES_PER_CHECKSUM / 2; + readDataFromBlock(block0Stream, readDataLen); // There should be one buffer and the buffer should not be released as // there is data pending to be read from the buffer checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); From d39fd6ba41db0fa454c390d9577b0d409df98fe0 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 17 Jun 2025 16:12:22 +0800 Subject: [PATCH 078/114] fix checkstyle --- .../hadoop/hdds/scm/storage/StreamBlockInputStream.java | 2 -- .../hadoop/hdds/scm/storage/TestStreamBlockInputStream.java | 1 - .../ozone/container/keyvalue/TestKeyValueHandler.java | 6 ++---- .../ozone/client/rpc/read/TestStreamBlockInputStream.java | 1 - 4 files changed, 2 insertions(+), 8 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index 1a5bac39e917..3db7fd8f6603 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -92,7 +92,6 @@ public class StreamBlockInputStream extends BlockExtendedInputStream private final RetryPolicy retryPolicy; private int retries; - public StreamBlockInputStream( BlockID blockID, long length, Pipeline pipeline, Token token, @@ -114,7 +113,6 @@ public StreamBlockInputStream( } - @Override public BlockID getBlockID() { return blockID; diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 43421abdfed9..f8f1174704fe 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -62,7 +62,6 @@ public class TestStreamBlockInputStream { private static final String CHUNK_NAME = "chunk-"; private OzoneConfiguration conf = new OzoneConfiguration(); - @BeforeEach public void setup() throws Exception { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index aeadcf6a4e24..28155d4ed9b3 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -93,10 +93,10 @@ import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; import org.apache.ozone.test.GenericTestUtils; +import org.apache.ozone.test.GenericTestUtils.LogCapturer; +import org.apache.hadoop.util.Time; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; -import org.apache.hadoop.util.Time; -import org.apache.ozone.test.GenericTestUtils.LogCapturer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -122,8 +122,6 @@ public class TestKeyValueHandler { private static final String DUMMY_PATH = "dummy/dir/doesnt/exist"; private static final long CHUNK_SIZE = 1024 * 1024; // 1MB private static final long BYTES_PER_CHECKSUM = 256 * 1024; - private final Function byteBufferToByteString = - ByteStringConversion.createByteBufferConversion(true); private HddsDispatcher dispatcher; private KeyValueHandler handler; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index f53ec8783a67..46d39858e500 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -66,7 +66,6 @@ void testAll(ContainerLayoutVersion layout) throws Exception { } } - /** * Test to verify that data read from blocks is stored in a list of buffers * with max capacity equal to the bytes per checksum. From 7656d66f7caf889d2e13718f0c716f45ad2317c4 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 17 Jun 2025 16:22:04 +0800 Subject: [PATCH 079/114] fix find bug --- .../org/apache/hadoop/hdds/scm/XceiverClientGrpc.java | 3 ++- .../client/rpc/read/TestStreamBlockInputStream.java | 10 ++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index cf74d59a2348..8808c205f153 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -648,13 +648,14 @@ public XceiverClientReply sendCommandReadBlock( ContainerProtos.ReadBlockResponseProto.Builder readBlock = ContainerProtos.ReadBlockResponseProto.newBuilder(); checkOpen(dn); + DatanodeID dnId = dn.getID(); Type cmdType = request.getCmdType(); semaphore.acquire(); long requestTime = System.currentTimeMillis(); metrics.incrPendingContainerOpsMetrics(cmdType); final StreamObserver requestObserver = - asyncStubs.get(dn.getUuid()).withDeadlineAfter(timeout, TimeUnit.SECONDS) + asyncStubs.get(dnId).withDeadlineAfter(timeout, TimeUnit.SECONDS) .send(new StreamObserver() { @Override public void onNext( diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 46d39858e500..0ac3c993eb66 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.storage.StreamBlockInputStream; @@ -35,6 +36,7 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; import org.apache.hadoop.ozone.om.TestBucket; +import org.junit.Assert; /** * Tests {@link StreamBlockInputStream}. @@ -83,7 +85,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { // To read 1 byte of chunk data, ChunkInputStream should get one full // checksum boundary worth of data from Container and store it in buffers. - block0Stream.read(new byte[1]); + IOUtils.readFully(block0Stream, new byte[1]); checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); // Read > checksum boundary of data from chunk0 @@ -201,14 +203,14 @@ private byte[] readDataFromBlock(StreamBlockInputStream streamBlockInputStream, int offset, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; streamBlockInputStream.seek(offset); - streamBlockInputStream.read(readData, 0, readDataLength); + IOUtils.readFully(streamBlockInputStream, readData); return readData; } private byte[] readDataFromBlock(StreamBlockInputStream streamBlockInputStream, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; - streamBlockInputStream.read(readData, 0, readDataLength); + IOUtils.readFully(streamBlockInputStream, readData); return readData; } @@ -235,7 +237,7 @@ private void testReadEmptyBlock(TestBucket bucket) throws Exception { byte[] readData = new byte[dataLength]; assertTrue(keyInputStream.getPartStreams().isEmpty()); - keyInputStream.read(readData); + IOUtils.readFully(keyInputStream, readData); for (byte b : readData) { assertEquals(b, (byte) 0); } From b794dd945aff3ad7dc935f3cff6a6dc3662c7b1a Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 17 Jun 2025 16:46:51 +0800 Subject: [PATCH 080/114] fix checkstyle --- .../hadoop/ozone/container/keyvalue/TestKeyValueHandler.java | 5 +---- .../ozone/client/rpc/read/TestStreamBlockInputStream.java | 1 - 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index 28155d4ed9b3..00c3779ac9e1 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -52,7 +52,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.fs.FileUtil; @@ -67,7 +66,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; -import org.apache.hadoop.hdds.scm.ByteStringConversion; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.security.token.TokenVerifier; @@ -92,10 +90,9 @@ import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; +import org.apache.hadoop.util.Time; import org.apache.ozone.test.GenericTestUtils; import org.apache.ozone.test.GenericTestUtils.LogCapturer; -import org.apache.hadoop.util.Time; -import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 0ac3c993eb66..1050b940c394 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -36,7 +36,6 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; import org.apache.hadoop.ozone.om.TestBucket; -import org.junit.Assert; /** * Tests {@link StreamBlockInputStream}. From eec8afb32513f4a5a4189c003053dd4666a1005e Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Tue, 17 Jun 2025 17:08:44 +0800 Subject: [PATCH 081/114] fix find bug --- .../ozone/client/rpc/read/TestStreamBlockInputStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 1050b940c394..85cee43c4a04 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -119,7 +119,7 @@ private void testBlockReadBuffers(TestBucket bucket) throws Exception { checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); // Read the last byte of chunk and verify that the buffers are released. - block0Stream.read(new byte[1]); + IOUtils.readFully(block0Stream, new byte[1]); assertNull(block0Stream.getCachedBuffers(), "ChunkInputStream did not release buffers after reaching EOF."); } @@ -238,7 +238,7 @@ private void testReadEmptyBlock(TestBucket bucket) throws Exception { assertTrue(keyInputStream.getPartStreams().isEmpty()); IOUtils.readFully(keyInputStream, readData); for (byte b : readData) { - assertEquals(b, (byte) 0); + assertEquals((byte) 0, b); } } } From fc9507914180e3cbacade49bc7c8e4e6299398d8 Mon Sep 17 00:00:00 2001 From: chungen0126 Date: Thu, 26 Jun 2025 02:04:40 +0800 Subject: [PATCH 082/114] fix test --- .../ozone/client/rpc/read/TestStreamBlockInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 85cee43c4a04..80ae5118467e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -236,7 +236,7 @@ private void testReadEmptyBlock(TestBucket bucket) throws Exception { byte[] readData = new byte[dataLength]; assertTrue(keyInputStream.getPartStreams().isEmpty()); - IOUtils.readFully(keyInputStream, readData); + IOUtils.read(keyInputStream, readData); for (byte b : readData) { assertEquals((byte) 0, b); } From 3f2396f8b73ec8b3ed3a10aa4563e076e77ec327 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Tue, 23 Sep 2025 15:45:44 +0100 Subject: [PATCH 083/114] Fix compile error due to no scanner message --- .../hadoop/ozone/container/common/impl/HddsDispatcher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c71791efdeba..e555460389de 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 @@ -868,7 +868,7 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, if (responseProto == null) { audit(action, eventType, msg, dispatcherContext, AuditEventStatus.SUCCESS, null); } else { - containerSet.scanContainer(containerID); + containerSet.scanContainer(containerID, "ReadBlock failed " + responseProto.getResult()); audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE, new Exception(responseProto.getMessage())); streamObserver.onNext(responseProto); From 1f90a0a607993d51fedddbfcb8466fa0aa894e80 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Wed, 24 Sep 2025 17:38:19 +0100 Subject: [PATCH 084/114] A ChunkOffsetInBlock to the ChunkInfo proto message to allow checksum offsets to be calculated correctly on the client --- .../hdds/scm/storage/StreamBlockInputStream.java | 10 ++++------ .../ozone/container/common/helpers/ChunkInfo.java | 15 +++++++++++++++ .../ozone/container/keyvalue/KeyValueHandler.java | 15 ++++++++++----- .../src/main/proto/DatanodeClientProtocol.proto | 1 + 4 files changed, 30 insertions(+), 11 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index 3db7fd8f6603..ab4a56ea6219 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -103,8 +103,7 @@ public StreamBlockInputStream( setPipeline(pipeline); tokenRef.set(token); this.xceiverClientFactory = xceiverClientFactory; - this.validators = ContainerProtocolCalls.toValidatorList( - (request, response) -> validateBlock(response)); + this.validators = ContainerProtocolCalls.toValidatorList(this::validateBlock); this.verifyChecksum = config.isChecksumVerify(); this.refreshFunction = refreshFunction; this.retryPolicy = @@ -617,6 +616,7 @@ protected synchronized void releaseClient() { } private void validateBlock( + ContainerProtos.ContainerCommandRequestProto request, ContainerProtos.ContainerCommandResponseProto response ) throws IOException { @@ -640,16 +640,14 @@ private void validateBlock( buffersLen)); } - if (verifyChecksum) { ChecksumData checksumData = ChecksumData.getFromProtoBuf( chunkInfo.getChecksumData()); - int startIndex = (int) readChunk.getChunkData().getOffset() / checksumData.getBytesPerChecksum(); - + int startIndex = (int) (readChunk.getChunkData().getOffset() - readChunk.getChunkData().getChunkOffsetInBlock()) + / checksumData.getBytesPerChecksum(); // ChecksumData stores checksum for each 'numBytesPerChecksum' // number of bytes in a list. Compute the index of the first // checksum to match with the read data - Checksum.verifyChecksum(byteStrings, checksumData, startIndex); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java index 586fe76a49aa..b7fc5629196e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java @@ -37,6 +37,7 @@ public class ChunkInfo { private ChecksumData checksumData; private final Map metadata; private ByteString stripeChecksum; + private long chunkOffsetInBlock; // For older clients reading chunks in V0 version (all read data should // reside in one buffer). This variable should be set to true for older @@ -97,6 +98,10 @@ public static ChunkInfo getFromProtoBuf(ContainerProtos.ChunkInfo info) chunkInfo.setStripeChecksum(info.getStripeChecksum()); } + if (info.hasChunkOffsetInBlock()) { + chunkInfo.setChunkOffsetInBlock(info.getChunkOffsetInBlock()); + } + return chunkInfo; } @@ -119,6 +124,7 @@ public ContainerProtos.ChunkInfo getProtoBufMessage() { } else { builder.setChecksumData(this.checksumData.getProtoBufMessage()); } + builder.setChunkOffsetInBlock(this.chunkOffsetInBlock); for (Map.Entry entry : metadata.entrySet()) { ContainerProtos.KeyValue.Builder keyValBuilder = @@ -183,12 +189,21 @@ public String getMetadata(String key) { return metadata.get(key); } + public void setChunkOffsetInBlock(long chunkOffsetInBlock) { + this.chunkOffsetInBlock = chunkOffsetInBlock; + } + + public long getChunkOffsetInBlock() { + return chunkOffsetInBlock; + } + @Override public String toString() { return "ChunkInfo{" + "chunkName='" + chunkName + ", offset=" + offset + ", len=" + len + + ", chunkOffsetInBlock=" + chunkOffsetInBlock + '}'; } 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 9a5e83aad6b4..1b65db6bfe82 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 @@ -2079,6 +2079,9 @@ public ContainerCommandResponseProto readBlock( long len = readBlock.getLen(); long adjustedChunkOffset, adjustedChunkLen; + // the first chunk length is the largest one. Either the block is less than the chunk size and there is only + // a single chunk, or the block is larger and the chunks are all fixed size (eg 4MB) until the final chunk. + long blockChunkSize = chunkInfos.get(0).getLen(); do { ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); if (readBlock.getVerifyChecksum()) { @@ -2093,10 +2096,12 @@ public ContainerCommandResponseProto readBlock( chunk.getLen() + chunk.getOffset() - chunkOffset, len); } - ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf( - ContainerProtos.ChunkInfo.newBuilder(chunk) - .setOffset(adjustedChunkOffset) - .setLen(adjustedChunkLen).build()); + ContainerProtos.ChunkInfo chunkInfoProto = ContainerProtos.ChunkInfo.newBuilder(chunk) + .setOffset(adjustedChunkOffset) + .setLen(adjustedChunkLen) + .setChunkOffsetInBlock(blockChunkSize * chunkIndex).build(); + ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunkInfoProto); + BlockUtils.verifyReplicaIdx(kvContainer, blockID); BlockUtils.verifyBCSId(kvContainer, blockID); data = getChunkManager().readChunk( @@ -2106,7 +2111,7 @@ public ContainerCommandResponseProto readBlock( streamObserver.onNext( getReadBlockResponse(request, blockData.getProtoBufMessage().getBlockID(), - chunkInfo.getProtoBufMessage(), + chunkInfoProto, data, byteBufferToByteString)); len -= adjustedChunkLen + adjustedChunkOffset - chunkOffset; chunkOffset = 0; diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 71bd653d3495..acb802227645 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -437,6 +437,7 @@ message ChunkInfo { repeated KeyValue metadata = 4; required ChecksumData checksumData =5; optional bytes stripeChecksum = 6; + optional uint64 chunkOffsetInBlock = 7; } message ChunkInfoList { From 8ba23fc6822eeda2b99ad4de41b056c15f3c0b2f Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Wed, 24 Sep 2025 18:19:03 +0100 Subject: [PATCH 085/114] Remove open telemetery reference --- .../hadoop/ozone/container/common/impl/HddsDispatcher.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 e555460389de..e6be4a490b43 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 @@ -24,7 +24,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.protobuf.ServiceException; -import io.opentracing.Span; +import io.opentelemetry.api.trace.Span; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -886,7 +886,7 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, s, ioe, ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED); streamObserver.onNext(ContainerUtils.logAndReturnError(LOG, sce, msg)); } finally { - span.finish(); + span.end(); } } From 7a3cd39e702df1002f7862b4b2c2db826737028b Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Wed, 24 Sep 2025 18:24:34 +0100 Subject: [PATCH 086/114] Fix pmd error --- .../hadoop/hdds/scm/storage/TestStreamBlockInputStream.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index f8f1174704fe..4df97806e787 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -57,7 +57,6 @@ public class TestStreamBlockInputStream { private List chunks; private Map chunkDataMap; private Checksum checksum; - private Function refreshFunction; private BlockID blockID; private static final String CHUNK_NAME = "chunk-"; private OzoneConfiguration conf = new OzoneConfiguration(); @@ -66,7 +65,7 @@ public class TestStreamBlockInputStream { public void setup() throws Exception { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setStreamReadBlock(true); - refreshFunction = mock(Function.class); + Function refreshFunction = mock(Function.class); blockID = new BlockID(new ContainerBlockID(1, 1)); checksum = new Checksum(ChecksumType.CRC32, BYTES_PER_CHECKSUM); createChunkList(5); From 4c91e42b5baf263feceff7963246d7d7b771d3ac Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Mon, 13 Oct 2025 17:40:02 +0100 Subject: [PATCH 087/114] Refactor to stream entire block to client in a single call. Commented on many tests to allow it to compile --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 101 +-- .../scm/storage/StreamBlockInputStream.java | 634 +++++------------- .../storage/DummyStreamBlockInputStream.java | 2 - .../storage/TestStreamBlockInputStream.java | 2 +- .../hadoop/hdds/scm/XceiverClientSpi.java | 6 + .../ContainerCommandResponseBuilders.java | 20 +- .../scm/storage/ContainerProtocolCalls.java | 40 +- .../container/keyvalue/KeyValueHandler.java | 109 ++- .../keyvalue/TestKeyValueHandler.java | 4 +- .../main/proto/DatanodeClientProtocol.proto | 6 +- .../rpc/read/TestStreamBlockInputStream.java | 120 ++-- 11 files changed, 329 insertions(+), 715 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 09e01593feb1..4431799185c1 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -61,10 +61,12 @@ import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.util.Time; +import org.apache.http.concurrent.Cancellable; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; +import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.slf4j.Logger; @@ -453,11 +455,7 @@ private XceiverClientReply sendCommandWithRetry( // sendCommandAsyncCall will create a new channel and async stub // in case these don't exist for the specific datanode. reply.addDatanode(dn); - if (request.getCmdType() == ContainerProtos.Type.ReadBlock) { - responseProto = sendCommandReadBlock(request, dn).getResponse().get(); - } else { - responseProto = sendCommandAsync(request, dn).getResponse().get(); - } + responseProto = sendCommandAsync(request, dn).getResponse().get(); if (validators != null && !validators.isEmpty()) { for (Validator validator : validators) { validator.accept(request, responseProto); @@ -510,6 +508,36 @@ private XceiverClientReply sendCommandWithRetry( } } + // TODO This is currently not doing retries or timeouts + @Override + public void streamRead(ContainerCommandRequestProto request, + StreamObserver streamObserver) throws IOException { + + DatanodeDetails dn = topologyAwareRead ? + pipeline.getClosestNode() : pipeline.getFirstNode(); + + XceiverClientProtocolServiceStub stub = asyncStubs.get(dn.getID()); + if (stub == null) { + connectToDatanode(dn); + stub = asyncStubs.get(dn.getID()); + } + + if (stub == null) { + throw new IOException("Failed to get gRPC stub for DataNode: " + dn); + } + + LOG.debug("Starting streaming read to DataNode {}", dn); + + try { + StreamObserver requestObserver = stub.send(streamObserver); + requestObserver.onNext(request); + requestObserver.onCompleted(); + } catch (Exception e) { + LOG.error("Failed to start streaming read to DataNode {}", dn, e); + throw new IOException("Streaming read failed", e); + } + } + private static List sortDatanodeByOperationalState( List datanodeList) { List sortedDatanodeList = new ArrayList<>(datanodeList); @@ -629,69 +657,6 @@ private void decreasePendingMetricsAndReleaseSemaphore() { return new XceiverClientReply(replyFuture); } - public XceiverClientReply sendCommandReadBlock( - ContainerCommandRequestProto request, DatanodeDetails dn) - throws IOException, InterruptedException { - - CompletableFuture future = - new CompletableFuture<>(); - ContainerCommandResponseProto.Builder response = - ContainerCommandResponseProto.newBuilder(); - ContainerProtos.ReadBlockResponseProto.Builder readBlock = - ContainerProtos.ReadBlockResponseProto.newBuilder(); - checkOpen(dn); - DatanodeID dnId = dn.getID(); - Type cmdType = request.getCmdType(); - semaphore.acquire(); - long requestTime = System.currentTimeMillis(); - metrics.incrPendingContainerOpsMetrics(cmdType); - - final StreamObserver requestObserver = - asyncStubs.get(dnId).withDeadlineAfter(timeout, TimeUnit.SECONDS) - .send(new StreamObserver() { - @Override - public void onNext( - ContainerCommandResponseProto responseProto) { - if (responseProto.getResult() == Result.SUCCESS) { - readBlock.addReadChunk(responseProto.getReadChunk()); - } else { - future.complete( - ContainerCommandResponseProto.newBuilder(responseProto) - .setCmdType(Type.ReadBlock).build()); - } - } - - @Override - public void onError(Throwable t) { - future.completeExceptionally(t); - metrics.decrPendingContainerOpsMetrics(cmdType); - metrics.addContainerOpsLatency( - cmdType, Time.monotonicNow() - requestTime); - semaphore.release(); - } - - @Override - public void onCompleted() { - if (readBlock.getReadChunkCount() > 0) { - future.complete(response.setReadBlock(readBlock) - .setCmdType(Type.ReadBlock).setResult(Result.SUCCESS).build()); - } - if (!future.isDone()) { - future.completeExceptionally(new IOException( - "Stream completed but no reply for request " + - processForDebug(request))); - } - metrics.decrPendingContainerOpsMetrics(cmdType); - metrics.addContainerOpsLatency( - cmdType, System.currentTimeMillis() - requestTime); - semaphore.release(); - } - }); - requestObserver.onNext(request); - requestObserver.onCompleted(); - return new XceiverClientReply(future); - } - private synchronized void checkOpen(DatanodeDetails dn) throws IOException { if (closed) { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index ab4a56ea6219..621bebd84f57 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -20,15 +20,14 @@ import static org.apache.hadoop.hdds.client.ReplicationConfig.getLegacyFactor; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.time.Instant; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; +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.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import org.apache.commons.lang3.NotImplementedException; @@ -39,7 +38,6 @@ import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; @@ -47,16 +45,14 @@ import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; -import org.apache.hadoop.hdds.security.exception.SCMSecurityException; 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.ozone.common.utils.BufferUtils; import org.apache.hadoop.security.token.Token; -import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.io.grpc.Status; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,27 +62,21 @@ */ public class StreamBlockInputStream extends BlockExtendedInputStream implements Seekable, CanUnbuffer, ByteBufferReadable { - private static final Logger LOG = - LoggerFactory.getLogger(StreamBlockInputStream.class); + private static final Logger LOG = LoggerFactory.getLogger(StreamBlockInputStream.class); + private static final int EOF = -1; + private final BlockID blockID; private final long blockLength; - private final AtomicReference pipelineRef = - new AtomicReference<>(); - private final AtomicReference> tokenRef = - new AtomicReference<>(); + private final AtomicReference pipelineRef = new AtomicReference<>(); + private final AtomicReference> tokenRef = new AtomicReference<>(); private XceiverClientFactory xceiverClientFactory; private XceiverClientSpi xceiverClient; - private List bufferOffsets; - private int bufferIndex; - private long blockPosition = -1; - private List buffers; - // Checks if the StreamBlockInputStream has already read data from the container. - private boolean allocated = false; - private long bufferOffsetWrtBlockData; - private long buffersSize; - private static final int EOF = -1; - private final List validators; + private ByteBuffer buffer; + private long position = 0; + private boolean initialized = false; + private StreamingReader streamingReader; + private final boolean verifyChecksum; private final Function refreshFunction; private final RetryPolicy retryPolicy; @@ -103,13 +93,10 @@ public StreamBlockInputStream( setPipeline(pipeline); tokenRef.set(token); this.xceiverClientFactory = xceiverClientFactory; - this.validators = ContainerProtocolCalls.toValidatorList(this::validateBlock); this.verifyChecksum = config.isChecksumVerify(); this.refreshFunction = refreshFunction; - this.retryPolicy = - HddsClientUtils.createRetryPolicy(config.getMaxReadRetryCount(), - TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); - + this.retryPolicy = HddsClientUtils.createRetryPolicy(config.getMaxReadRetryCount(), + TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); } @Override @@ -124,170 +111,52 @@ public long getLength() { @Override public synchronized long getPos() { - if (blockLength == 0) { - return 0; - } - if (blockPosition >= 0) { - return blockPosition; - } - - if (buffersHaveData()) { - // BufferOffset w.r.t to BlockData + BufferOffset w.r.t buffers + - // Position of current Buffer - return bufferOffsetWrtBlockData + bufferOffsets.get(bufferIndex) + - buffers.get(bufferIndex).position(); - } - if (allocated && !dataRemainingInBlock()) { - Preconditions.checkState( - bufferOffsetWrtBlockData + buffersSize == blockLength, - "EOF detected but not at the last byte of the chunk"); - return blockLength; - } - if (buffersAllocated()) { - return bufferOffsetWrtBlockData + buffersSize; - } - return 0; + return position; } @Override public synchronized int read() throws IOException { - int dataout = EOF; - int len = 1; - int available; - while (len > 0) { - try { - acquireClient(); - available = prepareRead(1); - retries = 0; - } catch (SCMSecurityException ex) { - throw ex; - } catch (StorageContainerException e) { - handleStorageContainerException(e); - continue; - } catch (IOException ioe) { - handleIOException(ioe); - continue; - } - if (available == EOF) { - // There is no more data in the chunk stream. The buffers should have - // been released by now - Preconditions.checkState(buffers == null); - } else { - dataout = Byte.toUnsignedInt(buffers.get(bufferIndex).get()); - } - - len -= available; - if (bufferEOF()) { - releaseBuffers(bufferIndex); - } + if (!dataAvailableToRead()) { + return EOF; } - - - return dataout; - - + return buffer.get(); } @Override public synchronized int read(byte[] b, int off, int len) throws IOException { - // According to the JavaDocs for InputStream, it is recommended that - // subclasses provide an override of bulk read if possible for performance - // reasons. In addition to performance, we need to do it for correctness - // reasons. The Ozone REST service uses PipedInputStream and - // PipedOutputStream to relay HTTP response data between a Jersey thread and - // a Netty thread. It turns out that PipedInputStream/PipedOutputStream - // have a subtle dependency (bug?) on the wrapped stream providing separate - // implementations of single-byte read and bulk read. Without this, get key - // responses might close the connection before writing all of the bytes - // advertised in the Content-Length. - if (b == null) { - throw new NullPointerException(); - } - if (off < 0 || len < 0 || len > b.length - off) { - throw new IndexOutOfBoundsException(); - } - if (len == 0) { - return 0; - } - int total = 0; - int available; - while (len > 0) { - try { - acquireClient(); - available = prepareRead(len); - retries = 0; - } catch (SCMSecurityException ex) { - throw ex; - } catch (StorageContainerException e) { - handleStorageContainerException(e); - continue; - } catch (IOException ioe) { - handleIOException(ioe); - continue; - } - if (available == EOF) { - // There is no more data in the block stream. The buffers should have - // been released by now - Preconditions.checkState(buffers == null); - return total != 0 ? total : EOF; - } - buffers.get(bufferIndex).get(b, off + total, available); - len -= available; - total += available; + ByteBuffer tmpBuffer = ByteBuffer.wrap(b, off, len); + return read(tmpBuffer); + } - if (bufferEOF()) { - releaseBuffers(bufferIndex); + @Override + public synchronized int read(ByteBuffer targetBuf) throws IOException { + 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 total; - + return read > 0 ? read : EOF; } - @Override - public synchronized int read(ByteBuffer byteBuffer) throws IOException { - if (byteBuffer == null) { - throw new NullPointerException(); + private boolean dataAvailableToRead() throws IOException { + // TODO - closed stream here? The stream should be closed automatically when the last chunk is read. + if (position >= blockLength) { + return false; } - int len = byteBuffer.remaining(); - if (len == 0) { - return 0; - } - int total = 0; - int available; - while (len > 0) { - try { - acquireClient(); - available = prepareRead(len); - retries = 0; - } catch (SCMSecurityException ex) { - throw ex; - } catch (StorageContainerException e) { - handleStorageContainerException(e); - continue; - } catch (IOException ioe) { - handleIOException(ioe); - continue; - } - if (available == EOF) { - // There is no more data in the block stream. The buffers should have - // been released by now - Preconditions.checkState(buffers == null); - return total != 0 ? total : EOF; - } - ByteBuffer readBuf = buffers.get(bufferIndex); - ByteBuffer tmpBuf = readBuf.duplicate(); - tmpBuf.limit(tmpBuf.position() + available); - byteBuffer.put(tmpBuf); - readBuf.position(tmpBuf.position()); - - len -= available; - total += available; - - if (bufferEOF()) { - releaseBuffers(bufferIndex); - } + initialize(); + if (buffer == null || buffer.remaining() == 0) { + int loaded = fillBuffer(); + return loaded != EOF; } - return total; + return true; } @Override @@ -297,22 +166,7 @@ protected int readWithStrategy(ByteReaderStrategy strategy) throws IOException { @Override public synchronized void seek(long pos) throws IOException { - if (pos == 0 && blockLength == 0) { - // It is possible for length and pos to be zero in which case - // seek should return instead of throwing exception - return; - } - if (pos < 0 || pos > blockLength) { - throw new EOFException("EOF encountered at pos: " + pos + " for block: " + blockID); - } - - if (buffersHavePosition(pos)) { - // The bufferPosition is w.r.t the current block. - // Adjust the bufferIndex and position to the seeked position. - adjustBufferPosition(pos - bufferOffsetWrtBlockData); - } else { - blockPosition = pos; - } + throw new NotImplementedException("seek is not implemented."); } @Override @@ -322,9 +176,8 @@ public synchronized boolean seekToNewSource(long l) throws IOException { @Override public synchronized void unbuffer() { - blockPosition = getPos(); + // TODO releaseClient(); - releaseBuffers(); } private void setPipeline(Pipeline pipeline) throws IOException { @@ -370,243 +223,31 @@ protected synchronized void acquireClient() throws IOException { } } - private synchronized int prepareRead(int len) throws IOException { - for (;;) { - if (blockPosition >= 0) { - if (buffersHavePosition(blockPosition)) { - // The current buffers have the seeked position. Adjust the buffer - // index and position to point to the buffer position. - adjustBufferPosition(blockPosition - bufferOffsetWrtBlockData); - } else { - // Read a required block data to fill the buffers with seeked - // position data - readDataFromContainer(len); - } - } - if (buffersHaveData()) { - // Data is available from buffers - ByteBuffer bb = buffers.get(bufferIndex); - return Math.min(len, bb.remaining()); - } else if (dataRemainingInBlock()) { - // There is more data in the block stream which has not - // been read into the buffers yet. - readDataFromContainer(len); - } else { - // All available input from this block stream has been consumed. - return EOF; - } - } - - - } - - private boolean buffersHavePosition(long pos) { - // Check if buffers have been allocated - if (buffersAllocated()) { - // Check if the current buffers cover the input position - // Released buffers should not be considered when checking if position - // is available - return pos >= bufferOffsetWrtBlockData + - bufferOffsets.get(0) && - pos < bufferOffsetWrtBlockData + buffersSize; - } - return false; + private void reinitialize() throws IOException { + // TODO: close streaming reader + // set initialized false + // call initialize() } - /** - * Check if the buffers have been allocated data and false otherwise. - */ - @VisibleForTesting - protected boolean buffersAllocated() { - return buffers != null && !buffers.isEmpty(); - } - - /** - * Adjust the buffers position to account for seeked position and/ or checksum - * boundary reads. - * @param bufferPosition the position to which the buffers must be advanced - */ - private void adjustBufferPosition(long bufferPosition) { - // The bufferPosition is w.r.t the current buffers. - // Adjust the bufferIndex and position to the seeked bufferPosition. - bufferIndex = Collections.binarySearch(bufferOffsets, bufferPosition); - // bufferIndex is negative if bufferPosition isn't found in bufferOffsets - // count (bufferIndex = -bufferIndex - 2) to get bufferPosition is between which offsets. - if (bufferIndex < 0) { - bufferIndex = -bufferIndex - 2; - } - - buffers.get(bufferIndex).position( - (int) (bufferPosition - bufferOffsets.get(bufferIndex))); - - // Reset buffers > bufferIndex to position 0. We do this to reset any - // previous reads/ seeks which might have updated any buffer position. - // For buffers < bufferIndex, we do not need to reset the position as it - // not required for this read. If a seek was done to a position in the - // previous indices, the buffer position reset would be performed in the - // seek call. - for (int i = bufferIndex + 1; i < buffers.size(); i++) { - buffers.get(i).position(0); - } - - // Reset the blockPosition as chunk stream has been initialized i.e. the - // buffers have been allocated. - blockPosition = -1; - } - - /** - * Reads full or partial Chunk from DN Container based on the current - * position of the ChunkInputStream, the number of bytes of data to read - * and the checksum boundaries. - * If successful, then the read data in saved in the buffers so that - * subsequent read calls can utilize it. - * @param len number of bytes of data to be read - * @throws IOException if there is an I/O error while performing the call - * to Datanode - */ - private synchronized void readDataFromContainer(int len) throws IOException { - // index of first byte to be read from the block - long startByteIndex; - if (blockPosition >= 0) { - // If seek operation was called to advance the buffer position, the - // chunk should be read from that position onwards. - startByteIndex = blockPosition; - } else { - // Start reading the block from the last blockPosition onwards. - startByteIndex = bufferOffsetWrtBlockData + buffersSize; - } - - // bufferOffsetWrtChunkData and buffersSize are updated after the data - // is read from Container and put into the buffers, but if read fails - // and is retried, we need the previous position. Position is reset after - // successful read in adjustBufferPosition() - blockPosition = getPos(); - bufferOffsetWrtBlockData = readData(startByteIndex, len); - long tempOffset = 0L; - buffersSize = 0L; - bufferOffsets = new ArrayList<>(buffers.size()); - for (ByteBuffer buffer : buffers) { - bufferOffsets.add(tempOffset); - tempOffset += buffer.limit(); - buffersSize += buffer.limit(); - - } - bufferIndex = 0; - allocated = true; - adjustBufferPosition(startByteIndex - bufferOffsetWrtBlockData); - - } - - @VisibleForTesting - protected long readData(long startByteIndex, long len) - throws IOException { - Pipeline pipeline = pipelineRef.get(); - buffers = new ArrayList<>(); - ReadBlockResponseProto response = - ContainerProtocolCalls.readBlock(xceiverClient, startByteIndex, - len, blockID, validators, tokenRef.get(), pipeline.getReplicaIndexes(), verifyChecksum); - List readBlocks = response.getReadChunkList(); - - for (ReadChunkResponseProto readBlock : readBlocks) { - if (readBlock.hasDataBuffers()) { - buffers.addAll(BufferUtils.getReadOnlyByteBuffers( - readBlock.getDataBuffers().getBuffersList())); - } else { - throw new IOException("Unexpected error while reading chunk data " + - "from container. No data returned."); - } - } - return response.getReadChunk(0) - .getChunkData().getOffset(); - } - - /** - * Check if the buffers have any data remaining between the current - * position and the limit. - */ - private boolean buffersHaveData() { - boolean hasData = false; - if (buffersAllocated()) { - int buffersLen = buffers.size(); - while (bufferIndex < buffersLen) { - ByteBuffer buffer = buffers.get(bufferIndex); - if (buffer != null && buffer.hasRemaining()) { - // current buffer has data - hasData = true; - break; - } else { - if (bufferIndex < buffersLen - 1) { - // move to next available buffer - ++bufferIndex; - Preconditions.checkState(bufferIndex < buffers.size()); - } else { - // no more buffers remaining - break; - } - } - } - } - - return hasData; - } - - /** - * Check if there is more data in the chunk which has not yet been read - * into the buffers. - */ - private boolean dataRemainingInBlock() { - long bufferPos; - if (blockPosition >= 0) { - bufferPos = blockPosition; - } else { - bufferPos = bufferOffsetWrtBlockData + buffersSize; + private void initialize() throws IOException { + if (initialized) { + return; } - - return bufferPos < blockLength; - } - - /** - * Check if current buffer had been read till the end. - */ - private boolean bufferEOF() { - return allocated && buffersAllocated() && !buffers.get(bufferIndex).hasRemaining(); + acquireClient(); + streamingReader = new StreamingReader(); + ContainerProtocolCalls.readBlock( + xceiverClient, position, blockID, tokenRef.get(), pipelineRef.get().getReplicaIndexes(), streamingReader); + initialized = true; } - /** - * Release the buffers upto the given index. - * @param releaseUptoBufferIndex bufferIndex (inclusive) upto which the - * buffers must be released - */ - private void releaseBuffers(int releaseUptoBufferIndex) { - int buffersLen = buffers.size(); - if (releaseUptoBufferIndex == buffersLen - 1) { - // Before releasing all the buffers, if block EOF is not reached, then - // blockPosition should be set to point to the last position of the - // buffers. This should be done so that getPos() can return the current - // block position - blockPosition = bufferOffsetWrtBlockData + - bufferOffsets.get(releaseUptoBufferIndex) + - buffers.get(releaseUptoBufferIndex).capacity(); - // Release all the buffers - releaseBuffers(); - } else { - buffers = buffers.subList(releaseUptoBufferIndex + 1, buffersLen); - bufferOffsets = bufferOffsets.subList( - releaseUptoBufferIndex + 1, buffersLen); - bufferIndex = 0; + private int fillBuffer() throws IOException { + if (!streamingReader.hasNext()) { + return EOF; } + buffer = streamingReader.readNext(); + return buffer == null ? EOF : buffer.limit(); } - /** - * If EOF is reached, release the buffers. - */ - private void releaseBuffers() { - buffers = null; - bufferIndex = 0; - // We should not reset bufferOffsetWrtBlockData and buffersSize here - // because when getPos() is called we use these - // values and determine whether chunk is read completely or not. - } protected synchronized void releaseClient() { if (xceiverClientFactory != null && xceiverClient != null) { @@ -615,49 +256,6 @@ protected synchronized void releaseClient() { } } - private void validateBlock( - ContainerProtos.ContainerCommandRequestProto request, - ContainerProtos.ContainerCommandResponseProto response - ) throws IOException { - - ReadBlockResponseProto readBlock = response.getReadBlock(); - for (ReadChunkResponseProto readChunk : readBlock.getReadChunkList()) { - List byteStrings; - - ContainerProtos.ChunkInfo chunkInfo = - readChunk.getChunkData(); - if (chunkInfo.getLen() <= 0) { - throw new IOException("Failed to get chunk: chunkName == " - + chunkInfo.getChunkName() + "len == " + chunkInfo.getLen()); - } - byteStrings = readChunk.getDataBuffers().getBuffersList(); - long buffersLen = BufferUtils.getBuffersLen(byteStrings); - if (buffersLen != chunkInfo.getLen()) { - // Bytes read from chunk should be equal to chunk size. - throw new OzoneChecksumException(String.format( - "Inconsistent read for chunk=%s len=%d bytesRead=%d", - chunkInfo.getChunkName(), chunkInfo.getLen(), - buffersLen)); - } - - if (verifyChecksum) { - ChecksumData checksumData = ChecksumData.getFromProtoBuf( - chunkInfo.getChecksumData()); - int startIndex = (int) (readChunk.getChunkData().getOffset() - readChunk.getChunkData().getChunkOffsetInBlock()) - / checksumData.getBytesPerChecksum(); - // ChecksumData stores checksum for each 'numBytesPerChecksum' - // number of bytes in a list. Compute the index of the first - // checksum to match with the read data - Checksum.verifyChecksum(byteStrings, checksumData, startIndex); - } - } - } - - @VisibleForTesting - protected synchronized void setBuffers(List buffers) { - this.buffers = buffers; - } - private boolean shouldRetryRead(IOException cause) throws IOException { RetryPolicy.RetryAction retryAction; try { @@ -701,12 +299,6 @@ private void refreshBlockInfo(IOException cause) throws IOException { } } - @VisibleForTesting - public synchronized ByteBuffer[] getCachedBuffers() { - return buffers == null ? null : - BufferUtils.getReadOnlyByteBuffers(buffers.toArray(new ByteBuffer[0])); - } - /** * Check if this exception is because datanodes are not reachable. */ @@ -716,8 +308,8 @@ private boolean isConnectivityIssue(IOException ex) { @Override public synchronized void close() throws IOException { + LOG.info("+++ Closing StreamBlockInputStream for block {}", blockID); releaseClient(); - releaseBuffers(); xceiverClientFactory = null; } @@ -742,4 +334,96 @@ private void handleIOException(IOException ioe) throws IOException { throw ioe; } } + + public class StreamingReader implements StreamObserver { + + private final BlockingQueue responseQueue = new LinkedBlockingQueue<>(1); + private final AtomicBoolean completed = new AtomicBoolean(false); + private final AtomicBoolean failed = new AtomicBoolean(false); + private final AtomicReference error = new AtomicReference<>(); + + // TODO: Semaphore in XceiverClient which count open stream? + public boolean hasNext() { + return !responseQueue.isEmpty() || (!completed.get() && !failed.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); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while waiting for response", e); + } + if (readBlock == null) { + if (failed.get()) { + Throwable cause = error.get(); + throw new IOException("Streaming read failed", cause); + } else if (completed.get()) { + return null; // Stream ended + } else { + throw new IOException("Timed out waiting for response"); + } + } + // The server always returns data starting from the last checksum boundary. Therefore if the reader position is + // ahead of the position we received from the server, we need to adjust the buffer position accordingly. + // If the reader position is behind + ByteBuffer buf = readBlock.getData().asReadOnlyByteBuffer(); + long blockOffset = readBlock.getOffset(); + if (position < blockOffset) { + // This should not happen, and if it does, we have a bug. + throw new IOException("Received data out of order. Position is " + position + " but received data at " + + blockOffset); + } + if (position > readBlock.getOffset()) { + int offset = (int)(position - readBlock.getOffset()); + buffer.position(offset); + } + return buf; + } + + @Override + public synchronized void onNext(ContainerProtos.ContainerCommandResponseProto containerCommandResponseProto) { + try { + LOG.info("+++ Called onNext"); + ReadBlockResponseProto readBlock = containerCommandResponseProto.getReadBlock(); + ByteBuffer data = readBlock.getData().asReadOnlyByteBuffer(); + if (verifyChecksum) { + ChecksumData checksumData = ChecksumData.getFromProtoBuf(readBlock.getChecksumData()); + Checksum.verifyChecksum(data, checksumData, 0); + } + responseQueue.put(readBlock); + LOG.info("+++ Processed {} read responses for block {}", processed, blockID); + } catch (OzoneChecksumException e) { + // Calling onError will cancel the stream on the server side and also set the failure state. + onError(e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + onError(e); + } + } + + @Override + public synchronized void onError(Throwable throwable) { + LOG.info("+++ Called onError"); + failed.set(true); + error.set(throwable); + } + + @Override + public synchronized void onCompleted() { + LOG.info("+++ Called onCompleted"); + completed.set(true); + } + } + } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInputStream.java index e141845954d3..b26dd672c339 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInputStream.java @@ -82,7 +82,6 @@ protected void releaseClient() { // no-op } - @Override protected long readData(long offset, long len) { int chunkIndex = Arrays.binarySearch(chunkOffsets, offset); if (chunkIndex < 0) { @@ -133,7 +132,6 @@ protected long readData(long offset, long len) { chunkOffset = 0; chunkIndex++; } - setBuffers(BufferUtils.getReadOnlyByteBuffers(readByteBuffers)); return bufferOffsetWrtBlockDataData; } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 4df97806e787..fd4f53bd10ab 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -253,7 +253,7 @@ public void testUnbuffered() throws Exception { blockStream.unbuffer(); - assertFalse(blockStream.buffersAllocated()); + // assertFalse(blockStream.buffersAllocated()); // Next read should start from the position of the last read + 1 i.e. 20 byte[] b2 = new byte[20]; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java index 35e65271bb1c..9e0ae77bce8b 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.function.CheckedBiConsumer; /** @@ -144,6 +145,11 @@ public ContainerCommandResponseProto sendCommand( } } + public void streamRead(ContainerCommandRequestProto request, + StreamObserver streamObserver) throws IOException { + throw new UnsupportedOperationException("Stream read is not supported"); + } + public static IOException getIOExceptionForSendCommand( ContainerCommandRequestProto request, Exception e) { return new IOException("Failed to execute command " diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index ededa8d070b6..3fc97e97ffaa 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkResponseProto; +import org.apache.hadoop.ozone.common.ChecksumData; import org.apache.hadoop.ozone.common.ChunkBufferToByteString; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations; @@ -335,19 +336,16 @@ public static ContainerCommandResponseProto getReadChunkResponse( } public static ContainerCommandResponseProto getReadBlockResponse( - ContainerCommandRequestProto request, DatanodeBlockID blockID, - ChunkInfo chunkInfo, ChunkBufferToByteString data, - Function byteBufferToByteString) { + ContainerCommandRequestProto request, ChecksumData checksumData, ByteBuffer data, long offset) { + + ContainerProtos.ReadBlockResponseProto response = ContainerProtos.ReadBlockResponseProto.newBuilder() + .setChecksumData(checksumData.getProtoBufMessage()) + .setData(ByteString.copyFrom(data)) + .setOffset(offset) + .build(); - ReadChunkResponseProto.Builder response; - response = ReadChunkResponseProto.newBuilder() - .setChunkData(chunkInfo) - .setDataBuffers(DataBuffers.newBuilder() - .addAllBuffers(data.toByteStringList(byteBufferToByteString)) - .build()) - .setBlockID(blockID); return getSuccessResponseBuilder(request) - .setReadChunk(response) + .setReadBlock(response) .build(); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 1117d29c1aa3..6e22a7807c90 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -77,6 +77,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.function.CheckedFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -911,23 +912,18 @@ public static List toValidatorList(Validator validator) { * * @param xceiverClient client to perform call * @param offset offset where block starts - * @param len length of data to read * @param blockID ID of the block - * @param validators functions to validate the response * @param token a token for this block (may be null) - * @return container protocol read chunk response * @throws IOException if there is an I/O error while performing the call */ @SuppressWarnings("checkstyle:ParameterNumber") - public static ContainerProtos.ReadBlockResponseProto readBlock( - XceiverClientSpi xceiverClient, long offset, long len, BlockID blockID, - List validators, Token token, - Map replicaIndexes, boolean verifyChecksum) throws IOException { + public static void readBlock( + XceiverClientSpi xceiverClient, long offset, BlockID blockID, Token token, + Map replicaIndexes, StreamObserver streamObserver) + throws IOException { final ReadBlockRequestProto.Builder readBlockRequest = ReadBlockRequestProto.newBuilder() - .setOffset(offset) - .setVerifyChecksum(verifyChecksum) - .setLen(len); + .setOffset(offset); final ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto.newBuilder().setCmdType(Type.ReadBlock) .setContainerID(blockID.getContainerID()); @@ -935,18 +931,18 @@ public static ContainerProtos.ReadBlockResponseProto readBlock( builder.setEncodedToken(token.encodeToUrlString()); } - return tryEachDatanode(xceiverClient.getPipeline(), - d -> readBlock(xceiverClient, - validators, blockID, builder, readBlockRequest, d, replicaIndexes), - d -> toErrorMessage(blockID, d)); + readBlock(xceiverClient, blockID, builder, readBlockRequest, xceiverClient.getPipeline().getFirstNode(), + replicaIndexes, streamObserver); + // tryEachDatanode(xceiverClient.getPipeline(), + // d -> readBlock(xceiverClient, + // validators, blockID, builder, readBlockRequest, d, replicaIndexes, streamObserver), + // d -> toErrorMessage(blockID, d)); } - private static ReadBlockResponseProto readBlock(XceiverClientSpi xceiverClient, - List validators, BlockID blockID, - ContainerCommandRequestProto.Builder builder, - ReadBlockRequestProto.Builder readBlockBuilder, - DatanodeDetails datanode, - Map replicaIndexes) throws IOException { + private static void readBlock(XceiverClientSpi xceiverClient, BlockID blockID, + ContainerCommandRequestProto.Builder builder, ReadBlockRequestProto.Builder readBlockBuilder, + DatanodeDetails datanode, Map replicaIndexes, + StreamObserver streamObserver) throws IOException { final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); int replicaIndex = replicaIndexes.getOrDefault(datanode, 0); if (replicaIndex > 0) { @@ -956,8 +952,6 @@ private static ReadBlockResponseProto readBlock(XceiverClientSpi xceiverClient, final ContainerCommandRequestProto request = builder .setDatanodeUuid(datanode.getUuidString()) .setReadBlock(readBlockBuilder).build(); - ContainerCommandResponseProto response = - xceiverClient.sendCommand(request, validators); - return response.getReadBlock(); + xceiverClient.streamRead(request, streamObserver); } } 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 1b65db6bfe82..6e1170b79ab9 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 @@ -57,26 +57,33 @@ import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion; import static org.apache.hadoop.ozone.OzoneConsts.INCREMENTAL_CHUNK_LIST; +import static org.apache.hadoop.ozone.OzoneConsts.LAYOUTVERSION; import static org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient.createSingleNodePipeline; import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.DEFAULT_LAYOUT; +import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_BLOCK; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.Striped; import java.io.File; +import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.FilenameFilter; 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.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.time.Clock; import java.time.Instant; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; @@ -124,6 +131,7 @@ import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl; import org.apache.hadoop.ozone.common.Checksum; +import org.apache.hadoop.ozone.common.ChecksumData; import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.common.ChunkBufferToByteString; import org.apache.hadoop.ozone.common.OzoneChecksumException; @@ -2037,6 +2045,12 @@ public ContainerCommandResponseProto readBlock( ContainerCommandRequestProto request, Container kvContainer, DispatcherContext dispatcherContext, StreamObserver 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()) { @@ -2054,71 +2068,44 @@ public ContainerCommandResponseProto readBlock( 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 chunkInfos = blockData.getChunks(); - long blockOffset = 0; - int chunkIndex = -1; - long chunkOffset = 0; - long offset = readBlock.getOffset(); - for (int i = 0; i < chunkInfos.size(); i++) { - final long chunkLen = chunkInfos.get(i).getLen(); - blockOffset += chunkLen; - if (blockOffset > offset) { - chunkIndex = i; - chunkOffset = offset - blockOffset + chunkLen; - break; + // 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. + int bytesPerChecksum = chunkInfos.get(0).getChecksumData().getBytesPerChecksum(); + ContainerProtos.ChecksumType checksumType = chunkInfos.get(0).getChecksumData().getType(); + // 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. + long adjustedOffset = readBlock.getOffset() - readBlock.getOffset() % bytesPerChecksum; + // 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. + + 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(); + int chunkIndex = (int) (adjustedOffset / bytesPerChunk); + int chunkOffset = (int) (adjustedOffset % bytesPerChunk); + int checksumIndex = chunkOffset / bytesPerChecksum; + ByteString checksum = blockData.getChunks().get(chunkIndex).getChecksumData().getChecksums(checksumIndex); + + ChecksumData checksumData = + new ChecksumData(checksumType, bytesPerChecksum, Collections.singletonList(checksum)); + streamObserver.onNext(getReadBlockResponse(request, checksumData, buffer, adjustedOffset)); + buffer.clear(); + + adjustedOffset += bytesPerChecksum; } } - Preconditions.checkState(chunkIndex >= 0); - - if (dispatcherContext == null) { - dispatcherContext = DispatcherContext.getHandleReadBlock(); - } - - ChunkBufferToByteString data; - - long len = readBlock.getLen(); - long adjustedChunkOffset, adjustedChunkLen; - // the first chunk length is the largest one. Either the block is less than the chunk size and there is only - // a single chunk, or the block is larger and the chunks are all fixed size (eg 4MB) until the final chunk. - long blockChunkSize = chunkInfos.get(0).getLen(); - do { - ContainerProtos.ChunkInfo chunk = chunkInfos.get(chunkIndex); - if (readBlock.getVerifyChecksum()) { - Pair adjustedOffsetAndLength = - computeChecksumBoundaries(chunk, chunkOffset, len); - adjustedChunkOffset = adjustedOffsetAndLength.getLeft(); - adjustedChunkLen = adjustedOffsetAndLength.getRight(); - adjustedChunkOffset += chunk.getOffset(); - } else { - adjustedChunkOffset = chunkOffset; - adjustedChunkLen = Math.min( - chunk.getLen() + chunk.getOffset() - chunkOffset, len); - } - - ContainerProtos.ChunkInfo chunkInfoProto = ContainerProtos.ChunkInfo.newBuilder(chunk) - .setOffset(adjustedChunkOffset) - .setLen(adjustedChunkLen) - .setChunkOffsetInBlock(blockChunkSize * chunkIndex).build(); - ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(chunkInfoProto); - - BlockUtils.verifyReplicaIdx(kvContainer, blockID); - BlockUtils.verifyBCSId(kvContainer, blockID); - data = getChunkManager().readChunk( - kvContainer, blockID, chunkInfo, dispatcherContext); - - Preconditions.checkNotNull(data, "Chunk data is null"); - streamObserver.onNext( - getReadBlockResponse(request, - blockData.getProtoBufMessage().getBlockID(), - chunkInfoProto, - data, byteBufferToByteString)); - len -= adjustedChunkLen + adjustedChunkOffset - chunkOffset; - chunkOffset = 0; - chunkIndex++; - } while (len > 0); - - metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen()); + // TODO metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen()); } catch (StorageContainerException ex) { responseProto = ContainerUtils.logAndReturnError(LOG, ex, request); } catch (IOException ioe) { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index b3524e7a73c4..db380ef38c4f 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -941,9 +941,7 @@ private static ContainerCommandRequestProto readBlockRequest( ContainerProtos.DatanodeBlockID.newBuilder() .setContainerID(DUMMY_CONTAINER_ID) .setLocalID(LOCAL_ID)) - .setOffset(offset) - .setLen(length) - .setVerifyChecksum(true)) + .setOffset(offset)) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(UUID.randomUUID().toString()) .build(); diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index acb802227645..fd225c9fbf67 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -401,12 +401,12 @@ message ListBlockResponseProto { message ReadBlockRequestProto { required DatanodeBlockID blockID = 1; required uint64 offset = 2; - required uint64 len = 3; - required bool verifyChecksum = 4; } message ReadBlockResponseProto { - repeated ReadChunkResponseProto readChunk = 1; + required ChecksumData checksumData = 1; + required uint64 offset = 2; + required bytes data = 3; } message EchoRequestProto { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 80ae5118467e..a81e4f403a7d 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -36,6 +36,7 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; import org.apache.hadoop.ozone.om.TestBucket; +import org.junit.jupiter.api.Test; /** * Tests {@link StreamBlockInputStream}. @@ -45,8 +46,9 @@ public class TestStreamBlockInputStream extends TestInputStreamBase { * Run the tests as a single test method to avoid needing a new mini-cluster * for each test. */ - @ContainerLayoutTestInfo.ContainerTest - void testAll(ContainerLayoutVersion layout) throws Exception { + // @ContainerLayoutTestInfo.ContainerTest + @Test + void testAll() throws Exception { try (MiniOzoneCluster cluster = newCluster()) { cluster.waitForClusterToBeReady(); @@ -56,13 +58,13 @@ void testAll(ContainerLayoutVersion layout) throws Exception { OzoneConfiguration copy = new OzoneConfiguration(conf); copy.setFromObject(clientConfig); try (OzoneClient client = OzoneClientFactory.getRpcClient(copy)) { - updateConfig(layout); + //updateConfig(layout); TestBucket bucket = TestBucket.newBuilder(client).build(); - testBlockReadBuffers(bucket); - testBufferRelease(bucket); - testCloseReleasesBuffers(bucket); - testReadEmptyBlock(bucket); + testReadKeyFully(bucket); + // testBufferRelease(bucket); + // testCloseReleasesBuffers(bucket); + // testReadEmptyBlock(bucket); } } } @@ -71,57 +73,39 @@ void testAll(ContainerLayoutVersion layout) throws Exception { * Test to verify that data read from blocks is stored in a list of buffers * with max capacity equal to the bytes per checksum. */ - private void testBlockReadBuffers(TestBucket bucket) throws Exception { + private void testReadKeyFully(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); + // Read the data fully into a large enough byte array try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - - StreamBlockInputStream block0Stream = - (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); - - - // To read 1 byte of chunk data, ChunkInputStream should get one full - // checksum boundary worth of data from Container and store it in buffers. - IOUtils.readFully(block0Stream, new byte[1]); - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - - // Read > checksum boundary of data from chunk0 - int readDataLen = BYTES_PER_CHECKSUM + (BYTES_PER_CHECKSUM / 2); - byte[] readData = readDataFromBlock(block0Stream, 0, readDataLen); - bucket.validateData(inputData, 0, readData); - - // The first checksum boundary size of data was already existing in the - // ChunkStream buffers. Once that data is read, the next checksum - // boundary size of data will be fetched again to read the remaining data. - // Hence, there should be 1 checksum boundary size of data stored in the - // ChunkStreams buffers at the end of the read. - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - - // Seek to a position in the third checksum boundary (so that current - // buffers do not have the seeked position) and read > BYTES_PER_CHECKSUM - // bytes of data. This should result in 2 * BYTES_PER_CHECKSUM amount of - // data being read into the buffers. There should be 2 buffers in the - // stream but the first buffer should be released after it is read - // and the second buffer should have BYTES_PER_CHECKSUM capacity. - int offset = 2 * BYTES_PER_CHECKSUM + 1; - readData = readDataFromBlock(block0Stream, offset, readDataLen); - bucket.validateData(inputData, offset, readData); - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - - - // Read the full chunk data -1 and verify that all chunk data is read into - // buffers. We read CHUNK_SIZE - 1 as otherwise all the buffers will be - // released once all chunk data is read. - readData = readDataFromBlock(block0Stream, 0, CHUNK_SIZE - 1); - bucket.validateData(inputData, 0, readData); - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - - // Read the last byte of chunk and verify that the buffers are released. - IOUtils.readFully(block0Stream, new byte[1]); - assertNull(block0Stream.getCachedBuffers(), - "ChunkInputStream did not release buffers after reaching EOF."); + byte[] readData = new byte[dataLength]; + int totalRead = keyInputStream.read(readData, 0, dataLength); + assertEquals(dataLength, totalRead); + for (int i = 0; i < dataLength; i++) { + assertEquals(inputData[i], readData[i], + "Read data is not same as written data at index " + i); + } + } + // Read the data 1 byte at a time + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + for (int i = 0; i < dataLength; i++) { + int b = keyInputStream.read(); + assertEquals(inputData[i], (byte) b, + "Read data is not same as written data at index " + i); + } + } + // Read the data into a large enough ByteBuffer + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + ByteBuffer readBuf = ByteBuffer.allocate(dataLength); + int totalRead = keyInputStream.read(readBuf); + assertEquals(dataLength, totalRead); + readBuf.flip(); + for (int i = 0; i < dataLength; i++) { + assertEquals(inputData[i], readBuf.get(), + "Read data is not same as written data at index " + i); + } } } @@ -134,11 +118,11 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); readDataFromBlock(block0Stream, 0, 1); - assertNotNull(block0Stream.getCachedBuffers()); + // assertNotNull(block0Stream.getCachedBuffers()); block0Stream.close(); - assertNull(block0Stream.getCachedBuffers()); + // assertNull(block0Stream.getCachedBuffers()); } } @@ -162,39 +146,39 @@ private void testBufferRelease(TestBucket bucket) throws Exception { // There should be 1 byte of data remaining in the buffer which is not // yet read. Hence, the buffer should not be released. - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - assertEquals(1, block0Stream.getCachedBuffers()[0].remaining()); + // checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); + // assertEquals(1, block0Stream.getCachedBuffers()[0].remaining()); // Reading the last byte in the buffer should result in all the buffers // being released. readData = readDataFromBlock(block0Stream, 1); bucket.validateData(inputData, readDataLen, readData); - assertNull(block0Stream.getCachedBuffers(), - "Chunk stream buffers not released after last byte is read"); - + // assertNull(block0Stream.getCachedBuffers(), + // "Chunk stream buffers not released after last byte is read"); +// // Read more data to get the data till the next checksum boundary. readDataLen = BYTES_PER_CHECKSUM / 2; readDataFromBlock(block0Stream, readDataLen); // There should be one buffer and the buffer should not be released as // there is data pending to be read from the buffer - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - ByteBuffer lastCachedBuffer = block0Stream.getCachedBuffers()[0]; - assertEquals(BYTES_PER_CHECKSUM - readDataLen, - lastCachedBuffer.remaining()); + // checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); + // ByteBuffer lastCachedBuffer = block0Stream.getCachedBuffers()[0]; + // assertEquals(BYTES_PER_CHECKSUM - readDataLen, + // lastCachedBuffer.remaining()); // Read more than the remaining data in buffer (but less than the next // checksum boundary). int position = (int) block0Stream.getPos(); - readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; + // readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; readData = readDataFromBlock(block0Stream, readDataLen); bucket.validateData(inputData, position, readData); // After reading the remaining data in the buffer, the buffer should be // released and next checksum size of data must be read into the buffers - checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); + // checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); // Verify that the previously cached buffer is released by comparing it // with the current cached buffer - assertNotEquals(lastCachedBuffer, - block0Stream.getCachedBuffers()[0]); + // assertNotEquals(lastCachedBuffer, + // block0Stream.getCachedBuffers()[0]); } } From e2cdb12b4b8a16c2df542328cbe7aaddee3e70a6 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Tue, 14 Oct 2025 14:02:17 +0100 Subject: [PATCH 088/114] Implement seek --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 3 +- .../scm/storage/StreamBlockInputStream.java | 91 ++++++++++++++----- .../hadoop/hdds/scm/XceiverClientSpi.java | 3 +- .../scm/storage/ContainerProtocolCalls.java | 11 ++- .../rpc/read/TestStreamBlockInputStream.java | 48 +++++++--- 5 files changed, 111 insertions(+), 45 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 4431799185c1..aa9c9a17843c 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -510,7 +510,7 @@ private XceiverClientReply sendCommandWithRetry( // TODO This is currently not doing retries or timeouts @Override - public void streamRead(ContainerCommandRequestProto request, + public ClientCallStreamObserver streamRead(ContainerCommandRequestProto request, StreamObserver streamObserver) throws IOException { DatanodeDetails dn = topologyAwareRead ? @@ -532,6 +532,7 @@ public void streamRead(ContainerCommandRequestProto request, StreamObserver requestObserver = stub.send(streamObserver); requestObserver.onNext(request); requestObserver.onCompleted(); + return (ClientCallStreamObserver) requestObserver; } catch (Exception e) { LOG.error("Failed to start streaming read to DataNode {}", dn, e); throw new IOException("Streaming read failed", e); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index 621bebd84f57..eafb7ba5a418 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -27,7 +27,6 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import org.apache.commons.lang3.NotImplementedException; @@ -52,6 +51,7 @@ import org.apache.hadoop.ozone.common.OzoneChecksumException; import org.apache.hadoop.security.token.Token; import org.apache.ratis.thirdparty.io.grpc.Status; +import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,6 +64,7 @@ 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; @@ -147,7 +148,6 @@ public synchronized int read(ByteBuffer targetBuf) throws IOException { } private boolean dataAvailableToRead() throws IOException { - // TODO - closed stream here? The stream should be closed automatically when the last chunk is read. if (position >= blockLength) { return false; } @@ -166,20 +166,40 @@ protected int readWithStrategy(ByteReaderStrategy strategy) throws IOException { @Override public synchronized void seek(long pos) throws IOException { - throw new NotImplementedException("seek is not implemented."); + 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() { - // TODO releaseClient(); } + private void closeStream() { + if (streamingReader != null) { + streamingReader.cancel(); + streamingReader = null; + } + initialized = false; + buffer = null; + } + private void setPipeline(Pipeline pipeline) throws IOException { if (pipeline == null) { return; @@ -223,20 +243,16 @@ protected synchronized void acquireClient() throws IOException { } } - private void reinitialize() throws IOException { - // TODO: close streaming reader - // set initialized false - // call initialize() - } - private void initialize() throws IOException { if (initialized) { return; } acquireClient(); streamingReader = new StreamingReader(); - ContainerProtocolCalls.readBlock( - xceiverClient, position, blockID, tokenRef.get(), pipelineRef.get().getReplicaIndexes(), streamingReader); + ClientCallStreamObserver requestObserver = + ContainerProtocolCalls.readBlock(xceiverClient, position, blockID, tokenRef.get(), + pipelineRef.get().getReplicaIndexes(), streamingReader); + streamingReader.setRequestObserver(requestObserver); initialized = true; } @@ -251,6 +267,7 @@ private int fillBuffer() throws IOException { protected synchronized void releaseClient() { if (xceiverClientFactory != null && xceiverClient != null) { + closeStream(); xceiverClientFactory.releaseClientForReadData(xceiverClient, false); xceiverClient = null; } @@ -308,7 +325,6 @@ private boolean isConnectivityIssue(IOException ex) { @Override public synchronized void close() throws IOException { - LOG.info("+++ Closing StreamBlockInputStream for block {}", blockID); releaseClient(); xceiverClientFactory = null; } @@ -341,6 +357,7 @@ public class StreamingReader implements StreamObserver error = new AtomicReference<>(); + private ClientCallStreamObserver requestObserver; // TODO: Semaphore in XceiverClient which count open stream? public boolean hasNext() { @@ -386,44 +403,70 @@ public ByteBuffer readNext() throws IOException { } if (position > readBlock.getOffset()) { int offset = (int)(position - readBlock.getOffset()); - buffer.position(offset); + buf.position(offset); } return buf; } + public void setRequestObserver( + ClientCallStreamObserver requestObserver) { + this.requestObserver = requestObserver; + } + + /** + * By calling cancel, the client will send a cancel signal to the server, which will stop sending more data and + * cause the onError() to be called in this observer with a CANCELLED exception. + */ + public void cancel() { + if (requestObserver != null) { + requestObserver.cancel("Cancelled by client", CANCELLED_EXCEPTION); + completed.set(true); + } + } + @Override public synchronized void onNext(ContainerProtos.ContainerCommandResponseProto containerCommandResponseProto) { try { - LOG.info("+++ Called onNext"); ReadBlockResponseProto readBlock = containerCommandResponseProto.getReadBlock(); ByteBuffer data = readBlock.getData().asReadOnlyByteBuffer(); if (verifyChecksum) { ChecksumData checksumData = ChecksumData.getFromProtoBuf(readBlock.getChecksumData()); Checksum.verifyChecksum(data, checksumData, 0); } - responseQueue.put(readBlock); - LOG.info("+++ Processed {} read responses for block {}", processed, blockID); + offerToQueue(readBlock); } catch (OzoneChecksumException e) { // Calling onError will cancel the stream on the server side and also set the failure state. onError(e); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - onError(e); } } @Override public synchronized void onError(Throwable throwable) { - LOG.info("+++ Called onError"); - failed.set(true); - error.set(throwable); + if (throwable == CANCELLED_EXCEPTION) { + completed.set(true); + } else { + failed.set(true); + error.set(throwable); + } } @Override public synchronized void onCompleted() { - LOG.info("+++ Called onCompleted"); completed.set(true); } + + private void offerToQueue(ReadBlockResponseProto item) { + while (!completed.get() && !failed.get()) { + try { + if (responseQueue.offer(item, 100, TimeUnit.MILLISECONDS)) { + return; + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return; + } + } + } } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java index 9e0ae77bce8b..f823946f7fbd 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.function.CheckedBiConsumer; @@ -145,7 +146,7 @@ public ContainerCommandResponseProto sendCommand( } } - public void streamRead(ContainerCommandRequestProto request, + public ClientCallStreamObserver streamRead(ContainerCommandRequestProto request, StreamObserver streamObserver) throws IOException { throw new UnsupportedOperationException("Stream read is not supported"); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 6e22a7807c90..a3df45b8c248 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -77,6 +77,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.function.CheckedFunction; import org.slf4j.Logger; @@ -917,7 +918,7 @@ public static List toValidatorList(Validator validator) { * @throws IOException if there is an I/O error while performing the call */ @SuppressWarnings("checkstyle:ParameterNumber") - public static void readBlock( + public static ClientCallStreamObserver readBlock( XceiverClientSpi xceiverClient, long offset, BlockID blockID, Token token, Map replicaIndexes, StreamObserver streamObserver) throws IOException { @@ -931,7 +932,7 @@ public static void readBlock( builder.setEncodedToken(token.encodeToUrlString()); } - readBlock(xceiverClient, blockID, builder, readBlockRequest, xceiverClient.getPipeline().getFirstNode(), + return readBlock(xceiverClient, blockID, builder, readBlockRequest, xceiverClient.getPipeline().getFirstNode(), replicaIndexes, streamObserver); // tryEachDatanode(xceiverClient.getPipeline(), // d -> readBlock(xceiverClient, @@ -939,8 +940,8 @@ public static void readBlock( // d -> toErrorMessage(blockID, d)); } - private static void readBlock(XceiverClientSpi xceiverClient, BlockID blockID, - ContainerCommandRequestProto.Builder builder, ReadBlockRequestProto.Builder readBlockBuilder, + private static ClientCallStreamObserver readBlock(XceiverClientSpi xceiverClient, + BlockID blockID, ContainerCommandRequestProto.Builder builder, ReadBlockRequestProto.Builder readBlockBuilder, DatanodeDetails datanode, Map replicaIndexes, StreamObserver streamObserver) throws IOException { final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); @@ -952,6 +953,6 @@ private static void readBlock(XceiverClientSpi xceiverClient, BlockID blockID, final ContainerCommandRequestProto request = builder .setDatanodeUuid(datanode.getUuidString()) .setReadBlock(readBlockBuilder).build(); - xceiverClient.streamRead(request, streamObserver); + return xceiverClient.streamRead(request, streamObserver); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index a81e4f403a7d..a3e983265dcb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -18,9 +18,7 @@ package org.apache.hadoop.ozone.client.rpc.read; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; @@ -33,8 +31,6 @@ import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.io.KeyInputStream; -import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; -import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; import org.apache.hadoop.ozone.om.TestBucket; import org.junit.jupiter.api.Test; @@ -46,6 +42,12 @@ public class TestStreamBlockInputStream extends TestInputStreamBase { * Run the tests as a single test method to avoid needing a new mini-cluster * for each test. */ + + private String keyName = getNewKeyName(); + private int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); + byte[] inputData; + private TestBucket bucket; + // @ContainerLayoutTestInfo.ContainerTest @Test void testAll() throws Exception { @@ -58,10 +60,10 @@ void testAll() throws Exception { OzoneConfiguration copy = new OzoneConfiguration(conf); copy.setFromObject(clientConfig); try (OzoneClient client = OzoneClientFactory.getRpcClient(copy)) { - //updateConfig(layout); - TestBucket bucket = TestBucket.newBuilder(client).build(); - - testReadKeyFully(bucket); + bucket = TestBucket.newBuilder(client).build(); + inputData = bucket.writeRandomBytes(keyName, dataLength); + testReadKeyFully(); + testSeek(); // testBufferRelease(bucket); // testCloseReleasesBuffers(bucket); // testReadEmptyBlock(bucket); @@ -73,11 +75,7 @@ void testAll() throws Exception { * Test to verify that data read from blocks is stored in a list of buffers * with max capacity equal to the bytes per checksum. */ - private void testReadKeyFully(TestBucket bucket) throws Exception { - String keyName = getNewKeyName(); - int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); - byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); - + private void testReadKeyFully() throws Exception { // Read the data fully into a large enough byte array try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { byte[] readData = new byte[dataLength]; @@ -109,6 +107,28 @@ private void testReadKeyFully(TestBucket bucket) throws Exception { } } + private void testSeek() throws IOException { + java.util.Random random = new java.util.Random(); + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + for (int i = 0; i < 100; i++) { + int position = random.nextInt(dataLength); + keyInputStream.seek(position); + int b = keyInputStream.read(); + assertEquals(inputData[position], (byte) b, "Read data is not same as written data at index " + position); + } + StreamBlockInputStream blockStream = (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); + long length = blockStream.getLength(); + blockStream.seek(10); + long position = blockStream.getPos(); + assertThrows(IOException.class, () -> blockStream.seek(length + 1), + "Seek beyond block length should throw exception"); + assertThrows(IOException.class, () -> blockStream.seek(-1), + "Seeking to a negative position should throw exception"); + assertEquals(position, blockStream.getPos(), + "Position should not change after failed seek attempts"); + } + } + private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); bucket.writeRandomBytes(keyName, CHUNK_SIZE); From f729fcc3fdeeb2dcd22d3692ae5674fa6cbf4bf8 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Tue, 14 Oct 2025 17:34:55 +0100 Subject: [PATCH 089/114] Test for and implement no checksum case --- .../container/keyvalue/KeyValueHandler.java | 30 +++++++++++------- .../rpc/read/TestStreamBlockInputStream.java | 31 ++++++++++++------- 2 files changed, 38 insertions(+), 23 deletions(-) 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 6e1170b79ab9..a44ea12563c0 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 @@ -184,6 +184,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; private final BlockManager blockManager; private final ChunkManager chunkManager; @@ -2077,28 +2078,35 @@ public ContainerCommandResponseProto readBlock( 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. - int bytesPerChecksum = chunkInfos.get(0).getChecksumData().getBytesPerChecksum(); 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; - // 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. - 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(); - int chunkIndex = (int) (adjustedOffset / bytesPerChunk); - int chunkOffset = (int) (adjustedOffset % bytesPerChunk); - int checksumIndex = chunkOffset / bytesPerChecksum; - ByteString checksum = blockData.getChunks().get(chunkIndex).getChecksumData().getChecksums(checksumIndex); - - ChecksumData checksumData = - new ChecksumData(checksumType, bytesPerChecksum, Collections.singletonList(checksum)); + 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(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index a3e983265dcb..f57a105c5a4b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -25,6 +25,7 @@ import java.nio.ByteBuffer; import org.apache.commons.io.IOUtils; 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.storage.StreamBlockInputStream; import org.apache.hadoop.ozone.MiniOzoneCluster; @@ -43,7 +44,6 @@ public class TestStreamBlockInputStream extends TestInputStreamBase { * for each test. */ - private String keyName = getNewKeyName(); private int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); byte[] inputData; private TestBucket bucket; @@ -59,14 +59,21 @@ void testAll() throws Exception { clientConfig.setStreamReadBlock(true); OzoneConfiguration copy = new OzoneConfiguration(conf); copy.setFromObject(clientConfig); + String keyName = getNewKeyName(); try (OzoneClient client = OzoneClientFactory.getRpcClient(copy)) { bucket = TestBucket.newBuilder(client).build(); inputData = bucket.writeRandomBytes(keyName, dataLength); - testReadKeyFully(); - testSeek(); - // testBufferRelease(bucket); - // testCloseReleasesBuffers(bucket); - // testReadEmptyBlock(bucket); + testReadKeyFully(keyName); + testSeek(keyName); + } + keyName = getNewKeyName(); + clientConfig.setChecksumType(ContainerProtos.ChecksumType.NONE); + copy.setFromObject(clientConfig); + try (OzoneClient client = OzoneClientFactory.getRpcClient(copy)) { + bucket = TestBucket.newBuilder(client).build(); + inputData = bucket.writeRandomBytes(keyName, dataLength); + testReadKeyFully(keyName); + testSeek(keyName); } } } @@ -75,9 +82,9 @@ void testAll() throws Exception { * Test to verify that data read from blocks is stored in a list of buffers * with max capacity equal to the bytes per checksum. */ - private void testReadKeyFully() throws Exception { + private void testReadKeyFully(String key) throws Exception { // Read the data fully into a large enough byte array - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(key)) { byte[] readData = new byte[dataLength]; int totalRead = keyInputStream.read(readData, 0, dataLength); assertEquals(dataLength, totalRead); @@ -87,7 +94,7 @@ private void testReadKeyFully() throws Exception { } } // Read the data 1 byte at a time - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(key)) { for (int i = 0; i < dataLength; i++) { int b = keyInputStream.read(); assertEquals(inputData[i], (byte) b, @@ -95,7 +102,7 @@ private void testReadKeyFully() throws Exception { } } // Read the data into a large enough ByteBuffer - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(key)) { ByteBuffer readBuf = ByteBuffer.allocate(dataLength); int totalRead = keyInputStream.read(readBuf); assertEquals(dataLength, totalRead); @@ -107,9 +114,9 @@ private void testReadKeyFully() throws Exception { } } - private void testSeek() throws IOException { + private void testSeek(String key) throws IOException { java.util.Random random = new java.util.Random(); - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(key)) { for (int i = 0; i < 100; i++) { int position = random.nextInt(dataLength); keyInputStream.seek(position); From 94ad4681ad8a34b8baf4565ba7ee7792dd084d0f Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Tue, 14 Oct 2025 17:45:32 +0100 Subject: [PATCH 090/114] Revert "Add ChunkOffsetInBlock to the ChunkInfo proto message to allow checksum offsets to be calculated correctly on the client" This reverts commit 1f90a0a607993d51fedddbfcb8466fa0aa894e80. --- .../ozone/container/common/helpers/ChunkInfo.java | 15 --------------- .../src/main/proto/DatanodeClientProtocol.proto | 1 - 2 files changed, 16 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java index b7fc5629196e..586fe76a49aa 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java @@ -37,7 +37,6 @@ public class ChunkInfo { private ChecksumData checksumData; private final Map metadata; private ByteString stripeChecksum; - private long chunkOffsetInBlock; // For older clients reading chunks in V0 version (all read data should // reside in one buffer). This variable should be set to true for older @@ -98,10 +97,6 @@ public static ChunkInfo getFromProtoBuf(ContainerProtos.ChunkInfo info) chunkInfo.setStripeChecksum(info.getStripeChecksum()); } - if (info.hasChunkOffsetInBlock()) { - chunkInfo.setChunkOffsetInBlock(info.getChunkOffsetInBlock()); - } - return chunkInfo; } @@ -124,7 +119,6 @@ public ContainerProtos.ChunkInfo getProtoBufMessage() { } else { builder.setChecksumData(this.checksumData.getProtoBufMessage()); } - builder.setChunkOffsetInBlock(this.chunkOffsetInBlock); for (Map.Entry entry : metadata.entrySet()) { ContainerProtos.KeyValue.Builder keyValBuilder = @@ -189,21 +183,12 @@ public String getMetadata(String key) { return metadata.get(key); } - public void setChunkOffsetInBlock(long chunkOffsetInBlock) { - this.chunkOffsetInBlock = chunkOffsetInBlock; - } - - public long getChunkOffsetInBlock() { - return chunkOffsetInBlock; - } - @Override public String toString() { return "ChunkInfo{" + "chunkName='" + chunkName + ", offset=" + offset + ", len=" + len + - ", chunkOffsetInBlock=" + chunkOffsetInBlock + '}'; } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index fd225c9fbf67..ccba043179ac 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -437,7 +437,6 @@ message ChunkInfo { repeated KeyValue metadata = 4; required ChecksumData checksumData =5; optional bytes stripeChecksum = 6; - optional uint64 chunkOffsetInBlock = 7; } message ChunkInfoList { From 03898c2d2a29d98b0b5bdac92f0e7ff7b76754e4 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Wed, 15 Oct 2025 17:40:34 +0100 Subject: [PATCH 091/114] Adapted unit tests to the new approach --- .../scm/storage/StreamBlockInputStream.java | 3 +- .../storage/DummyStreamBlockInputStream.java | 141 -------- .../storage/TestStreamBlockInputStream.java | 314 +++++++----------- 3 files changed, 129 insertions(+), 329 deletions(-) delete mode 100644 hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyStreamBlockInputStream.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index eafb7ba5a418..e8bff4266f29 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -120,6 +120,7 @@ public synchronized int read() throws IOException { if (!dataAvailableToRead()) { return EOF; } + position++; return buffer.get(); } @@ -361,7 +362,7 @@ public class StreamingReader implements StreamObserver readByteBuffers = new ArrayList<>(); - private final List chunks; - private final long[] chunkOffsets; - private final Map chunkDataMap; - - @SuppressWarnings("parameternumber") - DummyStreamBlockInputStream( - BlockID blockId, - long blockLen, - Pipeline pipeline, - Token token, - XceiverClientFactory xceiverClientManager, - Function refreshFunction, - OzoneClientConfig config, - List chunks, - Map chunkDataMap) throws IOException { - super(blockId, blockLen, pipeline, token, xceiverClientManager, - refreshFunction, config); - this.chunks = chunks; - this.chunkDataMap = chunkDataMap; - chunkOffsets = new long[chunks.size()]; - long temp = 0; - for (int i = 0; i < chunks.size(); i++) { - chunkOffsets[i] = temp; - temp += chunks.get(i).getLen(); - } - } - - @Override - protected synchronized void checkOpen() throws IOException { - // No action needed - } - - @Override - protected void acquireClient() { - // No action needed - } - - @Override - protected void releaseClient() { - // no-op - } - - protected long readData(long offset, long len) { - int chunkIndex = Arrays.binarySearch(chunkOffsets, offset); - if (chunkIndex < 0) { - chunkIndex = -chunkIndex - 2; - } - ChunkInfo chunkInfo = chunks.get(chunkIndex); - readByteBuffers.clear(); - long chunkOffset = offset - chunkInfo.getOffset(); - if (isVerifyChecksum()) { - ChecksumData checksumData = ChecksumData.getFromProtoBuf( - chunkInfo.getChecksumData()); - int bytesPerChecksum = checksumData.getBytesPerChecksum(); - chunkOffset = (chunkOffset / bytesPerChecksum) * bytesPerChecksum; - } - long bufferOffsetWrtBlockDataData = chunkOffsets[chunkIndex] + chunkOffset; - while (len > 0) { - ChunkInfo currentChunk = chunks.get(chunkIndex); - int bufferCapacity = currentChunk.getChecksumData().getBytesPerChecksum(); - long chunkLen = currentChunk.getLen(); - long remainingToRead = Math.min(chunkLen, len); - if (isVerifyChecksum()) { - if (len < chunkLen) { - final ChecksumData checksumData = ChecksumData.getFromProtoBuf(chunkInfo.getChecksumData()); - final long endByteIndex = len - 1; - final int bytesPerChecksum = checksumData.getBytesPerChecksum(); - remainingToRead = (endByteIndex / bytesPerChecksum + 1) * bytesPerChecksum; - } else { - remainingToRead = chunkLen; - } - } - - long bufferLen; - while (remainingToRead > 0) { - if (remainingToRead < bufferCapacity) { - bufferLen = remainingToRead; - } else { - bufferLen = bufferCapacity; - } - ByteString byteString = ByteString.copyFrom(chunkDataMap.get(chunks.get(chunkIndex).getChunkName()), - (int) chunkOffset, (int) bufferLen); - - readByteBuffers.add(byteString); - - chunkOffset += bufferLen; - remainingToRead -= bufferLen; - len -= bufferLen; - } - chunkOffset = 0; - chunkIndex++; - } - return bufferOffsetWrtBlockDataData; - } - - public List getReadByteBuffers() { - return readByteBuffers; - } -} diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index fd4f53bd10ab..27dabba73310 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -17,249 +17,189 @@ package org.apache.hadoop.hdds.scm.storage; -import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.when; -import com.google.common.primitives.Bytes; -import java.io.EOFException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; +import java.io.IOException; +import java.security.SecureRandom; import java.util.function.Function; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientGrpc; import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; 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.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.invocation.InvocationOnMock; /** * Tests for {@link TestStreamBlockInputStream}'s functionality. */ public class TestStreamBlockInputStream { - private int blockSize; - private static final int CHUNK_SIZE = 100; - private static final int BYTES_PER_CHECKSUM = 20; - private static final Random RANDOM = new Random(); - private DummyStreamBlockInputStream blockStream; - private byte[] blockData; - private List chunks; - private Map chunkDataMap; - private Checksum checksum; + private static final int BYTES_PER_CHECKSUM = 1024; + private static final int BLOCK_SIZE = 1024; + private StreamBlockInputStream blockStream; private BlockID blockID; - private static final String CHUNK_NAME = "chunk-"; private OzoneConfiguration conf = new OzoneConfiguration(); + private XceiverClientFactory xceiverClientFactory; + private XceiverClientGrpc xceiverClient; + private Pipeline pipeline; + private Checksum checksum; + private ChecksumData checksumData; + private byte[] data; + ClientCallStreamObserver requestObserver; @BeforeEach public void setup() throws Exception { + Token token = mock(Token.class); + when(token.encodeToUrlString()).thenReturn("url"); + pipeline = MockPipeline.createSingleNodePipeline(); + xceiverClient = mock(XceiverClientGrpc.class); + when(xceiverClient.getPipeline()).thenReturn(pipeline); + xceiverClientFactory = mock(XceiverClientFactory.class); + when(xceiverClientFactory.acquireClientForReadData(any())) + .thenReturn(xceiverClient); + requestObserver = mock(ClientCallStreamObserver.class); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setStreamReadBlock(true); Function refreshFunction = mock(Function.class); blockID = new BlockID(new ContainerBlockID(1, 1)); checksum = new Checksum(ChecksumType.CRC32, BYTES_PER_CHECKSUM); - createChunkList(5); - - Pipeline pipeline = MockPipeline.createSingleNodePipeline(); - blockStream = new DummyStreamBlockInputStream(blockID, blockSize, pipeline, - null, null, refreshFunction, clientConfig, chunks, chunkDataMap); + createDataAndChecksum(); + blockStream = new StreamBlockInputStream(blockID, BLOCK_SIZE, pipeline, + token, xceiverClientFactory, refreshFunction, clientConfig); } - /** - * Create a mock list of chunks. The first n-1 chunks of length CHUNK_SIZE - * and the last chunk with length CHUNK_SIZE/2. - */ - private void createChunkList(int numChunks) - throws Exception { - - chunks = new ArrayList<>(numChunks); - chunkDataMap = new HashMap<>(); - blockData = new byte[0]; - int i, chunkLen; - byte[] byteData; - String chunkName; - - for (i = 0; i < numChunks; i++) { - chunkName = CHUNK_NAME + i; - chunkLen = CHUNK_SIZE; - if (i == numChunks - 1) { - chunkLen = CHUNK_SIZE / 2; + @AfterEach + public void teardown() { + if (blockStream != null) { + try { + blockStream.close(); + } catch (IOException e) { + // ignore } - byteData = generateRandomData(chunkLen); - ChunkInfo chunkInfo = ChunkInfo.newBuilder() - .setChunkName(chunkName) - .setOffset(0) - .setLen(chunkLen) - .setChecksumData(checksum.computeChecksum( - byteData, 0, chunkLen).getProtoBufMessage()) - .build(); - - chunkDataMap.put(chunkName, byteData); - chunks.add(chunkInfo); - - blockSize += chunkLen; - blockData = Bytes.concat(blockData, byteData); } } - static byte[] generateRandomData(int length) { - byte[] bytes = new byte[length]; - RANDOM.nextBytes(bytes); - return bytes; - } - - /** - * Match readData with the chunkData byte-wise. - * @param readData Data read through ChunkInputStream - * @param inputDataStartIndex first index (inclusive) in chunkData to compare - * with read data - * @param length the number of bytes of data to match starting from - * inputDataStartIndex - */ - private void matchWithInputData(byte[] readData, int inputDataStartIndex, - int length) { - for (int i = inputDataStartIndex; i < inputDataStartIndex + length; i++) { - assertEquals(blockData[i], readData[i - inputDataStartIndex], "i: " + i); - } - } - - private void matchWithInputData(List byteStrings, - int inputDataStartIndex, int length) { - int offset = inputDataStartIndex; - int totalBufferLen = 0; - for (ByteString byteString : byteStrings) { - int bufferLen = byteString.size(); - matchWithInputData(byteString.toByteArray(), offset, bufferLen); - offset += bufferLen; - totalBufferLen += bufferLen; - } - assertEquals(length, totalBufferLen); + @Test + public void testCloseStreamReleasesResources() throws IOException { + setupSuccessfulRead(); + assertEquals(data[0], blockStream.read()); + blockStream.close(); + // Verify that cancel() was called on the requestObserver mock + org.mockito.Mockito.verify(requestObserver).cancel(any(), any()); + // Verify that release() was called on the xceiverClient mock + org.mockito.Mockito.verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); } - /** - * Seek to a position and verify through getPos(). - */ - private void seekAndVerify(int pos) throws Exception { - blockStream.seek(pos); - assertEquals(pos, blockStream.getPos(), - "Current position of buffer does not match with the sought position"); + @Test + public void testUnbufferReleasesResourcesAndResumesFromLastPosition() throws IOException { + setupSuccessfulRead(); + assertEquals(data[0], blockStream.read()); + assertEquals(1, blockStream.getPos()); + blockStream.unbuffer(); + // Verify that cancel() was called on the requestObserver mock + org.mockito.Mockito.verify(requestObserver).cancel(any(), any()); + // Verify that release() was called on the xceiverClient mock + org.mockito.Mockito.verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); + // The next read should "rebuffer" and continue from the last position + assertEquals(data[1], blockStream.read()); + assertEquals(2, blockStream.getPos()); } @Test - public void testFullChunkRead() throws Exception { - byte[] b = new byte[blockSize]; - int numBytesRead = blockStream.read(b, 0, blockSize); - assertEquals(blockSize, numBytesRead); - matchWithInputData(b, 0, blockSize); + public void testSeekReleasesTheStreamAndStartsFromNewPosition() throws IOException { + setupSuccessfulRead(); + assertEquals(data[0], blockStream.read()); + blockStream.seek(100); + assertEquals(100, blockStream.getPos()); + // Verify that cancel() was called on the requestObserver mock + org.mockito.Mockito.verify(requestObserver).cancel(any(), any()); + // The xceiverClient should not be released + org.mockito.Mockito.verify(xceiverClientFactory, never()) + .releaseClientForReadData(xceiverClient, false); + + assertEquals(data[100], blockStream.read()); + assertEquals(101, blockStream.getPos()); } @Test - public void testPartialChunkRead() throws Exception { - int len = blockSize / 2; - byte[] b = new byte[len]; - - int numBytesRead = blockStream.read(b, 0, len); - assertEquals(len, numBytesRead); - matchWithInputData(b, 0, len); - - // To read block data from index 0 to 225 (len = 225), we need to read - // chunk from offset 0 to 240 as the checksum boundary is at every 20 - // bytes. Verify that 60 bytes of chunk data are read and stored in the - // buffers. Since checksum boundary is at every 20 bytes, there should be - // 240/20 number of buffers. - matchWithInputData(blockStream.getReadByteBuffers(), 0, 240); + public void testErrorThrownIfStreamReturnsError() throws IOException { + // Note the error will only be thrown when the buffer needs to be refilled. I think case, as its the first + // read it will try to fill the buffer and encounter the error, but a reader could continue reading until the + // buffer is exhausted before seeing the error. + when(xceiverClient.streamRead(any(), any())).thenAnswer((InvocationOnMock invocation) -> { + StreamObserver streamObserver = invocation.getArgument(1); + streamObserver.onError(new IOException("Test induced error")); + return requestObserver; + }); + assertThrows(IOException.class, () -> blockStream.read()); } @Test - public void testSeek() throws Exception { - seekAndVerify(0); - EOFException eofException = assertThrows(EOFException.class, () -> seekAndVerify(blockSize + 1)); - assertThat(eofException).hasMessage("EOF encountered at pos: " + (blockSize + 1) + " for block: " + blockID); - - // Seek before read should update the BlockInputStream#blockPosition - seekAndVerify(25); - - // Read from the sought position. - // Reading from index 25 to 54 should result in the BlockInputStream - // copying chunk data from index 20 to 59 into the buffers (checksum - // boundaries). - byte[] b = new byte[30]; - int numBytesRead = blockStream.read(b, 0, 30); - assertEquals(30, numBytesRead); - matchWithInputData(b, 25, 30); - matchWithInputData(blockStream.getReadByteBuffers(), 20, 40); - - // After read, the position of the blockStream is evaluated from the - // buffers and the chunkPosition should be reset to -1. - - // Only the last BYTES_PER_CHECKSUM will be cached in the buffers as - // buffers are released after each checksum boundary is read. So the - // buffers should contain data from index 40 to 59. - // Seek to a position within the cached buffers. BlockPosition should - // still not be used to set the position. - seekAndVerify(45); - - // Seek to a position outside the current cached buffers. In this case, the - // chunkPosition should be updated to the seeked position. - seekAndVerify(75); - - // Read upto checksum boundary should result in all the buffers being - // released and hence chunkPosition updated with current position of chunk. - seekAndVerify(25); - b = new byte[15]; - numBytesRead = blockStream.read(b, 0, 15); - assertEquals(15, numBytesRead); - matchWithInputData(b, 25, 15); + public void seekOutOfBounds() throws IOException { + setupSuccessfulRead(); + assertThrows(IOException.class, () -> blockStream.seek(-1)); + assertThrows(IOException.class, () -> blockStream.seek(BLOCK_SIZE + 1)); } @Test - public void testSeekAndRead() throws Exception { - // Seek to a position and read data - seekAndVerify(50); - byte[] b1 = new byte[20]; - int numBytesRead = blockStream.read(b1, 0, 20); - assertEquals(20, numBytesRead); - matchWithInputData(b1, 50, 20); - - // Next read should start from the position of the last read + 1 i.e. 70 - byte[] b2 = new byte[20]; - numBytesRead = blockStream.read(b2, 0, 20); - assertEquals(20, numBytesRead); - matchWithInputData(b2, 70, 20); - - byte[] b3 = new byte[20]; - seekAndVerify(80); - numBytesRead = blockStream.read(b3, 0, 20); - assertEquals(20, numBytesRead); - matchWithInputData(b3, 80, 20); + public void readPastEOFReturnsEOF() throws IOException { + setupSuccessfulRead(); + blockStream.seek(BLOCK_SIZE); + // Ensure the stream is at EOF even after two attempts to read + assertEquals(-1, blockStream.read()); + assertEquals(-1, blockStream.read()); + assertEquals(BLOCK_SIZE, blockStream.getPos()); } - @Test - public void testUnbuffered() throws Exception { - byte[] b1 = new byte[20]; - int numBytesRead = blockStream.read(b1, 0, 20); - assertEquals(20, numBytesRead); - matchWithInputData(b1, 0, 20); - blockStream.unbuffer(); + private void createDataAndChecksum() throws OzoneChecksumException { + data = new byte[BLOCK_SIZE]; + new SecureRandom().nextBytes(data); + checksumData = checksum.computeChecksum(data); + } - // assertFalse(blockStream.buffersAllocated()); + private void setupSuccessfulRead() throws IOException { + when(xceiverClient.streamRead(any(), any())).thenAnswer((InvocationOnMock invocation) -> { + StreamObserver streamObserver = invocation.getArgument(1); + streamObserver.onNext(createChunkResponse()); + streamObserver.onCompleted(); + return requestObserver; + }); + } - // Next read should start from the position of the last read + 1 i.e. 20 - byte[] b2 = new byte[20]; - numBytesRead = blockStream.read(b2, 0, 20); - assertEquals(20, numBytesRead); - matchWithInputData(b2, 20, 20); + private ContainerProtos.ContainerCommandResponseProto createChunkResponse() { + ContainerProtos.ReadBlockResponseProto response = ContainerProtos.ReadBlockResponseProto.newBuilder() + .setChecksumData(checksumData.getProtoBufMessage()) + .setData(ByteString.copyFrom(data)) + .setOffset(0) + .build(); + + return ContainerProtos.ContainerCommandResponseProto.newBuilder() + .setCmdType(ContainerProtos.Type.ReadBlock) + .setReadBlock(response) + .setResult(ContainerProtos.Result.SUCCESS) + .build(); } } From 10f30944d7aebd35ff38028fe2d8c796abd7d0af Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 11:00:31 +0100 Subject: [PATCH 092/114] Fix style and remove commented tests --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 3 - .../scm/storage/StreamBlockInputStream.java | 4 +- .../storage/TestStreamBlockInputStream.java | 3 +- .../scm/storage/ContainerProtocolCalls.java | 1 - .../container/keyvalue/KeyValueHandler.java | 3 - .../rpc/read/TestStreamBlockInputStream.java | 118 +----------------- 6 files changed, 8 insertions(+), 124 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index aa9c9a17843c..72926674e817 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -44,8 +44,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc; import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc.XceiverClientProtocolServiceStub; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -61,7 +59,6 @@ import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.util.Time; -import org.apache.http.concurrent.Cancellable; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index e8bff4266f29..d250b9f9ab4c 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -265,7 +265,6 @@ private int fillBuffer() throws IOException { return buffer == null ? EOF : buffer.limit(); } - protected synchronized void releaseClient() { if (xceiverClientFactory != null && xceiverClient != null) { closeStream(); @@ -352,6 +351,9 @@ private void handleIOException(IOException ioe) throws IOException { } } + /** + * Implementation of a StreamObserver used to received and buffer streaming GRPC reads. + */ public class StreamingReader implements StreamObserver { private final BlockingQueue responseQueue = new LinkedBlockingQueue<>(1); diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 27dabba73310..13843ce1c5fa 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -65,7 +65,7 @@ public class TestStreamBlockInputStream { private Checksum checksum; private ChecksumData checksumData; private byte[] data; - ClientCallStreamObserver requestObserver; + private ClientCallStreamObserver requestObserver; @BeforeEach public void setup() throws Exception { @@ -172,7 +172,6 @@ public void readPastEOFReturnsEOF() throws IOException { assertEquals(BLOCK_SIZE, blockStream.getPos()); } - private void createDataAndChecksum() throws OzoneChecksumException { data = new byte[BLOCK_SIZE]; new SecureRandom().nextBytes(data); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index a3df45b8c248..ce3e6062400b 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -56,7 +56,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockRequestProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadChunkResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerRequestProto; 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 a44ea12563c0..55e7d9faa4de 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 @@ -57,7 +57,6 @@ import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion; import static org.apache.hadoop.ozone.OzoneConsts.INCREMENTAL_CHUNK_LIST; -import static org.apache.hadoop.ozone.OzoneConsts.LAYOUTVERSION; import static org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient.createSingleNodePipeline; import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.DEFAULT_LAYOUT; import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_BLOCK; @@ -66,7 +65,6 @@ import com.google.common.base.Preconditions; import com.google.common.util.concurrent.Striped; import java.io.File; -import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.FilenameFilter; import java.io.IOException; @@ -81,7 +79,6 @@ import java.time.Clock; import java.time.Instant; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index f57a105c5a4b..e544f313cbf9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.commons.io.IOUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; @@ -45,10 +44,9 @@ public class TestStreamBlockInputStream extends TestInputStreamBase { */ private int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); - byte[] inputData; + private byte[] inputData; private TestBucket bucket; - // @ContainerLayoutTestInfo.ContainerTest @Test void testAll() throws Exception { try (MiniOzoneCluster cluster = newCluster()) { @@ -65,6 +63,7 @@ void testAll() throws Exception { inputData = bucket.writeRandomBytes(keyName, dataLength); testReadKeyFully(keyName); testSeek(keyName); + testReadEmptyBlock(); } keyName = getNewKeyName(); clientConfig.setChecksumType(ContainerProtos.ChecksumType.NONE); @@ -136,121 +135,12 @@ private void testSeek(String key) throws IOException { } } - private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { + private void testReadEmptyBlock() throws Exception { String keyName = getNewKeyName(); - bucket.writeRandomBytes(keyName, CHUNK_SIZE); - - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - StreamBlockInputStream block0Stream = - (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); - - readDataFromBlock(block0Stream, 0, 1); - // assertNotNull(block0Stream.getCachedBuffers()); - - block0Stream.close(); - - // assertNull(block0Stream.getCachedBuffers()); - } - } - - /** - * Test that ChunkInputStream buffers are released as soon as the last byte - * of the buffer is read. - */ - private void testBufferRelease(TestBucket bucket) throws Exception { - String keyName = getNewKeyName(); - byte[] inputData = bucket.writeRandomBytes(keyName, CHUNK_SIZE); - - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - - StreamBlockInputStream block0Stream = - (StreamBlockInputStream) keyInputStream.getPartStreams().get(0); - - // Read checksum boundary - 1 bytes of data - int readDataLen = BYTES_PER_CHECKSUM - 1; - byte[] readData = readDataFromBlock(block0Stream, 0, readDataLen); - bucket.validateData(inputData, 0, readData); - - // There should be 1 byte of data remaining in the buffer which is not - // yet read. Hence, the buffer should not be released. - // checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - // assertEquals(1, block0Stream.getCachedBuffers()[0].remaining()); - - // Reading the last byte in the buffer should result in all the buffers - // being released. - readData = readDataFromBlock(block0Stream, 1); - bucket.validateData(inputData, readDataLen, readData); - // assertNull(block0Stream.getCachedBuffers(), - // "Chunk stream buffers not released after last byte is read"); -// - // Read more data to get the data till the next checksum boundary. - readDataLen = BYTES_PER_CHECKSUM / 2; - readDataFromBlock(block0Stream, readDataLen); - // There should be one buffer and the buffer should not be released as - // there is data pending to be read from the buffer - // checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - // ByteBuffer lastCachedBuffer = block0Stream.getCachedBuffers()[0]; - // assertEquals(BYTES_PER_CHECKSUM - readDataLen, - // lastCachedBuffer.remaining()); - - // Read more than the remaining data in buffer (but less than the next - // checksum boundary). - int position = (int) block0Stream.getPos(); - // readDataLen = lastCachedBuffer.remaining() + BYTES_PER_CHECKSUM / 2; - readData = readDataFromBlock(block0Stream, readDataLen); - bucket.validateData(inputData, position, readData); - // After reading the remaining data in the buffer, the buffer should be - // released and next checksum size of data must be read into the buffers - // checkBufferSizeAndCapacity(block0Stream.getCachedBuffers()); - // Verify that the previously cached buffer is released by comparing it - // with the current cached buffer - // assertNotEquals(lastCachedBuffer, - // block0Stream.getCachedBuffers()[0]); - } - } - - private byte[] readDataFromBlock(StreamBlockInputStream streamBlockInputStream, - int offset, int readDataLength) throws IOException { - byte[] readData = new byte[readDataLength]; - streamBlockInputStream.seek(offset); - IOUtils.readFully(streamBlockInputStream, readData); - return readData; - } - - private byte[] readDataFromBlock(StreamBlockInputStream streamBlockInputStream, - int readDataLength) throws IOException { - byte[] readData = new byte[readDataLength]; - IOUtils.readFully(streamBlockInputStream, readData); - return readData; - } - - /** - * Verify number of buffers and their capacities. - * @param buffers chunk stream buffers - */ - private void checkBufferSizeAndCapacity(ByteBuffer[] buffers) { - assertEquals(1, buffers.length, - "ChunkInputStream does not have expected number of " + - "ByteBuffers"); - for (ByteBuffer buffer : buffers) { - assertEquals(BYTES_PER_CHECKSUM, buffer.capacity(), - "ChunkInputStream ByteBuffer capacity is wrong"); - } - } - - private void testReadEmptyBlock(TestBucket bucket) throws Exception { - String keyName = getNewKeyName(); - int dataLength = 10; bucket.writeRandomBytes(keyName, 0); - try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { - - byte[] readData = new byte[dataLength]; assertTrue(keyInputStream.getPartStreams().isEmpty()); - IOUtils.read(keyInputStream, readData); - for (byte b : readData) { - assertEquals((byte) 0, b); - } + assertEquals(-1, keyInputStream.read()); } } } From 877df6602befd90c73ab5247c735aa76a14beb2b Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 11:04:32 +0100 Subject: [PATCH 093/114] Remove synchronization from streaming reader --- .../hadoop/hdds/scm/storage/StreamBlockInputStream.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index d250b9f9ab4c..b8d29262ec70 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -428,7 +428,7 @@ public void cancel() { } @Override - public synchronized void onNext(ContainerProtos.ContainerCommandResponseProto containerCommandResponseProto) { + public void onNext(ContainerProtos.ContainerCommandResponseProto containerCommandResponseProto) { try { ReadBlockResponseProto readBlock = containerCommandResponseProto.getReadBlock(); ByteBuffer data = readBlock.getData().asReadOnlyByteBuffer(); @@ -444,7 +444,7 @@ public synchronized void onNext(ContainerProtos.ContainerCommandResponseProto co } @Override - public synchronized void onError(Throwable throwable) { + public void onError(Throwable throwable) { if (throwable == CANCELLED_EXCEPTION) { completed.set(true); } else { @@ -454,7 +454,7 @@ public synchronized void onError(Throwable throwable) { } @Override - public synchronized void onCompleted() { + public void onCompleted() { completed.set(true); } From 5525770f5bf5e7cb60172d43899b1c7ee5636d74 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 11:11:31 +0100 Subject: [PATCH 094/114] Remove or comment out unused code --- .../scm/storage/StreamBlockInputStream.java | 31 +++++++------------ .../container/keyvalue/KeyValueHandler.java | 18 ----------- 2 files changed, 12 insertions(+), 37 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index b8d29262ec70..a1cd04a24d18 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -19,10 +19,8 @@ import static org.apache.hadoop.hdds.client.ReplicationConfig.getLegacyFactor; -import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.nio.ByteBuffer; -import java.time.Instant; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -42,7 +40,6 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.client.HddsClientUtils; -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; @@ -50,7 +47,6 @@ 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.Status; import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.slf4j.Logger; @@ -273,7 +269,16 @@ protected synchronized void releaseClient() { } } - private boolean shouldRetryRead(IOException cause) throws IOException { + @Override + public synchronized void close() throws IOException { + releaseClient(); + xceiverClientFactory = null; + } + + /* + Commenting out for now as we probably need these for retries which are yet to be implemented. + + private boolean shouldRetryRead(IOException cause) throws IOException { RetryPolicy.RetryAction retryAction; try { retryAction = retryPolicy.shouldRetry(cause, ++retries, 0, true); @@ -285,11 +290,6 @@ private boolean shouldRetryRead(IOException cause) throws IOException { return retryAction.action == RetryPolicy.RetryAction.RetryDecision.RETRY; } - @VisibleForTesting - public boolean isVerifyChecksum() { - return verifyChecksum; - } - private void refreshBlockInfo(IOException cause) throws IOException { LOG.info("Attempting to update pipeline and block token for block {} from pipeline {}: {}", blockID, pipelineRef.get().getId(), cause.getMessage()); @@ -316,19 +316,10 @@ private void refreshBlockInfo(IOException cause) throws IOException { } } - /** - * Check if this exception is because datanodes are not reachable. - */ private boolean isConnectivityIssue(IOException ex) { return Status.fromThrowable(ex).getCode() == Status.UNAVAILABLE.getCode(); } - @Override - public synchronized void close() throws IOException { - releaseClient(); - xceiverClientFactory = null; - } - private void handleStorageContainerException(StorageContainerException e) throws IOException { if (shouldRetryRead(e)) { releaseClient(); @@ -351,6 +342,8 @@ private void handleIOException(IOException ioe) throws IOException { } } + */ + /** * Implementation of a StreamObserver used to received and buffer streaming GRPC reads. */ 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 55e7d9faa4de..c10babbe381b 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 @@ -92,7 +92,6 @@ import java.util.concurrent.locks.Lock; import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.client.BlockID; @@ -2121,23 +2120,6 @@ public ContainerCommandResponseProto readBlock( return responseProto; } - private Pair computeChecksumBoundaries( - ContainerProtos.ChunkInfo chunkInfo, long startByteIndex, long dataLen) { - - int bytesPerChecksum = chunkInfo.getChecksumData().getBytesPerChecksum(); - - // index of the last byte to be read from chunk, inclusively. - final long endByteIndex = startByteIndex + dataLen - 1; - - long adjustedChunkOffset = (startByteIndex / bytesPerChecksum) - * bytesPerChecksum; // inclusive - final long endIndex = ((endByteIndex / bytesPerChecksum) + 1) - * bytesPerChecksum; // exclusive - long adjustedChunkLen = - Math.min(endIndex, chunkInfo.getLen()) - adjustedChunkOffset; - return Pair.of(adjustedChunkOffset, adjustedChunkLen); - } - @Override public void addFinalizedBlock(Container container, long localID) { KeyValueContainer keyValueContainer = (KeyValueContainer)container; From 8d3ddffe95083ce8c1bcc3e8419a028fd662bfc2 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 12:16:58 +0100 Subject: [PATCH 095/114] Throw IOException for ops attempts after closed --- .../scm/storage/StreamBlockInputStream.java | 23 +++++++++---------- .../storage/TestStreamBlockInputStream.java | 13 +++++++++++ 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index a1cd04a24d18..a105294ffbda 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -30,6 +30,7 @@ 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.client.StandaloneReplicationConfig; @@ -113,6 +114,7 @@ public synchronized long getPos() { @Override public synchronized int read() throws IOException { + checkOpen(); if (!dataAvailableToRead()) { return EOF; } @@ -128,6 +130,7 @@ public synchronized int read(byte[] b, int off, int len) throws IOException { @Override public synchronized int read(ByteBuffer targetBuf) throws IOException { + checkOpen(); int read = 0; while (targetBuf.hasRemaining()) { if (!dataAvailableToRead()) { @@ -163,6 +166,7 @@ protected int readWithStrategy(ByteReaderStrategy strategy) throws IOException { @Override public synchronized void seek(long pos) throws IOException { + checkOpen(); if (pos < 0) { throw new IOException("Cannot seek to negative offset"); } @@ -204,25 +208,21 @@ private void setPipeline(Pipeline pipeline) throws IOException { long replicaIndexes = pipeline.getNodes().stream().mapToInt(pipeline::getReplicaIndex).distinct().count(); if (replicaIndexes > 1) { - throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", - pipeline)); + throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", pipeline)); } - // irrespective of the container state, we will always read via Standalone - // protocol. - boolean okForRead = - pipeline.getType() == HddsProtos.ReplicationType.STAND_ALONE - || pipeline.getType() == HddsProtos.ReplicationType.EC; + // irrespective of the container state, we will always read via Standalone protocol. + boolean okForRead = pipeline.getType() == HddsProtos.ReplicationType.STAND_ALONE + || pipeline.getType() == HddsProtos.ReplicationType.EC; Pipeline readPipeline = okForRead ? pipeline : pipeline.copyForRead().toBuilder() - .setReplicationConfig(StandaloneReplicationConfig.getInstance( - getLegacyFactor(pipeline.getReplicationConfig()))) + .setReplicationConfig(StandaloneReplicationConfig.getInstance(getLegacyFactor(pipeline.getReplicationConfig()))) .build(); pipelineRef.set(readPipeline); } protected synchronized void checkOpen() throws IOException { if (xceiverClientFactory == null) { - throw new IOException("StreamBlockInputStream has been closed."); + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED + " Block: " + blockID); } } @@ -233,8 +233,7 @@ protected synchronized void acquireClient() throws IOException { try { xceiverClient = xceiverClientFactory.acquireClientForReadData(pipeline); } catch (IOException ioe) { - LOG.warn("Failed to acquire client for pipeline {}, block {}", - pipeline, blockID); + LOG.warn("Failed to acquire client for pipeline {}, block {}", pipeline, blockID); throw ioe; } } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 13843ce1c5fa..86bdb8c69e1f 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.when; import java.io.IOException; +import java.nio.ByteBuffer; import java.security.SecureRandom; import java.util.function.Function; import org.apache.hadoop.hdds.client.BlockID; @@ -172,6 +173,18 @@ public void readPastEOFReturnsEOF() throws IOException { assertEquals(BLOCK_SIZE, blockStream.getPos()); } + @Test + public void ensureExceptionThrownForReadAfterClosed() throws IOException { + setupSuccessfulRead(); + blockStream.close(); + ByteBuffer byteBuffer = ByteBuffer.allocate(10); + byte[] byteArray = new byte[10]; + assertThrows(IOException.class, () -> blockStream.read()); + assertThrows(IOException.class, () -> blockStream.read(byteArray, 0, 10)); + assertThrows(IOException.class, () -> blockStream.read(byteBuffer)); + assertThrows(IOException.class, () -> blockStream.seek(10)); + } + private void createDataAndChecksum() throws OzoneChecksumException { data = new byte[BLOCK_SIZE]; new SecureRandom().nextBytes(data); From 7a37fcc0dfb5348062d2ac8a999b8db0279e9e00 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 12:33:55 +0100 Subject: [PATCH 096/114] Move setPipeline into shared superclass --- .../scm/storage/BlockExtendedInputStream.java | 21 +++++++++++++++ .../hdds/scm/storage/BlockInputStream.java | 25 ++--------------- .../scm/storage/StreamBlockInputStream.java | 27 ++----------------- .../storage/TestStreamBlockInputStream.java | 8 +++--- 4 files changed, 28 insertions(+), 53 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java index 6753f600a91b..02a0ca65c665 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java @@ -17,7 +17,10 @@ package org.apache.hadoop.hdds.scm.storage; +import java.io.IOException; import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; /** * Abstract class used as an interface for input streams related to Ozone @@ -38,4 +41,22 @@ public long getRemaining() { @Override public abstract long getPos(); + + protected Pipeline setPipeline(Pipeline pipeline) throws IOException { + if (pipeline == null) { + return null; + } + long replicaIndexes = pipeline.getNodes().stream().mapToInt(pipeline::getReplicaIndex).distinct().count(); + + if (replicaIndexes > 1) { + throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", + pipeline)); + } + + // irrespective of the container state, we will always read via Standalone protocol. + boolean okForRead = pipeline.getType() == HddsProtos.ReplicationType.STAND_ALONE + || pipeline.getType() == HddsProtos.ReplicationType.EC; + return okForRead ? pipeline : pipeline.copyForRead(); + } + } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index c9731bcc4611..2cdb4139a540 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetBlockResponseProto; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; @@ -123,7 +122,7 @@ public BlockInputStream( this.blockInfo = blockInfo; this.blockID = blockInfo.getBlockID(); this.length = blockInfo.getLength(); - setPipeline(pipeline); + pipelineRef.set(setPipeline(pipeline)); tokenRef.set(token); this.verifyChecksum = config.isChecksumVerify(); this.xceiverClientFactory = xceiverClientFactory; @@ -231,7 +230,7 @@ private void refreshBlockInfo(IOException cause) throws IOException { if (blockLocationInfo == null) { LOG.warn("No new block location info for block {}", blockID); } else { - setPipeline(blockLocationInfo.getPipeline()); + pipelineRef.set((blockLocationInfo.getPipeline())); LOG.info("New pipeline for block {}: {}", blockID, blockLocationInfo.getPipeline()); @@ -277,26 +276,6 @@ protected BlockData getBlockDataUsingClient() throws IOException { return response.getBlockData(); } - private void setPipeline(Pipeline pipeline) throws IOException { - if (pipeline == null) { - return; - } - long replicaIndexes = pipeline.getNodes().stream().mapToInt(pipeline::getReplicaIndex).distinct().count(); - - if (replicaIndexes > 1) { - throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", - pipeline)); - } - - // irrespective of the container state, we will always read via Standalone - // protocol. - boolean okForRead = - pipeline.getType() == HddsProtos.ReplicationType.STAND_ALONE - || pipeline.getType() == HddsProtos.ReplicationType.EC; - Pipeline readPipeline = okForRead ? pipeline : pipeline.copyForRead(); - pipelineRef.set(readPipeline); - } - private static void validate(ContainerCommandResponseProto response) throws IOException { if (!response.hasGetBlock()) { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index a105294ffbda..c5da2a85ef0a 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -17,8 +17,6 @@ package org.apache.hadoop.hdds.scm.storage; -import static org.apache.hadoop.hdds.client.ReplicationConfig.getLegacyFactor; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.concurrent.BlockingQueue; @@ -33,10 +31,8 @@ import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.Seekable; import org.apache.hadoop.hdds.client.BlockID; -import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; @@ -88,7 +84,7 @@ public StreamBlockInputStream( OzoneClientConfig config) throws IOException { this.blockID = blockID; this.blockLength = length; - setPipeline(pipeline); + pipelineRef.set(setPipeline(pipeline)); tokenRef.set(token); this.xceiverClientFactory = xceiverClientFactory; this.verifyChecksum = config.isChecksumVerify(); @@ -201,25 +197,6 @@ private void closeStream() { buffer = null; } - private void setPipeline(Pipeline pipeline) throws IOException { - if (pipeline == null) { - return; - } - long replicaIndexes = pipeline.getNodes().stream().mapToInt(pipeline::getReplicaIndex).distinct().count(); - - if (replicaIndexes > 1) { - throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", pipeline)); - } - - // irrespective of the container state, we will always read via Standalone protocol. - boolean okForRead = pipeline.getType() == HddsProtos.ReplicationType.STAND_ALONE - || pipeline.getType() == HddsProtos.ReplicationType.EC; - Pipeline readPipeline = okForRead ? pipeline : pipeline.copyForRead().toBuilder() - .setReplicationConfig(StandaloneReplicationConfig.getInstance(getLegacyFactor(pipeline.getReplicationConfig()))) - .build(); - pipelineRef.set(readPipeline); - } - protected synchronized void checkOpen() throws IOException { if (xceiverClientFactory == null) { throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED + " Block: " + blockID); @@ -298,7 +275,7 @@ private void refreshBlockInfo(IOException cause) throws IOException { if (blockLocationInfo == null) { LOG.warn("No new block location info for block {}", blockID); } else { - setPipeline(blockLocationInfo.getPipeline()); + pipelineRef.set(getPipeline(blockLocationInfo.getPipeline())); LOG.info("New pipeline for block {}: {}", blockID, blockLocationInfo.getPipeline()); diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 86bdb8c69e1f..512c5fde7309 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -58,11 +58,9 @@ public class TestStreamBlockInputStream { private static final int BYTES_PER_CHECKSUM = 1024; private static final int BLOCK_SIZE = 1024; private StreamBlockInputStream blockStream; - private BlockID blockID; - private OzoneConfiguration conf = new OzoneConfiguration(); + private final OzoneConfiguration conf = new OzoneConfiguration(); private XceiverClientFactory xceiverClientFactory; private XceiverClientGrpc xceiverClient; - private Pipeline pipeline; private Checksum checksum; private ChecksumData checksumData; private byte[] data; @@ -72,7 +70,7 @@ public class TestStreamBlockInputStream { public void setup() throws Exception { Token token = mock(Token.class); when(token.encodeToUrlString()).thenReturn("url"); - pipeline = MockPipeline.createSingleNodePipeline(); + Pipeline pipeline = MockPipeline.createSingleNodePipeline(); xceiverClient = mock(XceiverClientGrpc.class); when(xceiverClient.getPipeline()).thenReturn(pipeline); xceiverClientFactory = mock(XceiverClientFactory.class); @@ -83,7 +81,7 @@ public void setup() throws Exception { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setStreamReadBlock(true); Function refreshFunction = mock(Function.class); - blockID = new BlockID(new ContainerBlockID(1, 1)); + BlockID blockID = new BlockID(new ContainerBlockID(1, 1)); checksum = new Checksum(ChecksumType.CRC32, BYTES_PER_CHECKSUM); createDataAndChecksum(); blockStream = new StreamBlockInputStream(blockID, BLOCK_SIZE, pipeline, From 559c8a145ee77a8553bb4faceca7fe43e1a66c48 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 12:41:46 +0100 Subject: [PATCH 097/114] Fix findbugs warning --- .../hadoop/hdds/scm/storage/StreamBlockInputStream.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index c5da2a85ef0a..dbe81bcca714 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -368,13 +368,14 @@ public ByteBuffer readNext() throws IOException { // If the reader position is behind ByteBuffer buf = readBlock.getData().asReadOnlyByteBuffer(); long blockOffset = readBlock.getOffset(); - if (position < blockOffset) { + long pos = getPos(); + if (pos < blockOffset) { // This should not happen, and if it does, we have a bug. throw new IOException("Received data out of order. Position is " + position + " but received data at " + blockOffset); } - if (position > readBlock.getOffset()) { - int offset = (int)(position - readBlock.getOffset()); + if (pos > readBlock.getOffset()) { + int offset = (int)(pos - readBlock.getOffset()); buf.position(offset); } return buf; From 71c89f0c7f90f63964d4e2bc0fa9f05c81e901aa Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 12:45:44 +0100 Subject: [PATCH 098/114] Fix PMD --- .../hdds/scm/storage/StreamBlockInputStream.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index dbe81bcca714..c6c42bf803e8 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -36,10 +36,8 @@ import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; -import org.apache.hadoop.hdds.scm.client.HddsClientUtils; 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; @@ -72,9 +70,9 @@ public class StreamBlockInputStream extends BlockExtendedInputStream private StreamingReader streamingReader; private final boolean verifyChecksum; - private final Function refreshFunction; - private final RetryPolicy retryPolicy; - private int retries; + //private final Function refreshFunction; + //private final RetryPolicy retryPolicy; + //private int retries; public StreamBlockInputStream( BlockID blockID, long length, Pipeline pipeline, @@ -88,9 +86,9 @@ public StreamBlockInputStream( tokenRef.set(token); this.xceiverClientFactory = xceiverClientFactory; this.verifyChecksum = config.isChecksumVerify(); - this.refreshFunction = refreshFunction; - this.retryPolicy = HddsClientUtils.createRetryPolicy(config.getMaxReadRetryCount(), - TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); + //this.refreshFunction = refreshFunction; + //this.retryPolicy = HddsClientUtils.createRetryPolicy(config.getMaxReadRetryCount(), + // TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); } @Override From aee40b493c8a4324380e176b9f5cda116ed23c24 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 16:42:57 +0100 Subject: [PATCH 099/114] Fix more findbugs --- .../hadoop/hdds/scm/storage/StreamBlockInputStream.java | 2 +- .../hadoop/hdds/scm/storage/TestStreamBlockInputStream.java | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index c6c42bf803e8..a2fd93819479 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -369,7 +369,7 @@ public ByteBuffer readNext() throws IOException { long pos = getPos(); if (pos < blockOffset) { // This should not happen, and if it does, we have a bug. - throw new IOException("Received data out of order. Position is " + position + " but received data at " + throw new IOException("Received data out of order. Position is " + pos + " but received data at " + blockOffset); } if (pos > readBlock.getOffset()) { diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 512c5fde7309..da7f7c39dd6a 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -178,7 +178,10 @@ public void ensureExceptionThrownForReadAfterClosed() throws IOException { ByteBuffer byteBuffer = ByteBuffer.allocate(10); byte[] byteArray = new byte[10]; assertThrows(IOException.class, () -> blockStream.read()); - assertThrows(IOException.class, () -> blockStream.read(byteArray, 0, 10)); + assertThrows(IOException.class, () -> { + // Findbugs complains about ignored return value without this :( + int r = blockStream.read(byteArray, 0, 10); + }); assertThrows(IOException.class, () -> blockStream.read(byteBuffer)); assertThrows(IOException.class, () -> blockStream.seek(10)); } From 5b99b8db6727790266d287d31d6f5b65c56fd6a7 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 16 Oct 2025 20:09:53 +0100 Subject: [PATCH 100/114] Remove test broken by refactor. It is covered by the integration test anyway --- .../container/keyvalue/KeyValueHandler.java | 9 +- .../keyvalue/TestKeyValueHandler.java | 90 ------------------- 2 files changed, 3 insertions(+), 96 deletions(-) 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 8dc5232214a2..2f4177b2d065 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 @@ -2069,16 +2069,14 @@ public ContainerCommandResponseProto readBlock( ContainerCommandResponseProto responseProto = null; if (!request.hasReadBlock()) { if (LOG.isDebugEnabled()) { - LOG.debug("Malformed Read Block request. trace ID: {}", - request.getTraceID()); + LOG.debug("Malformed Read Block request. trace ID: {}", request.getTraceID()); } return malformedRequest(request); } try { ReadBlockRequestProto readBlock = request.getReadBlock(); - BlockID blockID = BlockID.getFromProtobuf( - readBlock.getBlockID()); + 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); @@ -2132,8 +2130,7 @@ public ContainerCommandResponseProto readBlock( responseProto = ContainerUtils.logAndReturnError(LOG, ex, request); } catch (IOException ioe) { responseProto = ContainerUtils.logAndReturnError(LOG, - new StorageContainerException("Read Block failed", ioe, IO_EXCEPTION), - request); + new StorageContainerException("Read Block failed", ioe, IO_EXCEPTION), request); } return responseProto; } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index 1028f6b5e61a..5646d52bf6e2 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -42,10 +42,8 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.atMostOnce; -import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -55,11 +53,9 @@ import com.google.common.collect.Sets; import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.Path; import java.time.Clock; -import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; @@ -72,29 +68,22 @@ import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumData; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.security.token.TokenVerifier; -import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; import org.apache.hadoop.ozone.container.checksum.ContainerMerkleTreeWriter; import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; -import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerImplTestUtils; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; @@ -105,28 +94,22 @@ import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; -import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil; import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet; import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy; import org.apache.hadoop.ozone.container.common.volume.StorageVolume; import org.apache.hadoop.ozone.container.common.volume.VolumeSet; -import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils; -import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; -import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerScannerConfiguration; import org.apache.hadoop.ozone.container.ozoneimpl.OnDemandContainerScanner; import org.apache.hadoop.util.Time; import org.apache.ozone.test.GenericTestUtils; import org.apache.ozone.test.GenericTestUtils.LogCapturer; -import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -import org.mockito.MockedStatic; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -144,10 +127,7 @@ public class TestKeyValueHandler { private Path dbFile; private static final long DUMMY_CONTAINER_ID = 9999; - private static final long LOCAL_ID = 1; private static final String DUMMY_PATH = "dummy/dir/doesnt/exist"; - private static final long CHUNK_SIZE = 1024 * 1024; // 1MB - private static final long BYTES_PER_CHECKSUM = 256 * 1024; private static final String DATANODE_UUID = UUID.randomUUID().toString(); private static final String CLUSTER_ID = UUID.randomUUID().toString(); @@ -959,74 +939,4 @@ private KeyValueHandler createKeyValueHandler(Path path) throws IOException { return kvHandler; } - @Test - public void testReadBlock() throws IOException { - - StreamObserver streamObserver = mock(StreamObserver.class); - KeyValueContainer container = mock(KeyValueContainer.class); - final KeyValueHandler kvHandler = new KeyValueHandler(new OzoneConfiguration(), - UUID.randomUUID().toString(), mock(ContainerSet.class), mock(VolumeSet.class), mock(ContainerMetrics.class), - mock(IncrementalReportSender.class), mock(ContainerChecksumTreeManager.class)); - final KeyValueHandler keyValueHandler = spy(kvHandler); - DispatcherContext dispatcherContext = mock(DispatcherContext.class); - - List chunkInfoList = new ArrayList<>(); - BlockData blockData = new BlockData(new BlockID(1, 1)); - for (int i = 0; i < 4; i++) { - chunkInfoList.add(ContainerProtos.ChunkInfo - .newBuilder() - .setOffset(0) - .setLen(CHUNK_SIZE) - .setChecksumData( - ChecksumData.newBuilder().setBytesPerChecksum((int) BYTES_PER_CHECKSUM) - .setType(ChecksumType.CRC32).build()) - .setChunkName("chunkName" + i) - .build()); - } - blockData.setChunks(chunkInfoList); - - try (MockedStatic blockUtils = mockStatic(BlockUtils.class)) { - BlockManager blockManager = mock(BlockManager.class); - ChunkManager chunkManager = mock(ChunkManager.class); - when(keyValueHandler.getBlockManager()).thenReturn(blockManager); - when(keyValueHandler.getChunkManager()).thenReturn(chunkManager); - when(blockManager.getBlock(any(), any())).thenReturn(blockData); - ChunkBuffer data = ChunkBuffer.wrap(ByteBuffer.allocate(0)); - when(chunkManager.readChunk(any(), any(), - any(), any())) - .thenReturn(data); - testReadBlock(0, 1, keyValueHandler, dispatcherContext, - streamObserver, container); - testReadBlock(0, CHUNK_SIZE + 1, keyValueHandler, dispatcherContext, - streamObserver, container); - testReadBlock(CHUNK_SIZE / 2, 2 * CHUNK_SIZE, keyValueHandler, dispatcherContext, - streamObserver, container); - } - } - - private static ContainerCommandRequestProto readBlockRequest( - long offset, long length) { - return ContainerCommandRequestProto.newBuilder() - .setCmdType(Type.ReadBlock) - .setReadBlock( - ContainerProtos.ReadBlockRequestProto.newBuilder() - .setBlockID( - ContainerProtos.DatanodeBlockID.newBuilder() - .setContainerID(DUMMY_CONTAINER_ID) - .setLocalID(LOCAL_ID)) - .setOffset(offset)) - .setContainerID(DUMMY_CONTAINER_ID) - .setDatanodeUuid(UUID.randomUUID().toString()) - .build(); - } - - private static void testReadBlock( - long offset, long length, KeyValueHandler keyValueHandler, DispatcherContext dispatcherContext, - StreamObserver streamObserver, KeyValueContainer container) { - int responseCount = (int) (((offset + length - 1) / CHUNK_SIZE) + 1 - (offset / CHUNK_SIZE)); - ContainerCommandRequestProto requestProto = readBlockRequest(offset, length); - keyValueHandler.readBlock(requestProto, container, dispatcherContext, streamObserver); - verify(streamObserver, times(responseCount)).onNext(any()); - clearInvocations(streamObserver); - } } From 3f117b79d385a953cb6cd62372e7e66809229064 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 17 Oct 2025 11:51:18 +0100 Subject: [PATCH 101/114] Fix bug caused by refactor --- .../org/apache/hadoop/hdds/scm/storage/BlockInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index 2cdb4139a540..f34171bd575a 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -230,7 +230,7 @@ private void refreshBlockInfo(IOException cause) throws IOException { if (blockLocationInfo == null) { LOG.warn("No new block location info for block {}", blockID); } else { - pipelineRef.set((blockLocationInfo.getPipeline())); + pipelineRef.set(setPipeline(blockLocationInfo.getPipeline())); LOG.info("New pipeline for block {}: {}", blockID, blockLocationInfo.getPipeline()); From 7234baa1255521d0fa27854e726e30753d7adedc Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 17 Oct 2025 12:00:53 +0100 Subject: [PATCH 102/114] Tidy up formatting --- .../client/io/BlockInputStreamFactoryImpl.java | 8 +++----- .../container/common/impl/HddsDispatcher.java | 16 ++++++---------- .../common/interfaces/ContainerDispatcher.java | 3 +-- .../container/keyvalue/TestKeyValueHandler.java | 1 - 4 files changed, 10 insertions(+), 18 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 9249abaf42a6..2765e2a00a8f 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -90,9 +90,8 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, blockInfo, xceiverFactory, refreshFunction, ecBlockStreamFactory, config); } else if (config.isStreamReadBlock() && allDataNodesSupportStreamBlock(pipeline)) { - return new StreamBlockInputStream( - blockInfo.getBlockID(), blockInfo.getLength(), - pipeline, token, xceiverFactory, refreshFunction, config); + return new StreamBlockInputStream(blockInfo.getBlockID(), blockInfo.getLength(), pipeline, token, xceiverFactory, + refreshFunction, config); } else { return new BlockInputStream(blockInfo, pipeline, token, xceiverFactory, refreshFunction, @@ -104,8 +103,7 @@ private boolean allDataNodesSupportStreamBlock(Pipeline pipeline) { // return true only if all DataNodes in the pipeline are on a version // that supports for reading a block by streaming chunks.. for (DatanodeDetails dn : pipeline.getNodes()) { - if (dn.getCurrentVersion() < - STREAM_BLOCK_SUPPORT.toProtoValue()) { + if (dn.getCurrentVersion() < STREAM_BLOCK_SUPPORT.toProtoValue()) { return false; } } 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 e6be4a490b43..6dba6abf9d09 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 @@ -818,8 +818,8 @@ public StateMachine.DataChannel getStreamDataChannel( @Override public void streamDataReadOnly(ContainerCommandRequestProto msg, - StreamObserver streamObserver, - DispatcherContext dispatcherContext) { + StreamObserver streamObserver, + DispatcherContext dispatcherContext) { Type cmdType = msg.getCmdType(); String traceID = msg.getTraceID(); Span span = TracingUtil.importAndCreateSpan(cmdType.toString(), traceID); @@ -829,8 +829,7 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, try (UncheckedAutoCloseable ignored = protocolMetrics.measure(cmdType)) { Preconditions.checkNotNull(msg); if (LOG.isTraceEnabled()) { - LOG.trace("Command {}, trace ID: {}.", msg.getCmdType(), - traceID); + LOG.trace("Command {}, trace ID: {}.", msg.getCmdType(), traceID); } PerformanceStringBuilder perf = new PerformanceStringBuilder(); @@ -849,20 +848,17 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg, ContainerProtos.Result.CONTAINER_MISSING); } if (container == null) { - throw new StorageContainerException( - "ContainerID " + containerID + " does not exist", + throw new StorageContainerException("ContainerID " + containerID + " does not exist", ContainerProtos.Result.CONTAINER_NOT_FOUND); } ContainerType containerType = getContainerType(container); Handler handler = getHandler(containerType); if (handler == null) { - throw new StorageContainerException("Invalid " + - "ContainerType " + containerType, + throw new StorageContainerException("Invalid " + "ContainerType " + containerType, ContainerProtos.Result.CONTAINER_INTERNAL_ERROR); } perf.appendPreOpLatencyMs(Time.monotonicNow() - startTime); - responseProto = handler.readBlock( - msg, container, dispatcherContext, streamObserver); + responseProto = handler.readBlock(msg, container, dispatcherContext, streamObserver); long oPLatencyMS = Time.monotonicNow() - startTime; metrics.incContainerOpsLatencies(cmdType, oPLatencyMS); if (responseProto == null) { 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 d6c334550080..8a4a675187df 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 @@ -98,7 +98,6 @@ default void streamDataReadOnly( ContainerCommandRequestProto msg, StreamObserver streamObserver, DispatcherContext dispatcherContext) { - throw new UnsupportedOperationException( - "streamDataReadOnly not supported."); + throw new UnsupportedOperationException("streamDataReadOnly not supported."); } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index 5646d52bf6e2..6afee1c5d77f 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -938,5 +938,4 @@ private KeyValueHandler createKeyValueHandler(Path path) throws IOException { return kvHandler; } - } From af70910d67f2a224a4fcd634ecdde1778e6f7178 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 17 Oct 2025 18:23:57 +0100 Subject: [PATCH 103/114] Reuse datanode selection and deadline code in XCeiverGRPC --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 93 ++++++++++--------- .../rpc/read/TestStreamBlockInputStream.java | 2 +- 2 files changed, 51 insertions(+), 44 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 72926674e817..09ae5899afad 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -385,25 +385,9 @@ private XceiverClientReply sendCommandWithTraceIDAndRetry( }); } - private XceiverClientReply sendCommandWithRetry( - ContainerCommandRequestProto request, List validators) - throws IOException { - ContainerCommandResponseProto responseProto = null; - IOException ioException = null; - - // In case of an exception or an error, we will try to read from the - // datanodes in the pipeline in a round-robin fashion. - XceiverClientReply reply = new XceiverClientReply(null); + private List sortDatanodes(ContainerCommandRequestProto request) throws IOException { List datanodeList = null; - - DatanodeBlockID blockID = null; - if (request.getCmdType() == ContainerProtos.Type.GetBlock) { - blockID = request.getGetBlock().getBlockID(); - } else if (request.getCmdType() == ContainerProtos.Type.ReadChunk) { - blockID = request.getReadChunk().getBlockID(); - } else if (request.getCmdType() == ContainerProtos.Type.GetSmallFile) { - blockID = request.getGetSmallFile().getBlock().getBlockID(); - } + DatanodeBlockID blockID = getRequestBlockID(request); if (blockID != null) { if (request.getCmdType() != ContainerProtos.Type.ReadChunk) { @@ -441,6 +425,33 @@ private XceiverClientReply sendCommandWithRetry( if (!allInService) { datanodeList = sortDatanodeByOperationalState(datanodeList); } + return datanodeList; + } + + private static DatanodeBlockID getRequestBlockID(ContainerCommandRequestProto request) { + DatanodeBlockID blockID = null; + if (request.getCmdType() == ContainerProtos.Type.GetBlock) { + blockID = request.getGetBlock().getBlockID(); + } else if (request.getCmdType() == ContainerProtos.Type.ReadChunk) { + blockID = request.getReadChunk().getBlockID(); + } else if (request.getCmdType() == ContainerProtos.Type.GetSmallFile) { + blockID = request.getGetSmallFile().getBlock().getBlockID(); + } else if (request.getCmdType() == ContainerProtos.Type.ReadBlock) { + blockID = request.getReadBlock().getBlockID(); + } + return blockID; + } + + private XceiverClientReply sendCommandWithRetry( + ContainerCommandRequestProto request, List validators) + throws IOException { + ContainerCommandResponseProto responseProto = null; + IOException ioException = null; + + // In case of an exception or an error, we will try to read from the + // datanodes in the pipeline in a round-robin fashion. + XceiverClientReply reply = new XceiverClientReply(null); + List datanodeList = sortDatanodes(request); for (DatanodeDetails dn : datanodeList) { try { @@ -505,35 +516,31 @@ private XceiverClientReply sendCommandWithRetry( } } - // TODO This is currently not doing retries or timeouts @Override public ClientCallStreamObserver streamRead(ContainerCommandRequestProto request, StreamObserver streamObserver) throws IOException { - - DatanodeDetails dn = topologyAwareRead ? - pipeline.getClosestNode() : pipeline.getFirstNode(); - - XceiverClientProtocolServiceStub stub = asyncStubs.get(dn.getID()); - if (stub == null) { - connectToDatanode(dn); - stub = asyncStubs.get(dn.getID()); - } - - if (stub == null) { - throw new IOException("Failed to get gRPC stub for DataNode: " + dn); - } - - LOG.debug("Starting streaming read to DataNode {}", dn); - - try { - StreamObserver requestObserver = stub.send(streamObserver); - requestObserver.onNext(request); - requestObserver.onCompleted(); - return (ClientCallStreamObserver) requestObserver; - } catch (Exception e) { - LOG.error("Failed to start streaming read to DataNode {}", dn, e); - throw new IOException("Streaming read failed", e); + List datanodeList = sortDatanodes(request); + for (DatanodeDetails dn : datanodeList) { + try { + checkOpen(dn); + XceiverClientProtocolServiceStub stub = asyncStubs.get(dn.getID()); + if (stub == null) { + throw new IOException("Failed to get gRPC stub for DataNode: " + dn); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Executing command {} on datanode {}", processForDebug(request), dn); + } + StreamObserver requestObserver = stub + .withDeadlineAfter(timeout, TimeUnit.SECONDS) + .send(streamObserver); + requestObserver.onNext(request); + requestObserver.onCompleted(); + return (ClientCallStreamObserver) requestObserver; + } catch (Exception e) { + LOG.error("Failed to start streaming read to DataNode {}", dn, e); + } } + throw new IOException("Failed to start streaming read to any available DataNodes"); } private static List sortDatanodeByOperationalState( diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index e544f313cbf9..745e2c282059 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -43,7 +43,7 @@ public class TestStreamBlockInputStream extends TestInputStreamBase { * for each test. */ - private int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); + private final int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); private byte[] inputData; private TestBucket bucket; From 4b3706473258b2bb0e07e2b43e5759206c2308a2 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 17 Oct 2025 20:28:31 +0100 Subject: [PATCH 104/114] Fix findbugs --- .../rpc/read/TestStreamBlockInputStream.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 745e2c282059..884e8bbb9e01 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -43,7 +43,7 @@ public class TestStreamBlockInputStream extends TestInputStreamBase { * for each test. */ - private final int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); + private static final int DATA_LENGTH = (2 * BLOCK_SIZE) + (CHUNK_SIZE); private byte[] inputData; private TestBucket bucket; @@ -60,7 +60,7 @@ void testAll() throws Exception { String keyName = getNewKeyName(); try (OzoneClient client = OzoneClientFactory.getRpcClient(copy)) { bucket = TestBucket.newBuilder(client).build(); - inputData = bucket.writeRandomBytes(keyName, dataLength); + inputData = bucket.writeRandomBytes(keyName, DATA_LENGTH); testReadKeyFully(keyName); testSeek(keyName); testReadEmptyBlock(); @@ -70,7 +70,7 @@ void testAll() throws Exception { copy.setFromObject(clientConfig); try (OzoneClient client = OzoneClientFactory.getRpcClient(copy)) { bucket = TestBucket.newBuilder(client).build(); - inputData = bucket.writeRandomBytes(keyName, dataLength); + inputData = bucket.writeRandomBytes(keyName, DATA_LENGTH); testReadKeyFully(keyName); testSeek(keyName); } @@ -84,17 +84,17 @@ void testAll() throws Exception { private void testReadKeyFully(String key) throws Exception { // Read the data fully into a large enough byte array try (KeyInputStream keyInputStream = bucket.getKeyInputStream(key)) { - byte[] readData = new byte[dataLength]; - int totalRead = keyInputStream.read(readData, 0, dataLength); - assertEquals(dataLength, totalRead); - for (int i = 0; i < dataLength; i++) { + byte[] readData = new byte[DATA_LENGTH]; + int totalRead = keyInputStream.read(readData, 0, DATA_LENGTH); + assertEquals(DATA_LENGTH, totalRead); + for (int i = 0; i < DATA_LENGTH; i++) { assertEquals(inputData[i], readData[i], "Read data is not same as written data at index " + i); } } // Read the data 1 byte at a time try (KeyInputStream keyInputStream = bucket.getKeyInputStream(key)) { - for (int i = 0; i < dataLength; i++) { + for (int i = 0; i < DATA_LENGTH; i++) { int b = keyInputStream.read(); assertEquals(inputData[i], (byte) b, "Read data is not same as written data at index " + i); @@ -102,11 +102,11 @@ private void testReadKeyFully(String key) throws Exception { } // Read the data into a large enough ByteBuffer try (KeyInputStream keyInputStream = bucket.getKeyInputStream(key)) { - ByteBuffer readBuf = ByteBuffer.allocate(dataLength); + ByteBuffer readBuf = ByteBuffer.allocate(DATA_LENGTH); int totalRead = keyInputStream.read(readBuf); - assertEquals(dataLength, totalRead); + assertEquals(DATA_LENGTH, totalRead); readBuf.flip(); - for (int i = 0; i < dataLength; i++) { + for (int i = 0; i < DATA_LENGTH; i++) { assertEquals(inputData[i], readBuf.get(), "Read data is not same as written data at index " + i); } @@ -117,7 +117,7 @@ private void testSeek(String key) throws IOException { java.util.Random random = new java.util.Random(); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(key)) { for (int i = 0; i < 100; i++) { - int position = random.nextInt(dataLength); + int position = random.nextInt(DATA_LENGTH); keyInputStream.seek(position); int b = keyInputStream.read(); assertEquals(inputData[position], (byte) b, "Read data is not same as written data at index " + position); From 4d7447957d22eccd8b80bc8e96bfbabd35905498 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Mon, 20 Oct 2025 18:13:27 +0100 Subject: [PATCH 105/114] Retry initializing the reader if all DNs fail via pipeline refresh --- .../scm/storage/BlockExtendedInputStream.java | 68 +++++++++++ .../hdds/scm/storage/BlockInputStream.java | 62 +--------- .../scm/storage/StreamBlockInputStream.java | 106 +++++------------- .../storage/TestStreamBlockInputStream.java | 59 ++++++++-- .../scm/storage/ContainerProtocolCalls.java | 4 - 5 files changed, 152 insertions(+), 147 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java index 02a0ca65c665..617b3f6bebde 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java @@ -18,9 +18,18 @@ package org.apache.hadoop.hdds.scm.storage; import java.io.IOException; +import java.time.Instant; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.client.HddsClientUtils; 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.security.token.Token; /** * Abstract class used as an interface for input streams related to Ozone @@ -29,6 +38,8 @@ public abstract class BlockExtendedInputStream extends ExtendedInputStream implements PartInputStream { + private static final org.slf4j.Logger LOG = org.slf4j.LoggerFactory.getLogger(BlockExtendedInputStream.class); + public abstract BlockID getBlockID(); @Override @@ -59,4 +70,61 @@ protected Pipeline setPipeline(Pipeline pipeline) throws IOException { return okForRead ? pipeline : pipeline.copyForRead(); } + protected boolean shouldRetryRead(IOException cause, RetryPolicy retryPolicy, int retries) throws IOException { + RetryPolicy.RetryAction retryAction; + try { + retryAction = retryPolicy.shouldRetry(cause, retries, 0, true); + } catch (IOException e) { + throw e; + } catch (Exception e) { + throw new IOException(e); + } + if (retryAction.action == RetryPolicy.RetryAction.RetryDecision.RETRY) { + if (retryAction.delayMillis > 0) { + try { + LOG.debug("Retry read after {}ms", retryAction.delayMillis); + Thread.sleep(retryAction.delayMillis); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + String msg = "Interrupted: action=" + retryAction.action + ", retry policy=" + retryPolicy; + throw new IOException(msg, e); + } + } + return true; + } + return false; + } + + protected RetryPolicy getReadRetryPolicy(OzoneClientConfig config) { + return HddsClientUtils.createRetryPolicy(config.getMaxReadRetryCount(), + TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); + } + + protected void refreshBlockInfo(IOException cause, BlockID blockID, AtomicReference pipelineRef, + AtomicReference> tokenRef, Function refreshFunction) + throws IOException { + LOG.info("Attempting to update pipeline and block token for block {} from pipeline {}: {}", + blockID, pipelineRef.get().getId(), cause.getMessage()); + if (refreshFunction != null) { + LOG.debug("Re-fetching pipeline and block token for block {}", blockID); + BlockLocationInfo blockLocationInfo = refreshFunction.apply(blockID); + if (blockLocationInfo == null) { + LOG.warn("No new block location info for block {}", blockID); + } else { + pipelineRef.set(setPipeline(blockLocationInfo.getPipeline())); + LOG.info("New pipeline for block {}: {}", blockID, blockLocationInfo.getPipeline()); + + tokenRef.set(blockLocationInfo.getToken()); + if (blockLocationInfo.getToken() != null) { + OzoneBlockTokenIdentifier tokenId = new OzoneBlockTokenIdentifier(); + tokenId.readFromByteArray(tokenRef.get().getIdentifier()); + LOG.info("A new token is added for block {}. Expiry: {}", blockID, + Instant.ofEpochMilli(tokenId.getExpiryDate())); + } + } + } else { + throw cause; + } + } + } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index f34171bd575a..fe2717e171df 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -22,11 +22,9 @@ import java.io.EOFException; import java.io.IOException; import java.io.InputStream; -import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import org.apache.hadoop.hdds.client.BlockID; @@ -39,7 +37,6 @@ import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator; -import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.security.exception.SCMSecurityException; @@ -127,9 +124,7 @@ public BlockInputStream( this.verifyChecksum = config.isChecksumVerify(); this.xceiverClientFactory = xceiverClientFactory; this.refreshFunction = refreshFunction; - this.retryPolicy = - HddsClientUtils.createRetryPolicy(config.getMaxReadRetryCount(), - TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); + this.retryPolicy = getReadRetryPolicy(config); } // only for unit tests @@ -181,7 +176,7 @@ public synchronized void initialize() throws IOException { } catchEx = ex; } - } while (shouldRetryRead(catchEx)); + } while (shouldRetryRead(catchEx, retryPolicy, ++retries)); if (chunks == null) { throw catchEx; @@ -222,29 +217,7 @@ private boolean isConnectivityIssue(IOException ex) { } private void refreshBlockInfo(IOException cause) throws IOException { - LOG.info("Attempting to update pipeline and block token for block {} from pipeline {}: {}", - blockID, pipelineRef.get().getId(), cause.getMessage()); - if (refreshFunction != null) { - LOG.debug("Re-fetching pipeline and block token for block {}", blockID); - BlockLocationInfo blockLocationInfo = refreshFunction.apply(blockID); - if (blockLocationInfo == null) { - LOG.warn("No new block location info for block {}", blockID); - } else { - pipelineRef.set(setPipeline(blockLocationInfo.getPipeline())); - LOG.info("New pipeline for block {}: {}", blockID, - blockLocationInfo.getPipeline()); - - tokenRef.set(blockLocationInfo.getToken()); - if (blockLocationInfo.getToken() != null) { - OzoneBlockTokenIdentifier tokenId = new OzoneBlockTokenIdentifier(); - tokenId.readFromByteArray(tokenRef.get().getIdentifier()); - LOG.info("A new token is added for block {}. Expiry: {}", - blockID, Instant.ofEpochMilli(tokenId.getExpiryDate())); - } - } - } else { - throw cause; - } + refreshBlockInfo(cause, blockID, pipelineRef, tokenRef, refreshFunction); } /** @@ -361,14 +334,14 @@ protected synchronized int readWithStrategy(ByteReaderStrategy strategy) } catch (SCMSecurityException ex) { throw ex; } catch (StorageContainerException e) { - if (shouldRetryRead(e)) { + if (shouldRetryRead(e, retryPolicy, ++retries)) { handleReadError(e); continue; } else { throw e; } } catch (IOException ex) { - if (shouldRetryRead(ex)) { + if (shouldRetryRead(ex, retryPolicy, ++retries)) { if (isConnectivityIssue(ex)) { handleReadError(ex); } else { @@ -552,31 +525,6 @@ private synchronized void storePosition() { blockPosition = getPos(); } - private boolean shouldRetryRead(IOException cause) throws IOException { - RetryPolicy.RetryAction retryAction; - try { - retryAction = retryPolicy.shouldRetry(cause, ++retries, 0, true); - } catch (IOException e) { - throw e; - } catch (Exception e) { - throw new IOException(e); - } - if (retryAction.action == RetryPolicy.RetryAction.RetryDecision.RETRY) { - if (retryAction.delayMillis > 0) { - try { - LOG.debug("Retry read after {}ms", retryAction.delayMillis); - Thread.sleep(retryAction.delayMillis); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - String msg = "Interrupted: action=" + retryAction.action + ", retry policy=" + retryPolicy; - throw new IOException(msg, e); - } - } - return true; - } - return false; - } - private void handleReadError(IOException cause) throws IOException { releaseClient(); final List inputStreams = this.chunkStreams; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index a2fd93819479..cd36942912a7 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientSpi; 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; @@ -70,9 +71,9 @@ public class StreamBlockInputStream extends BlockExtendedInputStream private StreamingReader streamingReader; private final boolean verifyChecksum; - //private final Function refreshFunction; - //private final RetryPolicy retryPolicy; - //private int retries; + private final Function refreshFunction; + private final RetryPolicy retryPolicy; + private int retries = 0; public StreamBlockInputStream( BlockID blockID, long length, Pipeline pipeline, @@ -86,9 +87,8 @@ public StreamBlockInputStream( tokenRef.set(token); this.xceiverClientFactory = xceiverClientFactory; this.verifyChecksum = config.isChecksumVerify(); - //this.refreshFunction = refreshFunction; - //this.retryPolicy = HddsClientUtils.createRetryPolicy(config.getMaxReadRetryCount(), - // TimeUnit.SECONDS.toMillis(config.getReadRetryInterval())); + this.retryPolicy = getReadRetryPolicy(config); + this.refreshFunction = refreshFunction; } @Override @@ -218,13 +218,26 @@ private void initialize() throws IOException { if (initialized) { return; } - acquireClient(); - streamingReader = new StreamingReader(); - ClientCallStreamObserver requestObserver = - ContainerProtocolCalls.readBlock(xceiverClient, position, blockID, tokenRef.get(), - pipelineRef.get().getReplicaIndexes(), streamingReader); - streamingReader.setRequestObserver(requestObserver); - initialized = true; + while (true) { + try { + acquireClient(); + streamingReader = new StreamingReader(); + ClientCallStreamObserver requestObserver = + ContainerProtocolCalls.readBlock(xceiverClient, position, blockID, tokenRef.get(), + pipelineRef.get().getReplicaIndexes(), streamingReader); + streamingReader.setRequestObserver(requestObserver); + initialized = true; + return; + } catch (IOException ioe) { + if (shouldRetryRead(ioe, retryPolicy, retries++)) { + releaseClient(); + refreshBlockInfo(ioe); + LOG.warn("Retrying read for block {} due to {}", blockID, ioe.getMessage()); + } else { + throw ioe; + } + } + } } private int fillBuffer() throws IOException { @@ -249,75 +262,10 @@ public synchronized void close() throws IOException { xceiverClientFactory = null; } - /* - Commenting out for now as we probably need these for retries which are yet to be implemented. - - private boolean shouldRetryRead(IOException cause) throws IOException { - RetryPolicy.RetryAction retryAction; - try { - retryAction = retryPolicy.shouldRetry(cause, ++retries, 0, true); - } catch (IOException e) { - throw e; - } catch (Exception e) { - throw new IOException(e); - } - return retryAction.action == RetryPolicy.RetryAction.RetryDecision.RETRY; - } - private void refreshBlockInfo(IOException cause) throws IOException { - LOG.info("Attempting to update pipeline and block token for block {} from pipeline {}: {}", - blockID, pipelineRef.get().getId(), cause.getMessage()); - if (refreshFunction != null) { - LOG.debug("Re-fetching pipeline and block token for block {}", blockID); - BlockLocationInfo blockLocationInfo = refreshFunction.apply(blockID); - if (blockLocationInfo == null) { - LOG.warn("No new block location info for block {}", blockID); - } else { - pipelineRef.set(getPipeline(blockLocationInfo.getPipeline())); - LOG.info("New pipeline for block {}: {}", blockID, - blockLocationInfo.getPipeline()); - - tokenRef.set(blockLocationInfo.getToken()); - if (blockLocationInfo.getToken() != null) { - OzoneBlockTokenIdentifier tokenId = new OzoneBlockTokenIdentifier(); - tokenId.readFromByteArray(tokenRef.get().getIdentifier()); - LOG.info("A new token is added for block {}. Expiry: {}", - blockID, Instant.ofEpochMilli(tokenId.getExpiryDate())); - } - } - } else { - throw cause; - } + refreshBlockInfo(cause, blockID, pipelineRef, tokenRef, refreshFunction); } - private boolean isConnectivityIssue(IOException ex) { - return Status.fromThrowable(ex).getCode() == Status.UNAVAILABLE.getCode(); - } - - private void handleStorageContainerException(StorageContainerException e) throws IOException { - if (shouldRetryRead(e)) { - releaseClient(); - refreshBlockInfo(e); - } else { - throw e; - } - } - - private void handleIOException(IOException ioe) throws IOException { - if (shouldRetryRead(ioe)) { - if (isConnectivityIssue(ioe)) { - releaseClient(); - refreshBlockInfo(ioe); - } else { - releaseClient(); - } - } else { - throw ioe; - } - } - - */ - /** * Implementation of a StreamObserver used to received and buffer streaming GRPC reads. */ diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index da7f7c39dd6a..c156d1ca6ba5 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -22,17 +22,23 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.IOException; import java.nio.ByteBuffer; import java.security.SecureRandom; +import java.util.Collections; +import java.util.Set; +import java.util.UUID; import java.util.function.Function; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientGrpc; @@ -43,6 +49,7 @@ 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.util.Time; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; @@ -65,12 +72,25 @@ public class TestStreamBlockInputStream { private ChecksumData checksumData; private byte[] data; private ClientCallStreamObserver requestObserver; + private Function refreshFunction; @BeforeEach public void setup() throws Exception { Token token = mock(Token.class); when(token.encodeToUrlString()).thenReturn("url"); + + Set modes = + Collections.singleton(HddsProtos.BlockTokenSecretProto.AccessModeProto.READ); + OzoneBlockTokenIdentifier tokenIdentifier = new OzoneBlockTokenIdentifier("owner", new BlockID(1, 1), + modes, Time.monotonicNow() + 10000, 10); + tokenIdentifier.setSecretKeyId(UUID.randomUUID()); + when(token.getIdentifier()).thenReturn(tokenIdentifier.getBytes()); Pipeline pipeline = MockPipeline.createSingleNodePipeline(); + + BlockLocationInfo blockLocationInfo = mock(BlockLocationInfo.class); + when(blockLocationInfo.getPipeline()).thenReturn(pipeline); + when(blockLocationInfo.getToken()).thenReturn(token); + xceiverClient = mock(XceiverClientGrpc.class); when(xceiverClient.getPipeline()).thenReturn(pipeline); xceiverClientFactory = mock(XceiverClientFactory.class); @@ -80,7 +100,9 @@ public void setup() throws Exception { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setStreamReadBlock(true); - Function refreshFunction = mock(Function.class); + clientConfig.setMaxReadRetryCount(1); + refreshFunction = mock(Function.class); + when(refreshFunction.apply(any())).thenReturn(blockLocationInfo); BlockID blockID = new BlockID(new ContainerBlockID(1, 1)); checksum = new Checksum(ChecksumType.CRC32, BYTES_PER_CHECKSUM); createDataAndChecksum(); @@ -105,9 +127,9 @@ public void testCloseStreamReleasesResources() throws IOException { assertEquals(data[0], blockStream.read()); blockStream.close(); // Verify that cancel() was called on the requestObserver mock - org.mockito.Mockito.verify(requestObserver).cancel(any(), any()); + verify(requestObserver).cancel(any(), any()); // Verify that release() was called on the xceiverClient mock - org.mockito.Mockito.verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); + verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); } @Test @@ -117,9 +139,9 @@ public void testUnbufferReleasesResourcesAndResumesFromLastPosition() throws IOE assertEquals(1, blockStream.getPos()); blockStream.unbuffer(); // Verify that cancel() was called on the requestObserver mock - org.mockito.Mockito.verify(requestObserver).cancel(any(), any()); + verify(requestObserver).cancel(any(), any()); // Verify that release() was called on the xceiverClient mock - org.mockito.Mockito.verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); + verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); // The next read should "rebuffer" and continue from the last position assertEquals(data[1], blockStream.read()); assertEquals(2, blockStream.getPos()); @@ -132,9 +154,9 @@ public void testSeekReleasesTheStreamAndStartsFromNewPosition() throws IOExcepti blockStream.seek(100); assertEquals(100, blockStream.getPos()); // Verify that cancel() was called on the requestObserver mock - org.mockito.Mockito.verify(requestObserver).cancel(any(), any()); + verify(requestObserver).cancel(any(), any()); // The xceiverClient should not be released - org.mockito.Mockito.verify(xceiverClientFactory, never()) + verify(xceiverClientFactory, never()) .releaseClientForReadData(xceiverClient, false); assertEquals(data[100], blockStream.read()); @@ -186,6 +208,29 @@ public void ensureExceptionThrownForReadAfterClosed() throws IOException { assertThrows(IOException.class, () -> blockStream.seek(10)); } + @Test + public void testRefreshFunctionCalledForAllDNsBadOnInitialize() throws IOException { + // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline + when(xceiverClient.streamRead(any(), any())) + .thenThrow(new IOException("Test induced exception")) + .thenAnswer((InvocationOnMock invocation) -> { + StreamObserver streamObserver = invocation.getArgument(1); + streamObserver.onNext(createChunkResponse()); + streamObserver.onCompleted(); + return requestObserver; + }); + blockStream.read(); + verify(refreshFunction, times(1)).apply(any()); + } + + @Test + public void testExceptionThrownAfterRetriesExhaused() throws IOException { + // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline + when(xceiverClient.streamRead(any(), any())).thenThrow(new IOException("Test induced exception")); + assertThrows(IOException.class, () -> blockStream.read()); + verify(refreshFunction, times(1)).apply(any()); + } + private void createDataAndChecksum() throws OzoneChecksumException { data = new byte[BLOCK_SIZE]; new SecureRandom().nextBytes(data); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index ce3e6062400b..f8be5410576f 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -933,10 +933,6 @@ public static ClientCallStreamObserver readBlock( return readBlock(xceiverClient, blockID, builder, readBlockRequest, xceiverClient.getPipeline().getFirstNode(), replicaIndexes, streamObserver); - // tryEachDatanode(xceiverClient.getPipeline(), - // d -> readBlock(xceiverClient, - // validators, blockID, builder, readBlockRequest, d, replicaIndexes, streamObserver), - // d -> toErrorMessage(blockID, d)); } private static ClientCallStreamObserver readBlock(XceiverClientSpi xceiverClient, From 6bd9885864ce87c3d9834996d5b166601f6f8ffe Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Tue, 21 Oct 2025 16:35:57 +0100 Subject: [PATCH 106/114] Fix broken test caused by refactor --- .../apache/hadoop/hdds/scm/storage/TestBlockInputStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java index 2e9a84cad429..1b5e7667e845 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java @@ -267,8 +267,8 @@ public void testSeekAndRead() throws Exception { @Test public void testRefreshPipelineFunction() throws Exception { - LogCapturer logCapturer = LogCapturer.captureLogs(BlockInputStream.class); - GenericTestUtils.setLogLevel(BlockInputStream.class, Level.DEBUG); + LogCapturer logCapturer = LogCapturer.captureLogs(BlockExtendedInputStream.class); + GenericTestUtils.setLogLevel(BlockExtendedInputStream.class, Level.DEBUG); BlockID blockID = new BlockID(new ContainerBlockID(1, 1)); AtomicBoolean isRefreshed = new AtomicBoolean(); createChunkList(5); From b12e2353625c81a5b5e41d1bafa3fa1a999c3de8 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Tue, 21 Oct 2025 17:12:40 +0100 Subject: [PATCH 107/114] Refresh pipeline only for specific exception catagories --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 10 +++- .../scm/storage/BlockExtendedInputStream.java | 8 ++++ .../hdds/scm/storage/BlockInputStream.java | 8 ---- .../scm/storage/StreamBlockInputStream.java | 23 ++++++--- .../storage/TestStreamBlockInputStream.java | 47 +++++++++++++++++-- 5 files changed, 74 insertions(+), 22 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 09ae5899afad..c87d6f3a486b 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -520,6 +520,7 @@ private XceiverClientReply sendCommandWithRetry( public ClientCallStreamObserver streamRead(ContainerCommandRequestProto request, StreamObserver streamObserver) throws IOException { List datanodeList = sortDatanodes(request); + IOException lastException = null; for (DatanodeDetails dn : datanodeList) { try { checkOpen(dn); @@ -536,11 +537,16 @@ public ClientCallStreamObserver streamRead(Contain requestObserver.onNext(request); requestObserver.onCompleted(); return (ClientCallStreamObserver) requestObserver; - } catch (Exception e) { + } catch (IOException e) { LOG.error("Failed to start streaming read to DataNode {}", dn, e); + lastException = e; } } - throw new IOException("Failed to start streaming read to any available DataNodes"); + if (lastException != null) { + throw lastException; + } else { + throw new IOException("Failed to start streaming read to any available DataNodes"); + } } private static List sortDatanodeByOperationalState( diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java index 617b3f6bebde..dc47a9edacaf 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.security.token.Token; +import org.apache.ratis.thirdparty.io.grpc.Status; /** * Abstract class used as an interface for input streams related to Ozone @@ -127,4 +128,11 @@ protected void refreshBlockInfo(IOException cause, BlockID blockID, AtomicRefere } } + /** + * Check if this exception is because datanodes are not reachable. + */ + protected boolean isConnectivityIssue(IOException ex) { + return Status.fromThrowable(ex).getCode() == Status.UNAVAILABLE.getCode(); + } + } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index fe2717e171df..6f6b513422f7 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.security.token.Token; -import org.apache.ratis.thirdparty.io.grpc.Status; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -209,13 +208,6 @@ public synchronized void initialize() throws IOException { } } - /** - * Check if this exception is because datanodes are not reachable. - */ - private boolean isConnectivityIssue(IOException ex) { - return Status.fromThrowable(ex).getCode() == Status.UNAVAILABLE.getCode(); - } - private void refreshBlockInfo(IOException cause) throws IOException { refreshBlockInfo(cause, blockID, pipelineRef, tokenRef, refreshFunction); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index cd36942912a7..d06b223c56e4 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.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; @@ -229,14 +230,22 @@ private void initialize() throws IOException { initialized = true; return; } catch (IOException ioe) { - if (shouldRetryRead(ioe, retryPolicy, retries++)) { - releaseClient(); - refreshBlockInfo(ioe); - LOG.warn("Retrying read for block {} due to {}", blockID, ioe.getMessage()); - } else { - throw 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; } } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index c156d1ca6ba5..9c5a33f75361 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -17,6 +17,7 @@ package org.apache.hadoop.hdds.scm.storage; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; @@ -32,7 +33,9 @@ import java.util.Collections; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ExecutionException; import java.util.function.Function; +import java.util.stream.Stream; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -42,8 +45,10 @@ import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientGrpc; +import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.ozone.common.Checksum; import org.apache.hadoop.ozone.common.ChecksumData; @@ -51,11 +56,16 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Time; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.Status; +import org.apache.ratis.thirdparty.io.grpc.StatusException; import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.invocation.InvocationOnMock; /** @@ -208,11 +218,12 @@ public void ensureExceptionThrownForReadAfterClosed() throws IOException { assertThrows(IOException.class, () -> blockStream.seek(10)); } - @Test - public void testRefreshFunctionCalledForAllDNsBadOnInitialize() throws IOException { + @ParameterizedTest + @MethodSource("exceptionsTriggeringRefresh") + public void testRefreshFunctionCalledForAllDNsBadOnInitialize(IOException thrown) throws IOException { // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline when(xceiverClient.streamRead(any(), any())) - .thenThrow(new IOException("Test induced exception")) + .thenThrow(thrown) .thenAnswer((InvocationOnMock invocation) -> { StreamObserver streamObserver = invocation.getArgument(1); streamObserver.onNext(createChunkResponse()); @@ -223,10 +234,20 @@ public void testRefreshFunctionCalledForAllDNsBadOnInitialize() throws IOExcepti verify(refreshFunction, times(1)).apply(any()); } + @ParameterizedTest + @MethodSource("exceptionsNotTriggeringRefresh") + public void testRefreshNotCalledForAllDNsBadOnInitialize(IOException thrown) throws IOException { + // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline + when(xceiverClient.streamRead(any(), any())) + .thenThrow(thrown); + assertThrows(IOException.class, () -> blockStream.read()); + verify(refreshFunction, times(0)).apply(any()); + } + @Test - public void testExceptionThrownAfterRetriesExhaused() throws IOException { + public void testExceptionThrownAfterRetriesExhausted() throws IOException { // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline - when(xceiverClient.streamRead(any(), any())).thenThrow(new IOException("Test induced exception")); + when(xceiverClient.streamRead(any(), any())).thenThrow(new StorageContainerException(CONTAINER_NOT_FOUND)); assertThrows(IOException.class, () -> blockStream.read()); verify(refreshFunction, times(1)).apply(any()); } @@ -260,4 +281,20 @@ private ContainerProtos.ContainerCommandResponseProto createChunkResponse() { .build(); } + private static Stream exceptionsTriggeringRefresh() { + return Stream.of( + Arguments.of(new StorageContainerException(CONTAINER_NOT_FOUND)), + Arguments.of(new IOException(new ExecutionException( + new StatusException(Status.UNAVAILABLE)))) + ); + } + + private static Stream exceptionsNotTriggeringRefresh() { + return Stream.of( + Arguments.of(new SCMSecurityException("Security problem")), + Arguments.of(new OzoneChecksumException("checksum missing")), + Arguments.of(new IOException("Some random exception.")) + ); + } + } From 6e4463ed8e0f00f67ac9f658e0ba1bb0b8902b26 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 23 Oct 2025 18:32:10 +0100 Subject: [PATCH 108/114] Take and release semaphore when starting / completing streaming reads --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 32 +++++++++++-- .../scm/storage/StreamBlockInputStream.java | 31 +++++++++---- .../storage/TestStreamBlockInputStream.java | 36 +++++++++------ .../hdds/scm/StreamingReadResponse.java | 46 +++++++++++++++++++ .../hadoop/hdds/scm/XceiverClientSpi.java | 9 ++-- .../scm/storage/ContainerProtocolCalls.java | 10 ++-- 6 files changed, 130 insertions(+), 34 deletions(-) create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReadResponse.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index c87d6f3a486b..ff8a8d87f4b2 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -516,14 +516,29 @@ private XceiverClientReply sendCommandWithRetry( } } + /** + * Starts a streaming read operation, intended to read entire blocks from the datanodes. This method expects a + * {@link StreamObserver} to be passed in, which will be used to receive the streamed data from the datanode. + * Upon successfully starting the streaming read, a {@link StreamingReadResponse} is returned, which contains + * information about the datanode used for the read, and the request observer that can be used to manage the stream + * (e.g., to cancel it if needed). A semaphore is acquired to limit the number of concurrent streaming reads so upon + * successful return of this method, the caller must ensure to call {@link #completeStreamRead(StreamingReadResponse)} + * to release the semaphore once the streaming read is complete. + * @param request The container command request to initiate the streaming read. + * @param streamObserver The observer that will handle the streamed responses. + * @return A {@link StreamingReadResponse} containing details of the streaming read operation. + * @throws IOException + * @throws InterruptedException + */ @Override - public ClientCallStreamObserver streamRead(ContainerCommandRequestProto request, - StreamObserver streamObserver) throws IOException { + public StreamingReadResponse streamRead(ContainerCommandRequestProto request, + StreamObserver streamObserver) throws IOException, InterruptedException { List datanodeList = sortDatanodes(request); IOException lastException = null; for (DatanodeDetails dn : datanodeList) { try { checkOpen(dn); + semaphore.acquire(); XceiverClientProtocolServiceStub stub = asyncStubs.get(dn.getID()); if (stub == null) { throw new IOException("Failed to get gRPC stub for DataNode: " + dn); @@ -536,9 +551,10 @@ public ClientCallStreamObserver streamRead(Contain .send(streamObserver); requestObserver.onNext(request); requestObserver.onCompleted(); - return (ClientCallStreamObserver) requestObserver; + return new StreamingReadResponse(dn, (ClientCallStreamObserver) requestObserver); } catch (IOException e) { LOG.error("Failed to start streaming read to DataNode {}", dn, e); + semaphore.release(); lastException = e; } } @@ -549,6 +565,16 @@ public ClientCallStreamObserver streamRead(Contain } } + /** + * This method should be called to indicate the end of streaming read. Its primary purpose is to release the + * semaphore acquired when starting the streaming read, but is also used to update any metrics or debug logs as + * needed. + */ + @Override + public void completeStreamRead(StreamingReadResponse streamingReadResponse) { + semaphore.release(); + } + private static List sortDatanodeByOperationalState( List datanodeList) { List sortedDatanodeList = new ArrayList<>(datanodeList); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index d06b223c56e4..57b1b3a09326 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -34,6 +34,7 @@ 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.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; @@ -44,7 +45,6 @@ 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.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -223,12 +223,15 @@ private void initialize() throws IOException { try { acquireClient(); streamingReader = new StreamingReader(); - ClientCallStreamObserver requestObserver = + StreamingReadResponse response = ContainerProtocolCalls.readBlock(xceiverClient, position, blockID, tokenRef.get(), pipelineRef.get().getReplicaIndexes(), streamingReader); - streamingReader.setRequestObserver(requestObserver); + streamingReader.setRequestResponse(response); initialized = true; return; + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + handleExceptions(new IOException("Interrupted", ie)); } catch (IOException ioe) { handleExceptions(ioe); } @@ -283,8 +286,9 @@ public class StreamingReader implements StreamObserver 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 error = new AtomicReference<>(); - private ClientCallStreamObserver requestObserver; + private StreamingReadResponse response; // TODO: Semaphore in XceiverClient which count open stream? public boolean hasNext() { @@ -336,9 +340,8 @@ public ByteBuffer readNext() throws IOException { return buf; } - public void setRequestObserver( - ClientCallStreamObserver requestObserver) { - this.requestObserver = requestObserver; + public void setRequestResponse(StreamingReadResponse streamingReadResponse) { + this.response = streamingReadResponse; } /** @@ -346,9 +349,17 @@ public void setRequestObserver( * cause the onError() to be called in this observer with a CANCELLED exception. */ public void cancel() { - if (requestObserver != null) { - requestObserver.cancel("Cancelled by client", CANCELLED_EXCEPTION); + if (response != null && response.getRequestObserver() != null) { + response.getRequestObserver().cancel("Cancelled by client", CANCELLED_EXCEPTION); completed.set(true); + releaseResources(); + } + } + + private void releaseResources() { + boolean wasNotYetComplete = semaphoreReleased.getAndSet(true); + if (wasNotYetComplete) { + xceiverClient.completeStreamRead(response); } } @@ -375,12 +386,14 @@ public void onError(Throwable throwable) { } else { failed.set(true); error.set(throwable); + releaseResources(); } } @Override public void onCompleted() { completed.set(true); + releaseResources(); } private void offerToQueue(ReadBlockResponseProto item) { diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index 9c5a33f75361..be4c8b0a3fe4 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -39,10 +39,12 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.StreamingReadResponse; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientGrpc; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; @@ -132,7 +134,7 @@ public void teardown() { } @Test - public void testCloseStreamReleasesResources() throws IOException { + public void testCloseStreamReleasesResources() throws IOException, InterruptedException { setupSuccessfulRead(); assertEquals(data[0], blockStream.read()); blockStream.close(); @@ -140,10 +142,11 @@ public void testCloseStreamReleasesResources() throws IOException { verify(requestObserver).cancel(any(), any()); // Verify that release() was called on the xceiverClient mock verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); + verify(xceiverClient, times(1)).completeStreamRead(any()); } @Test - public void testUnbufferReleasesResourcesAndResumesFromLastPosition() throws IOException { + public void testUnbufferReleasesResourcesAndResumesFromLastPosition() throws IOException, InterruptedException { setupSuccessfulRead(); assertEquals(data[0], blockStream.read()); assertEquals(1, blockStream.getPos()); @@ -152,19 +155,21 @@ public void testUnbufferReleasesResourcesAndResumesFromLastPosition() throws IOE verify(requestObserver).cancel(any(), any()); // Verify that release() was called on the xceiverClient mock verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); + verify(xceiverClient, times(1)).completeStreamRead(any()); // The next read should "rebuffer" and continue from the last position assertEquals(data[1], blockStream.read()); assertEquals(2, blockStream.getPos()); } @Test - public void testSeekReleasesTheStreamAndStartsFromNewPosition() throws IOException { + public void testSeekReleasesTheStreamAndStartsFromNewPosition() throws IOException, InterruptedException { setupSuccessfulRead(); assertEquals(data[0], blockStream.read()); blockStream.seek(100); assertEquals(100, blockStream.getPos()); // Verify that cancel() was called on the requestObserver mock verify(requestObserver).cancel(any(), any()); + verify(xceiverClient, times(1)).completeStreamRead(any()); // The xceiverClient should not be released verify(xceiverClientFactory, never()) .releaseClientForReadData(xceiverClient, false); @@ -174,27 +179,28 @@ public void testSeekReleasesTheStreamAndStartsFromNewPosition() throws IOExcepti } @Test - public void testErrorThrownIfStreamReturnsError() throws IOException { + public void testErrorThrownIfStreamReturnsError() throws IOException, InterruptedException { // Note the error will only be thrown when the buffer needs to be refilled. I think case, as its the first // read it will try to fill the buffer and encounter the error, but a reader could continue reading until the // buffer is exhausted before seeing the error. when(xceiverClient.streamRead(any(), any())).thenAnswer((InvocationOnMock invocation) -> { StreamObserver streamObserver = invocation.getArgument(1); streamObserver.onError(new IOException("Test induced error")); - return requestObserver; + return new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); }); assertThrows(IOException.class, () -> blockStream.read()); + verify(xceiverClient, times(0)).completeStreamRead(any()); } @Test - public void seekOutOfBounds() throws IOException { + public void seekOutOfBounds() throws IOException, InterruptedException { setupSuccessfulRead(); assertThrows(IOException.class, () -> blockStream.seek(-1)); assertThrows(IOException.class, () -> blockStream.seek(BLOCK_SIZE + 1)); } @Test - public void readPastEOFReturnsEOF() throws IOException { + public void readPastEOFReturnsEOF() throws IOException, InterruptedException { setupSuccessfulRead(); blockStream.seek(BLOCK_SIZE); // Ensure the stream is at EOF even after two attempts to read @@ -204,7 +210,7 @@ public void readPastEOFReturnsEOF() throws IOException { } @Test - public void ensureExceptionThrownForReadAfterClosed() throws IOException { + public void ensureExceptionThrownForReadAfterClosed() throws IOException, InterruptedException { setupSuccessfulRead(); blockStream.close(); ByteBuffer byteBuffer = ByteBuffer.allocate(10); @@ -220,7 +226,8 @@ public void ensureExceptionThrownForReadAfterClosed() throws IOException { @ParameterizedTest @MethodSource("exceptionsTriggeringRefresh") - public void testRefreshFunctionCalledForAllDNsBadOnInitialize(IOException thrown) throws IOException { + public void testRefreshFunctionCalledForAllDNsBadOnInitialize(IOException thrown) + throws IOException, InterruptedException { // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline when(xceiverClient.streamRead(any(), any())) .thenThrow(thrown) @@ -228,7 +235,7 @@ public void testRefreshFunctionCalledForAllDNsBadOnInitialize(IOException thrown StreamObserver streamObserver = invocation.getArgument(1); streamObserver.onNext(createChunkResponse()); streamObserver.onCompleted(); - return requestObserver; + return new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); }); blockStream.read(); verify(refreshFunction, times(1)).apply(any()); @@ -236,7 +243,8 @@ public void testRefreshFunctionCalledForAllDNsBadOnInitialize(IOException thrown @ParameterizedTest @MethodSource("exceptionsNotTriggeringRefresh") - public void testRefreshNotCalledForAllDNsBadOnInitialize(IOException thrown) throws IOException { + public void testRefreshNotCalledForAllDNsBadOnInitialize(IOException thrown) + throws IOException, InterruptedException { // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline when(xceiverClient.streamRead(any(), any())) .thenThrow(thrown); @@ -245,7 +253,7 @@ public void testRefreshNotCalledForAllDNsBadOnInitialize(IOException thrown) thr } @Test - public void testExceptionThrownAfterRetriesExhausted() throws IOException { + public void testExceptionThrownAfterRetriesExhausted() throws IOException, InterruptedException { // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline when(xceiverClient.streamRead(any(), any())).thenThrow(new StorageContainerException(CONTAINER_NOT_FOUND)); assertThrows(IOException.class, () -> blockStream.read()); @@ -258,12 +266,12 @@ private void createDataAndChecksum() throws OzoneChecksumException { checksumData = checksum.computeChecksum(data); } - private void setupSuccessfulRead() throws IOException { + private void setupSuccessfulRead() throws IOException, InterruptedException { when(xceiverClient.streamRead(any(), any())).thenAnswer((InvocationOnMock invocation) -> { StreamObserver streamObserver = invocation.getArgument(1); streamObserver.onNext(createChunkResponse()); streamObserver.onCompleted(); - return requestObserver; + return new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); }); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReadResponse.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReadResponse.java new file mode 100644 index 000000000000..ea8694cd8b78 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReadResponse.java @@ -0,0 +1,46 @@ +/* + * 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; + +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; + +/** + * Streaming read response holding datanode details and + * request observer to send read requests. + */ +public class StreamingReadResponse { + + private final DatanodeDetails dn; + private final ClientCallStreamObserver requestObserver; + + public StreamingReadResponse(DatanodeDetails dn, + ClientCallStreamObserver requestObserver) { + this.dn = dn; + this.requestObserver = requestObserver; + } + + public DatanodeDetails getDatanodeDetails() { + return dn; + } + + public ClientCallStreamObserver getRequestObserver() { + return requestObserver; + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java index f823946f7fbd..b73133f4aac2 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; -import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.function.CheckedBiConsumer; @@ -146,8 +145,12 @@ public ContainerCommandResponseProto sendCommand( } } - public ClientCallStreamObserver streamRead(ContainerCommandRequestProto request, - StreamObserver streamObserver) throws IOException { + public StreamingReadResponse streamRead(ContainerCommandRequestProto request, + StreamObserver streamObserver) throws IOException, InterruptedException { + throw new UnsupportedOperationException("Stream read is not supported"); + } + + public void completeStreamRead(StreamingReadResponse streamingReadResponse) { throw new UnsupportedOperationException("Stream read is not supported"); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index f8be5410576f..424ecb345a2b 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -62,6 +62,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; +import org.apache.hadoop.hdds.scm.StreamingReadResponse; import org.apache.hadoop.hdds.scm.XceiverClientReply; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator; @@ -76,7 +77,6 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.function.CheckedFunction; import org.slf4j.Logger; @@ -917,10 +917,10 @@ public static List toValidatorList(Validator validator) { * @throws IOException if there is an I/O error while performing the call */ @SuppressWarnings("checkstyle:ParameterNumber") - public static ClientCallStreamObserver readBlock( + public static StreamingReadResponse readBlock( XceiverClientSpi xceiverClient, long offset, BlockID blockID, Token token, Map replicaIndexes, StreamObserver streamObserver) - throws IOException { + throws IOException, InterruptedException { final ReadBlockRequestProto.Builder readBlockRequest = ReadBlockRequestProto.newBuilder() .setOffset(offset); @@ -935,10 +935,10 @@ public static ClientCallStreamObserver readBlock( replicaIndexes, streamObserver); } - private static ClientCallStreamObserver readBlock(XceiverClientSpi xceiverClient, + private static StreamingReadResponse readBlock(XceiverClientSpi xceiverClient, BlockID blockID, ContainerCommandRequestProto.Builder builder, ReadBlockRequestProto.Builder readBlockBuilder, DatanodeDetails datanode, Map replicaIndexes, - StreamObserver streamObserver) throws IOException { + StreamObserver streamObserver) throws IOException, InterruptedException { final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); int replicaIndex = replicaIndexes.getOrDefault(datanode, 0); if (replicaIndex > 0) { From ca6131214c6b58f561bd73c8859c1fdc02355fb2 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 23 Oct 2025 20:16:33 +0100 Subject: [PATCH 109/114] Fix findbugs sync warning --- .../hadoop/hdds/scm/storage/StreamBlockInputStream.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index 57b1b3a09326..1803577b4857 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -278,6 +278,10 @@ private void refreshBlockInfo(IOException cause) throws IOException { refreshBlockInfo(cause, blockID, pipelineRef, tokenRef, refreshFunction); } + private synchronized void releaseStreamResources(StreamingReadResponse response) { + xceiverClient.completeStreamRead(response); + } + /** * Implementation of a StreamObserver used to received and buffer streaming GRPC reads. */ @@ -359,7 +363,7 @@ public void cancel() { private void releaseResources() { boolean wasNotYetComplete = semaphoreReleased.getAndSet(true); if (wasNotYetComplete) { - xceiverClient.completeStreamRead(response); + releaseStreamResources(response); } } From 79ceaa907bd4275794fb7b06326fd20638082f1f Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Thu, 23 Oct 2025 20:31:41 +0100 Subject: [PATCH 110/114] Address review comment for handling local errors with onError --- .../hdds/scm/storage/StreamBlockInputStream.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index 1803577b4857..f4f96fa9a892 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -292,7 +292,7 @@ public class StreamingReader implements StreamObserver error = new AtomicReference<>(); - private StreamingReadResponse response; + private volatile StreamingReadResponse response; // TODO: Semaphore in XceiverClient which count open stream? public boolean hasNext() { @@ -378,7 +378,16 @@ public void onNext(ContainerProtos.ContainerCommandResponseProto containerComman } offerToQueue(readBlock); } catch (OzoneChecksumException e) { - // Calling onError will cancel the stream on the server side and also set the failure state. + // Inform the server we want to cancel the RPC by cancelling the request observer + // (this tells the other side to stop sending). Then use local onError handling + // to set failure state and release resources. + try { + if (response != null && response.getRequestObserver() != null) { + response.getRequestObserver().cancel("Checksum failed", e); + } + } catch (Throwable cancelEx) { + LOG.warn("Failed to cancel request observer after checksum error", cancelEx); + } onError(e); } } From 5b704124b728335547b3c16e6f6abea60bd432f2 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 24 Oct 2025 16:36:18 +0100 Subject: [PATCH 111/114] Refactor somewhat to address problems when local checksum validation fails --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 23 ++-- .../scm/storage/StreamBlockInputStream.java | 99 ++++++++++------- .../storage/TestStreamBlockInputStream.java | 100 +++++++++++++----- .../hadoop/hdds/scm/XceiverClientSpi.java | 5 +- .../scm/storage/ContainerProtocolCalls.java | 15 ++- .../rpc/read/TestStreamBlockInputStream.java | 1 - 6 files changed, 152 insertions(+), 91 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index ff8a8d87f4b2..b6a3d00f010f 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -518,21 +518,20 @@ private XceiverClientReply sendCommandWithRetry( /** * Starts a streaming read operation, intended to read entire blocks from the datanodes. This method expects a - * {@link StreamObserver} to be passed in, which will be used to receive the streamed data from the datanode. - * Upon successfully starting the streaming read, a {@link StreamingReadResponse} is returned, which contains - * information about the datanode used for the read, and the request observer that can be used to manage the stream - * (e.g., to cancel it if needed). A semaphore is acquired to limit the number of concurrent streaming reads so upon - * successful return of this method, the caller must ensure to call {@link #completeStreamRead(StreamingReadResponse)} - * to release the semaphore once the streaming read is complete. + * {@link StreamingReaderSpi} to be passed in, which will be used to receive the streamed data from the datanode. + * Upon successfully starting the streaming read, a {@link StreamingReadResponse} is set into the pass StreamObserver, + * which contains information about the datanode used for the read, and the request observer that can be used to + * manage the stream (e.g., to cancel it if needed). A semaphore is acquired to limit the number of concurrent + * streaming reads so upon successful return of this method, the caller must ensure to call + * {@link #completeStreamRead(StreamingReadResponse)} to release the semaphore once the streaming read is complete. * @param request The container command request to initiate the streaming read. - * @param streamObserver The observer that will handle the streamed responses. - * @return A {@link StreamingReadResponse} containing details of the streaming read operation. + * @param streamObserver The observer that will handle the streamed responses.= * @throws IOException * @throws InterruptedException */ @Override - public StreamingReadResponse streamRead(ContainerCommandRequestProto request, - StreamObserver streamObserver) throws IOException, InterruptedException { + public void streamRead(ContainerCommandRequestProto request, + StreamingReaderSpi streamObserver) throws IOException, InterruptedException { List datanodeList = sortDatanodes(request); IOException lastException = null; for (DatanodeDetails dn : datanodeList) { @@ -549,9 +548,11 @@ public StreamingReadResponse streamRead(ContainerCommandRequestProto request, StreamObserver requestObserver = stub .withDeadlineAfter(timeout, TimeUnit.SECONDS) .send(streamObserver); + streamObserver.setStreamingReadResponse(new StreamingReadResponse(dn, + (ClientCallStreamObserver) requestObserver)); requestObserver.onNext(request); requestObserver.onCompleted(); - return new StreamingReadResponse(dn, (ClientCallStreamObserver) requestObserver); + return; } catch (IOException e) { LOG.error("Failed to start streaming read to DataNode {}", dn, e); semaphore.release(); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index f4f96fa9a892..cb2f80ca7c83 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -35,6 +35,7 @@ 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; @@ -45,7 +46,7 @@ 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.stub.StreamObserver; +import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -223,10 +224,8 @@ private void initialize() throws IOException { try { acquireClient(); streamingReader = new StreamingReader(); - StreamingReadResponse response = - ContainerProtocolCalls.readBlock(xceiverClient, position, blockID, tokenRef.get(), - pipelineRef.get().getReplicaIndexes(), streamingReader); - streamingReader.setRequestResponse(response); + ContainerProtocolCalls.readBlock(xceiverClient, position, blockID, tokenRef.get(), + pipelineRef.get().getReplicaIndexes(), streamingReader); initialized = true; return; } catch (InterruptedException ie) { @@ -279,13 +278,15 @@ private void refreshBlockInfo(IOException cause) throws IOException { } private synchronized void releaseStreamResources(StreamingReadResponse response) { - xceiverClient.completeStreamRead(response); + if (xceiverClient != null) { + xceiverClient.completeStreamRead(response); + } } /** * Implementation of a StreamObserver used to received and buffer streaming GRPC reads. */ - public class StreamingReader implements StreamObserver { + public class StreamingReader implements StreamingReaderSpi { private final BlockingQueue responseQueue = new LinkedBlockingQueue<>(1); private final AtomicBoolean completed = new AtomicBoolean(false); @@ -294,7 +295,6 @@ public class StreamingReader implements StreamObserver error = new AtomicReference<>(); private volatile StreamingReadResponse response; - // TODO: Semaphore in XceiverClient which count open stream? public boolean hasNext() { return !responseQueue.isEmpty() || !completed.get(); } @@ -344,22 +344,6 @@ public ByteBuffer readNext() throws IOException { return buf; } - public void setRequestResponse(StreamingReadResponse streamingReadResponse) { - this.response = streamingReadResponse; - } - - /** - * By calling cancel, the client will send a cancel signal to the server, which will stop sending more data and - * cause the onError() to be called in this observer with a CANCELLED exception. - */ - public void cancel() { - if (response != null && response.getRequestObserver() != null) { - response.getRequestObserver().cancel("Cancelled by client", CANCELLED_EXCEPTION); - completed.set(true); - releaseResources(); - } - } - private void releaseResources() { boolean wasNotYetComplete = semaphoreReleased.getAndSet(true); if (wasNotYetComplete) { @@ -378,37 +362,65 @@ public void onNext(ContainerProtos.ContainerCommandResponseProto containerComman } offerToQueue(readBlock); } catch (OzoneChecksumException e) { - // Inform the server we want to cancel the RPC by cancelling the request observer - // (this tells the other side to stop sending). Then use local onError handling - // to set failure state and release resources. - try { - if (response != null && response.getRequestObserver() != null) { - response.getRequestObserver().cancel("Checksum failed", e); - } - } catch (Throwable cancelEx) { - LOG.warn("Failed to cancel request observer after checksum error", cancelEx); - } - onError(e); + LOG.warn("Checksum verification failed for block {} from datanode {}", + getBlockID(), response.getDatanodeDetails(), e); + cancelDueToError(e); } } @Override public void onError(Throwable throwable) { - if (throwable == CANCELLED_EXCEPTION) { - completed.set(true); + if (throwable instanceof StatusRuntimeException) { + if (((StatusRuntimeException) throwable).getStatus().getCode().name().equals("CANCELLED")) { + // This is expected when the client cancels the stream. + setCompleted(); + } } else { - failed.set(true); - error.set(throwable); - releaseResources(); + setFailed(throwable); } + releaseResources(); } @Override public void onCompleted() { - completed.set(true); + setCompleted(); releaseResources(); } + /** + * By calling cancel, the client will send a cancel signal to the server, which will stop sending more data and + * cause the onError() to be called in this observer with a CANCELLED exception. + */ + public void cancel() { + if (response != null && response.getRequestObserver() != null) { + response.getRequestObserver().cancel("Cancelled by client", CANCELLED_EXCEPTION); + setCompleted(); + releaseResources(); + } + } + + public void cancelDueToError(Throwable exception) { + if (response != null && response.getRequestObserver() != null) { + response.getRequestObserver().onError(exception); + setFailed(exception); + releaseResources(); + } + } + + private void setFailed(Throwable throwable) { + if (completed.get()) { + throw new IllegalArgumentException("Cannot mark a completed stream as failed"); + } + failed.set(true); + error.set(throwable); + } + + private void setCompleted() { + if (!failed.get()) { + completed.set(true); + } + } + private void offerToQueue(ReadBlockResponseProto item) { while (!completed.get() && !failed.get()) { try { @@ -421,6 +433,11 @@ private void offerToQueue(ReadBlockResponseProto item) { } } } + + @Override + public void setStreamingReadResponse(StreamingReadResponse streamingReadResponse) { + response = streamingReadResponse; + } } } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index be4c8b0a3fe4..b2cb3fb865cb 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -21,6 +21,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -45,6 +46,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; 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.XceiverClientGrpc; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; @@ -61,7 +63,6 @@ import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusException; import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; -import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -183,11 +184,14 @@ public void testErrorThrownIfStreamReturnsError() throws IOException, Interrupte // Note the error will only be thrown when the buffer needs to be refilled. I think case, as its the first // read it will try to fill the buffer and encounter the error, but a reader could continue reading until the // buffer is exhausted before seeing the error. - when(xceiverClient.streamRead(any(), any())).thenAnswer((InvocationOnMock invocation) -> { - StreamObserver streamObserver = invocation.getArgument(1); + doAnswer((InvocationOnMock invocation) -> { + StreamingReaderSpi streamObserver = invocation.getArgument(1); + StreamingReadResponse resp = + new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); + streamObserver.setStreamingReadResponse(resp); streamObserver.onError(new IOException("Test induced error")); - return new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); - }); + return null; + }).when(xceiverClient).streamRead(any(), any()); assertThrows(IOException.class, () -> blockStream.read()); verify(xceiverClient, times(0)).completeStreamRead(any()); } @@ -229,14 +233,18 @@ public void ensureExceptionThrownForReadAfterClosed() throws IOException, Interr public void testRefreshFunctionCalledForAllDNsBadOnInitialize(IOException thrown) throws IOException, InterruptedException { // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline - when(xceiverClient.streamRead(any(), any())) - .thenThrow(thrown) - .thenAnswer((InvocationOnMock invocation) -> { - StreamObserver streamObserver = invocation.getArgument(1); - streamObserver.onNext(createChunkResponse()); - streamObserver.onCompleted(); - return new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); - }); + + doAnswer((InvocationOnMock invocation) -> { + throw thrown; + }).doAnswer((InvocationOnMock invocation) -> { + StreamingReaderSpi streamObserver = invocation.getArgument(1); + StreamingReadResponse resp = + new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); + streamObserver.setStreamingReadResponse(resp); + streamObserver.onNext(createChunkResponse(false)); + streamObserver.onCompleted(); + return null; + }).when(xceiverClient).streamRead(any(), any()); blockStream.read(); verify(refreshFunction, times(1)).apply(any()); } @@ -246,8 +254,9 @@ public void testRefreshFunctionCalledForAllDNsBadOnInitialize(IOException thrown public void testRefreshNotCalledForAllDNsBadOnInitialize(IOException thrown) throws IOException, InterruptedException { // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline - when(xceiverClient.streamRead(any(), any())) - .thenThrow(thrown); + doAnswer((InvocationOnMock invocation) -> { + throw thrown; + }).when(xceiverClient).streamRead(any(), any()); assertThrows(IOException.class, () -> blockStream.read()); verify(refreshFunction, times(0)).apply(any()); } @@ -255,11 +264,28 @@ public void testRefreshNotCalledForAllDNsBadOnInitialize(IOException thrown) @Test public void testExceptionThrownAfterRetriesExhausted() throws IOException, InterruptedException { // In this case, if the first attempt to connect to any of the DNs fails, it should retry by refreshing the pipeline - when(xceiverClient.streamRead(any(), any())).thenThrow(new StorageContainerException(CONTAINER_NOT_FOUND)); + doAnswer((InvocationOnMock invocation) -> { + throw new StorageContainerException(CONTAINER_NOT_FOUND); + }).when(xceiverClient).streamRead(any(), any()); + assertThrows(IOException.class, () -> blockStream.read()); verify(refreshFunction, times(1)).apply(any()); } + @Test + public void testInvalidChecksumThrowsException() throws IOException, InterruptedException { + doAnswer((InvocationOnMock invocation) -> { + StreamingReaderSpi streamObserver = invocation.getArgument(1); + StreamingReadResponse resp = + new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); + streamObserver.setStreamingReadResponse(resp); + streamObserver.onNext(createChunkResponse(true)); + streamObserver.onCompleted(); + return null; + }).when(xceiverClient).streamRead(any(), any()); + assertThrows(IOException.class, () -> blockStream.read()); + } + private void createDataAndChecksum() throws OzoneChecksumException { data = new byte[BLOCK_SIZE]; new SecureRandom().nextBytes(data); @@ -267,20 +293,20 @@ private void createDataAndChecksum() throws OzoneChecksumException { } private void setupSuccessfulRead() throws IOException, InterruptedException { - when(xceiverClient.streamRead(any(), any())).thenAnswer((InvocationOnMock invocation) -> { - StreamObserver streamObserver = invocation.getArgument(1); - streamObserver.onNext(createChunkResponse()); + doAnswer((InvocationOnMock invocation) -> { + StreamingReaderSpi streamObserver = invocation.getArgument(1); + StreamingReadResponse resp = + new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); + streamObserver.setStreamingReadResponse(resp); + streamObserver.onNext(createChunkResponse(false)); streamObserver.onCompleted(); - return new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); - }); + return null; + }).when(xceiverClient).streamRead(any(), any()); } - private ContainerProtos.ContainerCommandResponseProto createChunkResponse() { - ContainerProtos.ReadBlockResponseProto response = ContainerProtos.ReadBlockResponseProto.newBuilder() - .setChecksumData(checksumData.getProtoBufMessage()) - .setData(ByteString.copyFrom(data)) - .setOffset(0) - .build(); + private ContainerProtos.ContainerCommandResponseProto createChunkResponse(boolean invalidChecksum) { + ContainerProtos.ReadBlockResponseProto response = invalidChecksum ? + createInValidChecksumResponse() : createValidResponse(); return ContainerProtos.ContainerCommandResponseProto.newBuilder() .setCmdType(ContainerProtos.Type.ReadBlock) @@ -289,6 +315,26 @@ private ContainerProtos.ContainerCommandResponseProto createChunkResponse() { .build(); } + private ContainerProtos.ReadBlockResponseProto createValidResponse() { + return ContainerProtos.ReadBlockResponseProto.newBuilder() + .setChecksumData(checksumData.getProtoBufMessage()) + .setData(ByteString.copyFrom(data)) + .setOffset(0) + .build(); + } + + private ContainerProtos.ReadBlockResponseProto createInValidChecksumResponse() { + byte[] invalidData = new byte[data.length]; + System.arraycopy(data, 0, invalidData, 0, data.length); + // Corrupt the data + invalidData[0] = (byte) (invalidData[0] + 1); + return ContainerProtos.ReadBlockResponseProto.newBuilder() + .setChecksumData(checksumData.getProtoBufMessage()) + .setData(ByteString.copyFrom(invalidData)) + .setOffset(0) + .build(); + } + private static Stream exceptionsTriggeringRefresh() { return Stream.of( Arguments.of(new StorageContainerException(CONTAINER_NOT_FOUND)), diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java index b73133f4aac2..f1bf7a8ef855 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; -import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.function.CheckedBiConsumer; /** @@ -145,8 +144,8 @@ public ContainerCommandResponseProto sendCommand( } } - public StreamingReadResponse streamRead(ContainerCommandRequestProto request, - StreamObserver streamObserver) throws IOException, InterruptedException { + public void streamRead(ContainerCommandRequestProto request, + StreamingReaderSpi streamObserver) throws IOException, InterruptedException { throw new UnsupportedOperationException("Stream read is not supported"); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 424ecb345a2b..c6e5d75b5caf 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -62,7 +62,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadContainerResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; -import org.apache.hadoop.hdds.scm.StreamingReadResponse; +import org.apache.hadoop.hdds.scm.StreamingReaderSpi; import org.apache.hadoop.hdds.scm.XceiverClientReply; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator; @@ -77,7 +77,6 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.function.CheckedFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -917,9 +916,9 @@ public static List toValidatorList(Validator validator) { * @throws IOException if there is an I/O error while performing the call */ @SuppressWarnings("checkstyle:ParameterNumber") - public static StreamingReadResponse readBlock( + public static void readBlock( XceiverClientSpi xceiverClient, long offset, BlockID blockID, Token token, - Map replicaIndexes, StreamObserver streamObserver) + Map replicaIndexes, StreamingReaderSpi streamObserver) throws IOException, InterruptedException { final ReadBlockRequestProto.Builder readBlockRequest = ReadBlockRequestProto.newBuilder() @@ -931,14 +930,14 @@ public static StreamingReadResponse readBlock( builder.setEncodedToken(token.encodeToUrlString()); } - return readBlock(xceiverClient, blockID, builder, readBlockRequest, xceiverClient.getPipeline().getFirstNode(), + readBlock(xceiverClient, blockID, builder, readBlockRequest, xceiverClient.getPipeline().getFirstNode(), replicaIndexes, streamObserver); } - private static StreamingReadResponse readBlock(XceiverClientSpi xceiverClient, + private static void readBlock(XceiverClientSpi xceiverClient, BlockID blockID, ContainerCommandRequestProto.Builder builder, ReadBlockRequestProto.Builder readBlockBuilder, DatanodeDetails datanode, Map replicaIndexes, - StreamObserver streamObserver) throws IOException, InterruptedException { + StreamingReaderSpi streamObserver) throws IOException, InterruptedException { final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); int replicaIndex = replicaIndexes.getOrDefault(datanode, 0); if (replicaIndex > 0) { @@ -948,6 +947,6 @@ private static StreamingReadResponse readBlock(XceiverClientSpi xceiverClient, final ContainerCommandRequestProto request = builder .setDatanodeUuid(datanode.getUuidString()) .setReadBlock(readBlockBuilder).build(); - return xceiverClient.streamRead(request, streamObserver); + xceiverClient.streamRead(request, streamObserver); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java index 884e8bbb9e01..bb66a303155e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestStreamBlockInputStream.java @@ -42,7 +42,6 @@ public class TestStreamBlockInputStream extends TestInputStreamBase { * Run the tests as a single test method to avoid needing a new mini-cluster * for each test. */ - private static final int DATA_LENGTH = (2 * BLOCK_SIZE) + (CHUNK_SIZE); private byte[] inputData; private TestBucket bucket; From 726f8e4857f42446fe34e5acdbb43a3c0469697f Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Fri, 24 Oct 2025 16:40:51 +0100 Subject: [PATCH 112/114] Adding new file missed in last commit --- .../hadoop/hdds/scm/StreamingReaderSpi.java | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReaderSpi.java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReaderSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReaderSpi.java new file mode 100644 index 000000000000..0206253784cf --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReaderSpi.java @@ -0,0 +1,30 @@ +/* + * 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; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; + +/** + * SPI for streaming reader to set the streaming read response. + */ +public interface StreamingReaderSpi extends StreamObserver { + + void setStreamingReadResponse(StreamingReadResponse streamingReadResponse); + +} From 117fa17b989c3090de8341115434939c36d480b9 Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Tue, 18 Nov 2025 12:14:09 +0000 Subject: [PATCH 113/114] Implement flow control to backoff the stream when the client is slow --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 15 ++-- .../scm/storage/StreamBlockInputStream.java | 32 ++++--- .../storage/TestStreamBlockInputStream.java | 29 +++---- .../hadoop/hdds/scm/StreamingReaderSpi.java | 8 +- .../hadoop/hdds/scm/XceiverClientSpi.java | 2 +- .../transport/server/GrpcXceiverService.java | 6 ++ .../container/keyvalue/KeyValueHandler.java | 84 ++++++++++++++----- .../main/proto/DatanodeClientProtocol.proto | 2 +- 8 files changed, 112 insertions(+), 66 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index b6a3d00f010f..923a61e384e2 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -63,7 +63,6 @@ import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; -import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.slf4j.Logger; @@ -523,7 +522,7 @@ private XceiverClientReply sendCommandWithRetry( * which contains information about the datanode used for the read, and the request observer that can be used to * manage the stream (e.g., to cancel it if needed). A semaphore is acquired to limit the number of concurrent * streaming reads so upon successful return of this method, the caller must ensure to call - * {@link #completeStreamRead(StreamingReadResponse)} to release the semaphore once the streaming read is complete. + * {@link #completeStreamRead()} to release the semaphore once the streaming read is complete. * @param request The container command request to initiate the streaming read. * @param streamObserver The observer that will handle the streamed responses.= * @throws IOException @@ -545,13 +544,9 @@ public void streamRead(ContainerCommandRequestProto request, if (LOG.isDebugEnabled()) { LOG.debug("Executing command {} on datanode {}", processForDebug(request), dn); } - StreamObserver requestObserver = stub - .withDeadlineAfter(timeout, TimeUnit.SECONDS) - .send(streamObserver); - streamObserver.setStreamingReadResponse(new StreamingReadResponse(dn, - (ClientCallStreamObserver) requestObserver)); - requestObserver.onNext(request); - requestObserver.onCompleted(); + stub.withDeadlineAfter(timeout, TimeUnit.SECONDS) + .streamBlock(request, streamObserver); + streamObserver.setStreamingDatanode(dn); return; } catch (IOException e) { LOG.error("Failed to start streaming read to DataNode {}", dn, e); @@ -572,7 +567,7 @@ public void streamRead(ContainerCommandRequestProto request, * needed. */ @Override - public void completeStreamRead(StreamingReadResponse streamingReadResponse) { + public void completeStreamRead() { semaphore.release(); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index cb2f80ca7c83..c0ef63cd9cf4 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -31,10 +31,10 @@ 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.DatanodeDetails; 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; @@ -47,6 +47,7 @@ import org.apache.hadoop.ozone.common.OzoneChecksumException; import org.apache.hadoop.security.token.Token; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; +import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -277,9 +278,9 @@ private void refreshBlockInfo(IOException cause) throws IOException { refreshBlockInfo(cause, blockID, pipelineRef, tokenRef, refreshFunction); } - private synchronized void releaseStreamResources(StreamingReadResponse response) { + private synchronized void releaseStreamResources() { if (xceiverClient != null) { - xceiverClient.completeStreamRead(response); + xceiverClient.completeStreamRead(); } } @@ -293,7 +294,8 @@ public class StreamingReader implements StreamingReaderSpi { private final AtomicBoolean failed = new AtomicBoolean(false); private final AtomicBoolean semaphoreReleased = new AtomicBoolean(false); private final AtomicReference error = new AtomicReference<>(); - private volatile StreamingReadResponse response; + private volatile ClientCallStreamObserver requestObserver; + private volatile DatanodeDetails streamingDatanodeDetails; public boolean hasNext() { return !responseQueue.isEmpty() || !completed.get(); @@ -347,7 +349,7 @@ public ByteBuffer readNext() throws IOException { private void releaseResources() { boolean wasNotYetComplete = semaphoreReleased.getAndSet(true); if (wasNotYetComplete) { - releaseStreamResources(response); + releaseStreamResources(); } } @@ -363,7 +365,7 @@ public void onNext(ContainerProtos.ContainerCommandResponseProto containerComman offerToQueue(readBlock); } catch (OzoneChecksumException e) { LOG.warn("Checksum verification failed for block {} from datanode {}", - getBlockID(), response.getDatanodeDetails(), e); + getBlockID(), streamingDatanodeDetails, e); cancelDueToError(e); } } @@ -392,16 +394,16 @@ public void onCompleted() { * cause the onError() to be called in this observer with a CANCELLED exception. */ public void cancel() { - if (response != null && response.getRequestObserver() != null) { - response.getRequestObserver().cancel("Cancelled by client", CANCELLED_EXCEPTION); + if (requestObserver != null) { + requestObserver.cancel("Cancelled by client", CANCELLED_EXCEPTION); setCompleted(); releaseResources(); } } public void cancelDueToError(Throwable exception) { - if (response != null && response.getRequestObserver() != null) { - response.getRequestObserver().onError(exception); + if (requestObserver != null) { + requestObserver.cancel("Cancelled by client due to error", CANCELLED_EXCEPTION); setFailed(exception); releaseResources(); } @@ -435,8 +437,14 @@ private void offerToQueue(ReadBlockResponseProto item) { } @Override - public void setStreamingReadResponse(StreamingReadResponse streamingReadResponse) { - response = streamingReadResponse; + public void setStreamingDatanode(DatanodeDetails datanodeDetails) { + streamingDatanodeDetails = datanodeDetails; + } + + @Override + public void beforeStart( + ClientCallStreamObserver clientCallStreamObserver) { + this.requestObserver = clientCallStreamObserver; } } diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java index b2cb3fb865cb..3fa1e8ac3567 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestStreamBlockInputStream.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; 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.XceiverClientGrpc; @@ -143,7 +142,7 @@ public void testCloseStreamReleasesResources() throws IOException, InterruptedEx verify(requestObserver).cancel(any(), any()); // Verify that release() was called on the xceiverClient mock verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); - verify(xceiverClient, times(1)).completeStreamRead(any()); + verify(xceiverClient, times(1)).completeStreamRead(); } @Test @@ -156,7 +155,7 @@ public void testUnbufferReleasesResourcesAndResumesFromLastPosition() throws IOE verify(requestObserver).cancel(any(), any()); // Verify that release() was called on the xceiverClient mock verify(xceiverClientFactory).releaseClientForReadData(xceiverClient, false); - verify(xceiverClient, times(1)).completeStreamRead(any()); + verify(xceiverClient, times(1)).completeStreamRead(); // The next read should "rebuffer" and continue from the last position assertEquals(data[1], blockStream.read()); assertEquals(2, blockStream.getPos()); @@ -170,7 +169,7 @@ public void testSeekReleasesTheStreamAndStartsFromNewPosition() throws IOExcepti assertEquals(100, blockStream.getPos()); // Verify that cancel() was called on the requestObserver mock verify(requestObserver).cancel(any(), any()); - verify(xceiverClient, times(1)).completeStreamRead(any()); + verify(xceiverClient, times(1)).completeStreamRead(); // The xceiverClient should not be released verify(xceiverClientFactory, never()) .releaseClientForReadData(xceiverClient, false); @@ -186,14 +185,13 @@ public void testErrorThrownIfStreamReturnsError() throws IOException, Interrupte // buffer is exhausted before seeing the error. doAnswer((InvocationOnMock invocation) -> { StreamingReaderSpi streamObserver = invocation.getArgument(1); - StreamingReadResponse resp = - new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); - streamObserver.setStreamingReadResponse(resp); + streamObserver.setStreamingDatanode(MockDatanodeDetails.randomDatanodeDetails()); + streamObserver.beforeStart(requestObserver); streamObserver.onError(new IOException("Test induced error")); return null; }).when(xceiverClient).streamRead(any(), any()); assertThrows(IOException.class, () -> blockStream.read()); - verify(xceiverClient, times(0)).completeStreamRead(any()); + verify(xceiverClient, times(0)).completeStreamRead(); } @Test @@ -238,9 +236,8 @@ public void testRefreshFunctionCalledForAllDNsBadOnInitialize(IOException thrown throw thrown; }).doAnswer((InvocationOnMock invocation) -> { StreamingReaderSpi streamObserver = invocation.getArgument(1); - StreamingReadResponse resp = - new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); - streamObserver.setStreamingReadResponse(resp); + streamObserver.setStreamingDatanode(MockDatanodeDetails.randomDatanodeDetails()); + streamObserver.beforeStart(requestObserver); streamObserver.onNext(createChunkResponse(false)); streamObserver.onCompleted(); return null; @@ -276,9 +273,8 @@ public void testExceptionThrownAfterRetriesExhausted() throws IOException, Inter public void testInvalidChecksumThrowsException() throws IOException, InterruptedException { doAnswer((InvocationOnMock invocation) -> { StreamingReaderSpi streamObserver = invocation.getArgument(1); - StreamingReadResponse resp = - new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); - streamObserver.setStreamingReadResponse(resp); + streamObserver.setStreamingDatanode(MockDatanodeDetails.randomDatanodeDetails()); + streamObserver.beforeStart(requestObserver); streamObserver.onNext(createChunkResponse(true)); streamObserver.onCompleted(); return null; @@ -295,9 +291,8 @@ private void createDataAndChecksum() throws OzoneChecksumException { private void setupSuccessfulRead() throws IOException, InterruptedException { doAnswer((InvocationOnMock invocation) -> { StreamingReaderSpi streamObserver = invocation.getArgument(1); - StreamingReadResponse resp = - new StreamingReadResponse(MockDatanodeDetails.randomDatanodeDetails(), requestObserver); - streamObserver.setStreamingReadResponse(resp); + streamObserver.setStreamingDatanode(MockDatanodeDetails.randomDatanodeDetails()); + streamObserver.beforeStart(requestObserver); streamObserver.onNext(createChunkResponse(false)); streamObserver.onCompleted(); return null; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReaderSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReaderSpi.java index 0206253784cf..e0c45c956e8f 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReaderSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/StreamingReaderSpi.java @@ -17,14 +17,16 @@ package org.apache.hadoop.hdds.scm; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; +import org.apache.ratis.thirdparty.io.grpc.stub.ClientResponseObserver; /** * SPI for streaming reader to set the streaming read response. */ -public interface StreamingReaderSpi extends StreamObserver { +public interface StreamingReaderSpi extends ClientResponseObserver + { - void setStreamingReadResponse(StreamingReadResponse streamingReadResponse); + void setStreamingDatanode(DatanodeDetails datanodeDetails); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java index f1bf7a8ef855..e008765f7e7e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java @@ -149,7 +149,7 @@ public void streamRead(ContainerCommandRequestProto request, throw new UnsupportedOperationException("Stream read is not supported"); } - public void completeStreamRead(StreamingReadResponse streamingReadResponse) { + public void completeStreamRead() { throw new UnsupportedOperationException("Stream read is not supported"); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java index 041958b42279..e5499ed5d3a4 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java @@ -90,6 +90,12 @@ private static void addZeroCopyMethod( newServiceBuilder.addMethod(newMethod, serverCallHandler); } + @Override + public void streamBlock(ContainerCommandRequestProto request, + StreamObserver responseObserver) { + dispatcher.streamDataReadOnly(request, responseObserver, null); + } + @Override public StreamObserver send( StreamObserver responseObserver) { 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 2f4177b2d065..fbf0d6aa8f84 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 @@ -169,6 +169,7 @@ import org.apache.hadoop.util.Time; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.stub.ServerCallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -2091,40 +2092,79 @@ public ContainerCommandResponseProto readBlock( // 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; + ChecksumData noChecksumChecksumData; int bytesPerChecksum = STREAMING_BYTES_PER_CHUNK; if (checksumType == ContainerProtos.ChecksumType.NONE) { - checksumData = new ChecksumData(checksumType, 0); + noChecksumChecksumData = new ChecksumData(checksumType, 0); } else { + noChecksumChecksumData = null; bytesPerChecksum = chunkInfos.get(0).getChecksumData().getBytesPerChecksum(); } + final int finalBytesPerChecksum = bytesPerChecksum; // 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; + ServerCallStreamObserver serverCallStreamObserver = + ((ServerCallStreamObserver) streamObserver); + Runnable dataStream = new Runnable() { + + private final RandomAccessFile file = new RandomAccessFile(blockFile, "r"); + private final FileChannel channel = file.getChannel(); + private final ByteBuffer buffer = ByteBuffer.allocate(finalBytesPerChecksum); + private long adjustedOffset = readBlock.getOffset() - readBlock.getOffset() % finalBytesPerChecksum; + + @Override + public void run() { + try { + channel.position(adjustedOffset); + while (serverCallStreamObserver.isReady() && !serverCallStreamObserver.isCancelled()) { + int read = channel.read(buffer); + if (read == -1) { + serverCallStreamObserver.onCompleted(); + channel.close(); + file.close(); + break; + } + buffer.flip(); + ChecksumData checksumData; + 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 / finalBytesPerChecksum; + ByteString checksum = + blockData.getChunks().get(chunkIndex).getChecksumData().getChecksums(checksumIndex); + checksumData = + new ChecksumData(checksumType, finalBytesPerChecksum, Collections.singletonList(checksum)); + } else { + checksumData = noChecksumChecksumData; + } + serverCallStreamObserver.onNext(getReadBlockResponse(request, checksumData, buffer, adjustedOffset)); + adjustedOffset += finalBytesPerChecksum; + buffer.clear(); + } + if (serverCallStreamObserver.isCancelled()) { + channel.close(); + file.close(); + } + } catch (IOException e) { + LOG.error("Error while reading block file {}", blockFile, e); + serverCallStreamObserver.onError(e); + try { + channel.close(); + file.close(); + } catch (IOException ioException) { + LOG.error("Error while closing file/channel for block file {}", blockFile, ioException); + } + } } - } + }; + serverCallStreamObserver.setOnReadyHandler(dataStream); + dataStream.run(); // TODO metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen()); } catch (StorageContainerException ex) { responseProto = ContainerUtils.logAndReturnError(LOG, ex, request); diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 6b4d8f1bd7f9..8ef8705c713e 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -600,7 +600,7 @@ service XceiverClientProtocolService { // A client-to-datanode RPC to send container commands rpc send(stream ContainerCommandRequestProto) returns (stream ContainerCommandResponseProto) {}; - + rpc streamBlock(ContainerCommandRequestProto) returns (stream ContainerCommandResponseProto); } service IntraDatanodeProtocolService { From 4375666df733bd5ca2d41590165e60512f245e28 Mon Sep 17 00:00:00 2001 From: Stephen O'Donnell Date: Tue, 25 Nov 2025 13:25:44 +0000 Subject: [PATCH 114/114] Update hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java Co-authored-by: Chung En Lee --- .../apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java index c0ef63cd9cf4..6284bd8b35f1 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java @@ -347,7 +347,7 @@ public ByteBuffer readNext() throws IOException { } private void releaseResources() { - boolean wasNotYetComplete = semaphoreReleased.getAndSet(true); + boolean wasNotYetComplete = !semaphoreReleased.getAndSet(true); if (wasNotYetComplete) { releaseStreamResources(); }