From 5bb86cacc2e5261d05145e3a343b03beeaf0db9d Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Thu, 7 Aug 2025 12:32:50 +0530 Subject: [PATCH 01/10] use actual container size when exporting/importing containers --- .../replication/ContainerDownloader.java | 4 +-- .../replication/ContainerImporter.java | 16 ++++++++-- .../ContainerReplicationSource.java | 13 +++++++++ .../replication/ContainerUploader.java | 2 +- .../CopyContainerResponseStream.java | 22 +++++++++++--- .../DownloadAndImportReplicator.java | 24 +++++++++++++-- .../replication/GrpcContainerUploader.java | 4 +-- .../replication/GrpcReplicationClient.java | 25 ++++++++++++---- .../replication/GrpcReplicationService.java | 10 ++++++- .../OnDemandContainerReplicationSource.java | 10 +++++++ .../container/replication/PushReplicator.java | 11 ++++++- .../SendContainerOutputStream.java | 16 ++++++---- .../SendContainerRequestHandler.java | 29 ++++++++++++++++--- .../SimpleContainerDownloader.java | 7 +++-- .../TestGrpcContainerUploader.java | 2 +- .../TestGrpcReplicationService.java | 2 +- .../replication/TestPushReplicator.java | 2 +- .../TestReplicationSupervisor.java | 7 +++-- .../TestSendContainerOutputStream.java | 4 +-- .../TestSimpleContainerDownloader.java | 13 +++++---- .../main/proto/DatanodeClientProtocol.proto | 2 ++ .../ozoneimpl/TestOzoneContainerWithTLS.java | 7 +++-- 22 files changed, 181 insertions(+), 51 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerDownloader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerDownloader.java index 28879ffde18d..8063a7187de5 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerDownloader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerDownloader.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.nio.file.Path; import java.util.List; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.protocol.DatanodeDetails; /** @@ -32,8 +33,7 @@ */ public interface ContainerDownloader extends Closeable { - Path getContainerDataFromReplicas(long containerId, + Pair getContainerDataFromReplicas(long containerId, List sources, Path downloadDir, CopyContainerCompression compression); - } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java index 66aa19d5580a..f72f8b92ea00 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java @@ -60,7 +60,7 @@ public class ContainerImporter { private final ContainerController controller; private final MutableVolumeSet volumeSet; private final VolumeChoosingPolicy volumeChoosingPolicy; - private final long containerSize; + private final long defaultContainerSize; private final Set importContainerProgress = Collections.synchronizedSet(new HashSet<>()); @@ -76,7 +76,7 @@ public ContainerImporter(@Nonnull ConfigurationSource conf, this.controller = controller; this.volumeSet = volumeSet; this.volumeChoosingPolicy = volumeChoosingPolicy; - containerSize = (long) conf.getStorageSize( + defaultContainerSize = (long) conf.getStorageSize( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); this.conf = conf; @@ -174,6 +174,16 @@ protected TarContainerPacker getPacker(CopyContainerCompression compression) { } public long getDefaultReplicationSpace() { - return HddsServerUtil.requiredReplicationSpace(containerSize); + return HddsServerUtil.requiredReplicationSpace(defaultContainerSize); + } + + /** + * Calculate required replication space based on actual container size. + * + * @param actualContainerSize the actual size of the container in bytes + * @return required space for replication (2 * actualContainerSize) + */ + public long getRequiredReplicationSpace(long actualContainerSize) { + return HddsServerUtil.requiredReplicationSpace(actualContainerSize); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerReplicationSource.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerReplicationSource.java index ad0e1cddde6d..65cb9b1f49d0 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerReplicationSource.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerReplicationSource.java @@ -48,4 +48,17 @@ void copyData(long containerId, OutputStream destination, CopyContainerCompression compression) throws IOException; + /** + * Get the actual size of the container in bytes. + * This is used for space reservation during replication. + * + * @param containerId ID of the container + * @return actual container size in bytes, or null if size cannot be determined + * @throws IOException if container is not found or size cannot be retrieved + */ + default Long getContainerSize(long containerId) throws IOException { + // Default implementation returns null for backward compatibility + return null; + } + } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java index 55874511ba7c..b5c1ea17c434 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java @@ -27,6 +27,6 @@ */ public interface ContainerUploader { OutputStream startUpload(long containerId, DatanodeDetails target, - CompletableFuture callback, CopyContainerCompression compression) + CompletableFuture callback, CopyContainerCompression compression, Long containerSize) throws IOException; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/CopyContainerResponseStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/CopyContainerResponseStream.java index 61cecf1255b1..6b078f7a6447 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/CopyContainerResponseStream.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/CopyContainerResponseStream.java @@ -27,22 +27,36 @@ class CopyContainerResponseStream extends GrpcOutputStream { + private final Long containerSize; + CopyContainerResponseStream( CallStreamObserver streamObserver, long containerId, int bufferSize) { + this(streamObserver, containerId, bufferSize, null); + } + + CopyContainerResponseStream( + CallStreamObserver streamObserver, + long containerId, int bufferSize, Long containerSize) { super(streamObserver, containerId, bufferSize); + this.containerSize = containerSize; } @Override protected void sendPart(boolean eof, int length, ByteString data) { - CopyContainerResponseProto response = + CopyContainerResponseProto.Builder responseBuilder = CopyContainerResponseProto.newBuilder() .setContainerID(getContainerId()) .setData(data) .setEof(eof) .setReadOffset(getWrittenBytes()) - .setLen(length) - .build(); - getStreamObserver().onNext(response); + .setLen(length); + + // Include container size in first response + if (getWrittenBytes() == 0 && containerSize != null) { + responseBuilder.setActualContainerSize(containerSize); + } + + getStreamObserver().onNext(responseBuilder.build()); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java index 240ba9473d3d..dab2c562df5d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java @@ -21,6 +21,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.List; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; @@ -71,18 +72,35 @@ public void replicate(ReplicationTask task) { LOG.info("Starting replication of container {} from {} using {}", containerID, sourceDatanodes, compression); HddsVolume targetVolume = null; + long spaceReserved = containerImporter.getDefaultReplicationSpace(); try { targetVolume = containerImporter.chooseNextVolume(); // Wait for the download. This thread pool is limiting the parallel // downloads, so it's ok to block here and wait for the full download. - Path tarFilePath = + Pair downloadResult = downloader.getContainerDataFromReplicas(containerID, sourceDatanodes, ContainerImporter.getUntarDirectory(targetVolume), compression); - if (tarFilePath == null) { + if (downloadResult == null || downloadResult.getLeft() == null) { task.setStatus(Status.FAILED); return; } + + Path tarFilePath = downloadResult.getLeft(); + Long actualContainerSize = downloadResult.getRight(); + + if (actualContainerSize != null) { + long actualSpaceNeeded = containerImporter.getRequiredReplicationSpace(actualContainerSize); + long spaceAdjustment = actualSpaceNeeded - spaceReserved; + + if (spaceAdjustment > 0) { + targetVolume.incCommittedBytes(spaceAdjustment); + spaceReserved = actualSpaceNeeded; + LOG.info("Container {} space adjusted by {} bytes (actual size: {} bytes)", + containerID, spaceAdjustment, actualContainerSize); + } + } + long bytes = Files.size(tarFilePath); LOG.info("Container {} is downloaded with size {}, starting to import.", containerID, bytes); @@ -98,7 +116,7 @@ public void replicate(ReplicationTask task) { task.setStatus(Status.FAILED); } finally { if (targetVolume != null) { - targetVolume.incCommittedBytes(-containerImporter.getDefaultReplicationSpace()); + targetVolume.incCommittedBytes(-spaceReserved); } } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java index bf381e3715be..34e8710a0b97 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java @@ -54,7 +54,7 @@ public GrpcContainerUploader( @Override public OutputStream startUpload(long containerId, DatanodeDetails target, - CompletableFuture callback, CopyContainerCompression compression) + CompletableFuture callback, CopyContainerCompression compression, Long containerSize) throws IOException { GrpcReplicationClient client = createReplicationClient(target, compression); try { @@ -68,7 +68,7 @@ public OutputStream startUpload(long containerId, DatanodeDetails target, (CallStreamObserver) client.upload( responseObserver), responseObserver); return new SendContainerOutputStream(requestStream, containerId, - GrpcReplicationService.BUFFER_SIZE, compression) { + GrpcReplicationService.BUFFER_SIZE, compression, containerSize) { @Override public void close() throws IOException { try { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java index 7b9b24071f93..6ccb6bcf1f9a 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java @@ -26,6 +26,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.CopyContainerRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.CopyContainerResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerRequest; @@ -97,7 +98,7 @@ public GrpcReplicationClient( LOG.debug("{}: created", this); } - public CompletableFuture download(long containerId, Path dir) { + public CompletableFuture> download(long containerId, Path dir) { CopyContainerRequestProto request = CopyContainerRequestProto.newBuilder() .setContainerID(containerId) @@ -106,7 +107,7 @@ public CompletableFuture download(long containerId, Path dir) { .setCompression(compression.toProto()) .build(); - CompletableFuture response = new CompletableFuture<>(); + CompletableFuture> response = new CompletableFuture<>(); Path destinationPath = dir .resolve(ContainerUtils.getContainerTarName(containerId)); @@ -151,12 +152,14 @@ public String toString() { public static class StreamDownloader implements StreamObserver { - private final CompletableFuture response; + private final CompletableFuture> response; private final long containerId; private final OutputStream stream; + private Long actualContainerSize = null; private final Path outputPath; + private boolean firstChunk = true; - public StreamDownloader(long containerId, CompletableFuture response, + public StreamDownloader(long containerId, CompletableFuture> response, Path outputPath) { this.response = response; this.containerId = containerId; @@ -175,6 +178,18 @@ public StreamDownloader(long containerId, CompletableFuture response, @Override public void onNext(CopyContainerResponseProto chunk) { try { + // Capture container size from first chunk + if (firstChunk) { + firstChunk = false; + if (chunk.hasActualContainerSize()) { + actualContainerSize = chunk.getActualContainerSize(); + LOG.info("Container {} actual size received: {} bytes", + containerId, actualContainerSize); + } else { + LOG.info("Container {} actual size not provided, will use default", containerId); + } + } + chunk.getData().writeTo(stream); } catch (IOException e) { LOG.error("Failed to write the stream buffer to {} for container {}", @@ -211,7 +226,7 @@ public void onCompleted() { try { stream.close(); LOG.info("Container {} is downloaded to {}", containerId, outputPath); - response.complete(outputPath); + response.complete(Pair.of(outputPath, actualContainerSize)); } catch (IOException e) { LOG.error("Downloaded container {} OK, but failed to close {}", containerId, outputPath, e); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java index 10cba29845f3..9d640ab77a67 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java @@ -126,11 +126,19 @@ public void download(CopyContainerRequestProto request, "with compression {}", containerID, compression); OutputStream outputStream = null; try { + // Get container size before creating stream + Long containerSize = null; + try { + containerSize = source.getContainerSize(containerID); + } catch (Exception e) { + LOG.warn("Could not get container size for {}, proceeding without size info", containerID, e); + } + outputStream = new CopyContainerResponseStream( // gRPC runtime always provides implementation of CallStreamObserver // that allows flow control. (CallStreamObserver) responseObserver, - containerID, BUFFER_SIZE); + containerID, BUFFER_SIZE, containerSize); source.copyData(containerID, outputStream, compression); } catch (IOException e) { LOG.warn("Error streaming container {}", containerID, e); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/OnDemandContainerReplicationSource.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/OnDemandContainerReplicationSource.java index 422dd370d1fd..0ab637582a5b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/OnDemandContainerReplicationSource.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/OnDemandContainerReplicationSource.java @@ -61,4 +61,14 @@ public void copyData(long containerId, OutputStream destination, container.getContainerType(), containerId, destination, new TarContainerPacker(compression)); } + + @Override + public Long getContainerSize(long containerId) throws IOException { + Container container = controller.getContainer(containerId); + if (container == null) { + throw new StorageContainerException("Container " + containerId + + " is not found.", CONTAINER_NOT_FOUND); + } + return container.getContainerData().getBytesUsed(); + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java index 759aff722baf..5c17d667e1ac 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java @@ -56,12 +56,21 @@ public void replicate(ReplicationTask task) { LOG.info("Starting replication of container {} to {} using {}", containerID, target, compression); + // Get container size before starting upload (same as pull replication) + Long containerSize = null; + try { + containerSize = source.getContainerSize(containerID); + LOG.info("Container {} actual size: {} bytes", containerID, containerSize); + } catch (Exception e) { + LOG.warn("Could not get container size for {}, proceeding without size info", containerID, e); + } + source.prepare(containerID); CountingOutputStream output = null; try { output = new CountingOutputStream( - uploader.startUpload(containerID, target, fut, compression)); + uploader.startUpload(containerID, target, fut, compression, containerSize)); source.copyData(containerID, output, compression); fut.get(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java index 5824e2d9dfa8..ced206bce7a3 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java @@ -27,22 +27,28 @@ class SendContainerOutputStream extends GrpcOutputStream { private final CopyContainerCompression compression; + private final Long containerSize; SendContainerOutputStream( CallStreamObserver streamObserver, - long containerId, int bufferSize, CopyContainerCompression compression) { + long containerId, int bufferSize, CopyContainerCompression compression, Long containerSize) { super(streamObserver, containerId, bufferSize); this.compression = compression; + this.containerSize = containerSize; } @Override protected void sendPart(boolean eof, int length, ByteString data) { - SendContainerRequest request = SendContainerRequest.newBuilder() + SendContainerRequest.Builder requestBuilder = SendContainerRequest.newBuilder() .setContainerID(getContainerId()) .setData(data) .setOffset(getWrittenBytes()) - .setCompression(compression.toProto()) - .build(); - getStreamObserver().onNext(request); + .setCompression(compression.toProto()); + + // Include container size in first request + if (getWrittenBytes() == 0 && containerSize != null) { + requestBuilder.setActualContainerSize(containerSize); + } + getStreamObserver().onNext(requestBuilder.build()); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java index 9cb07a21c5dc..a8e0e66a5e98 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java @@ -54,6 +54,9 @@ class SendContainerRequestHandler private Path path; private CopyContainerCompression compression; private final ZeroCopyMessageMarshaller marshaller; + private Long actualContainerSize = null; + private long spaceReserved = 0; + private boolean firstRequest = true; SendContainerRequestHandler( ContainerImporter importer, @@ -84,6 +87,8 @@ public void onNext(SendContainerRequest req) { if (containerId == -1) { containerId = req.getContainerID(); + spaceReserved = importer.getDefaultReplicationSpace(); + volume = importer.chooseNextVolume(); Path dir = ContainerImporter.getUntarDirectory(volume); @@ -95,6 +100,22 @@ public void onNext(SendContainerRequest req) { LOG.info("Accepting container {}", req.getContainerID()); } + if (firstRequest) { + firstRequest = false; + if (req.hasActualContainerSize()) { + actualContainerSize = req.getActualContainerSize(); + long actualSpaceNeeded = importer.getRequiredReplicationSpace(actualContainerSize); + long spaceAdjustment = actualSpaceNeeded - spaceReserved; + + if (spaceAdjustment > 0) { + volume.incCommittedBytes(spaceAdjustment); + spaceReserved = actualSpaceNeeded; + LOG.info("Container {} space adjusted by {} bytes (actual size: {} bytes)", + containerId, spaceAdjustment, actualContainerSize); + } + } + } + assertSame(containerId, req.getContainerID(), "containerID"); req.getData().writeTo(output); @@ -117,8 +138,8 @@ public void onError(Throwable t) { deleteTarball(); responseObserver.onError(t); } finally { - if (volume != null) { - volume.incCommittedBytes(-importer.getDefaultReplicationSpace()); + if (volume != null && spaceReserved > 0) { + volume.incCommittedBytes(-spaceReserved); } } } @@ -146,8 +167,8 @@ public void onCompleted() { responseObserver.onError(t); } } finally { - if (volume != null) { - volume.incCommittedBytes(-importer.getDefaultReplicationSpace()); + if (volume != null && spaceReserved > 0) { + volume.incCommittedBytes(-spaceReserved); } } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java index 145d63680c23..6bf5423ce0f9 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name; @@ -56,7 +57,7 @@ public SimpleContainerDownloader( } @Override - public Path getContainerDataFromReplicas( + public Pair getContainerDataFromReplicas( long containerId, List sourceDatanodes, Path downloadDir, CopyContainerCompression compression) { @@ -73,7 +74,7 @@ public Path getContainerDataFromReplicas( GrpcReplicationClient client = null; try { client = createReplicationClient(datanode, compression); - CompletableFuture result = + CompletableFuture> result = downloadContainer(client, containerId, downloadDir); return result.get(); } catch (InterruptedException e) { @@ -127,7 +128,7 @@ protected GrpcReplicationClient createReplicationClient( } @VisibleForTesting - protected CompletableFuture downloadContainer( + protected CompletableFuture> downloadContainer( GrpcReplicationClient client, long containerId, Path downloadDir) { return client.download(containerId, downloadDir); } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcContainerUploader.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcContainerUploader.java index b8df5c18e8c9..490f15ccf44f 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcContainerUploader.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcContainerUploader.java @@ -129,7 +129,7 @@ protected GrpcReplicationClient createReplicationClient( private static OutputStream startUpload(GrpcContainerUploader subject, CompletableFuture callback) throws IOException { DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); - return subject.startUpload(1, target, callback, NO_COMPRESSION); + return subject.startUpload(1, target, callback, NO_COMPRESSION, null); } /** diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java index 8d0ed0401b71..22148db184c4 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java @@ -173,7 +173,7 @@ public void testDownload() throws IOException { Path result = downloader.getContainerDataFromReplicas( CONTAINER_ID, Collections.singletonList(datanode), downloadDir, - CopyContainerCompression.NO_COMPRESSION); + CopyContainerCompression.NO_COMPRESSION).getLeft(); assertTrue(result.toString().startsWith(downloadDir.toString())); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java index a4463410cea5..5259f4a20691 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java @@ -140,7 +140,7 @@ private ContainerReplicator createSubject( when( uploader.startUpload(eq(containerID), eq(target), - futureArgument.capture(), compressionArgument.capture() + futureArgument.capture(), compressionArgument.capture(), any(Long.class) )) .thenReturn(outputStream); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java index 8dee27e488b1..e9c0328930cc 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java @@ -73,6 +73,7 @@ import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -337,7 +338,7 @@ public void testDownloadAndImportReplicatorFailure(ContainerLayoutVersion layout when( moc.getContainerDataFromReplicas(anyLong(), anyList(), any(Path.class), any())) - .thenReturn(res); + .thenReturn(Pair.of(res, null)); final String testDir = tempFile.getPath(); MutableVolumeSet volumeSet = mock(MutableVolumeSet.class); @@ -407,7 +408,7 @@ public void testReplicationImportReserveSpace(ContainerLayoutVersion layout) when( moc.getContainerDataFromReplicas(anyLong(), anyList(), any(Path.class), any())) - .thenReturn(tarFile.toPath()); + .thenReturn(Pair.of(tarFile.toPath(), null)); ContainerImporter importer = new ContainerImporter(conf, set, controllerMock, volumeSet, volumeChoosingPolicy); @@ -582,7 +583,7 @@ public void testMultipleReplication(ContainerLayoutVersion layout, SimpleContainerDownloader moc = mock(SimpleContainerDownloader.class); Path res = Paths.get("file:/tmp/no-such-file"); when(moc.getContainerDataFromReplicas(anyLong(), anyList(), - any(Path.class), any())).thenReturn(res); + any(Path.class), any())).thenReturn(Pair.of(res, null)); final String testDir = tempFile.getPath(); MutableVolumeSet volumeSet = mock(MutableVolumeSet.class); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerOutputStream.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerOutputStream.java index c688b6495103..716bf4d3aebc 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerOutputStream.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerOutputStream.java @@ -40,14 +40,14 @@ class TestSendContainerOutputStream @Override protected OutputStream createSubject() { return new SendContainerOutputStream(getObserver(), - getContainerId(), getBufferSize(), NO_COMPRESSION); + getContainerId(), getBufferSize(), NO_COMPRESSION, null); } @ParameterizedTest @EnumSource void usesCompression(CopyContainerCompression compression) throws Exception { OutputStream subject = new SendContainerOutputStream( - getObserver(), getContainerId(), getBufferSize(), compression); + getObserver(), getContainerId(), getBufferSize(), compression, null); byte[] bytes = getRandomBytes(16); subject.write(bytes, 0, bytes.length); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java index 076fb17c711b..e94e4e1a4e59 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; @@ -56,7 +57,7 @@ public void testGetContainerDataFromReplicasHappyPath() throws Exception { //WHEN Path result = downloader.getContainerDataFromReplicas(1L, datanodes, - tempDir, NO_COMPRESSION); + tempDir, NO_COMPRESSION).getLeft(); //THEN assertEquals(datanodes.get(0).getUuidString(), @@ -77,7 +78,7 @@ public void testGetContainerDataFromReplicasDirectFailure() //WHEN final Path result = downloader.getContainerDataFromReplicas(1L, datanodes, - tempDir, NO_COMPRESSION); + tempDir, NO_COMPRESSION).getLeft(); //THEN //first datanode is failed, second worked @@ -98,7 +99,7 @@ public void testGetContainerDataFromReplicasAsyncFailure() throws Exception { //WHEN final Path result = downloader.getContainerDataFromReplicas(1L, datanodes, - tempDir, NO_COMPRESSION); + tempDir, NO_COMPRESSION).getLeft(); //THEN //first datanode is failed, second worked @@ -123,7 +124,7 @@ public void testRandomSelection() throws Exception { //returned. for (int i = 0; i < 10000; i++) { Path path = downloader.getContainerDataFromReplicas(1L, datanodes, - tempDir, NO_COMPRESSION); + tempDir, NO_COMPRESSION).getLeft(); if (path.toString().equals(datanodes.get(1).getUuidString())) { return; } @@ -206,7 +207,7 @@ protected GrpcReplicationClient createReplicationClient( } @Override - protected CompletableFuture downloadContainer( + protected CompletableFuture> downloadContainer( GrpcReplicationClient client, long containerId, Path downloadPath) { @@ -225,7 +226,7 @@ protected CompletableFuture downloadContainer( //path includes the dn id to make it possible to assert. return CompletableFuture.completedFuture( - Paths.get(datanode.getUuidString())); + Pair.of(Paths.get(datanode.getUuidString()), null)); } } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index bdba99cffd08..225773ca9878 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -541,6 +541,7 @@ message CopyContainerResponseProto { required bool eof = 4; required bytes data = 5; optional int64 checksum = 6; + optional uint64 actualContainerSize = 7; } message SendContainerRequest { @@ -549,6 +550,7 @@ message SendContainerRequest { required bytes data = 3; optional int64 checksum = 4; optional CopyContainerCompressProto compression = 5; + optional int64 actualContainerSize = 6; } message SendContainerResponse { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java index af42ce3b7527..8429146f691f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java @@ -59,6 +59,7 @@ import java.util.Date; import java.util.List; import java.util.UUID; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -402,10 +403,10 @@ private void assertDownloadContainerFails(long containerId, LogCapturer logCapture = captureLogs(SimpleContainerDownloader.class); SimpleContainerDownloader downloader = new SimpleContainerDownloader(conf, caClient); - Path file = downloader.getContainerDataFromReplicas(containerId, + Pair result = downloader.getContainerDataFromReplicas(containerId, sourceDatanodes, tempFolder.resolve("tmp"), NO_COMPRESSION); downloader.close(); - assertNull(file); + assertNull(result); assertThat(logCapture.getOutput()) .contains("java.security.cert.CertificateExpiredException"); } @@ -416,7 +417,7 @@ private void assertDownloadContainerWorks(List containers, SimpleContainerDownloader downloader = new SimpleContainerDownloader(conf, caClient); Path file = downloader.getContainerDataFromReplicas(cId, sourceDatanodes, - tempFolder.resolve("tmp"), NO_COMPRESSION); + tempFolder.resolve("tmp"), NO_COMPRESSION).getLeft(); downloader.close(); assertNotNull(file); } From aa63fc901626b5a1a8cb4bfe91ee229aba8b7e76 Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Wed, 3 Sep 2025 13:05:05 +0530 Subject: [PATCH 02/10] use replicateContainerCommandProto to send replicateSize and updated unit test --- .../replication/ContainerDownloader.java | 4 +- .../replication/ContainerImporter.java | 4 +- .../ContainerReplicationSource.java | 13 -- .../replication/ContainerUploader.java | 4 +- .../CopyContainerResponseStream.java | 22 +-- .../DownloadAndImportReplicator.java | 34 ++-- .../replication/GrpcContainerUploader.java | 6 +- .../replication/GrpcReplicationClient.java | 25 +-- .../replication/GrpcReplicationService.java | 10 +- .../OnDemandContainerReplicationSource.java | 10 -- .../container/replication/PushReplicator.java | 12 +- .../replication/ReplicationTask.java | 4 + .../SendContainerOutputStream.java | 15 +- .../SendContainerRequestHandler.java | 29 ++-- .../SimpleContainerDownloader.java | 7 +- .../commands/ReplicateContainerCommand.java | 26 +++ .../TestDownloadAndImportReplicator.java | 95 ++++++++++- .../TestGrpcReplicationService.java | 4 +- .../replication/TestPushReplicator.java | 109 +++++++++++-- .../TestReplicationSupervisor.java | 7 +- .../TestSendContainerRequestHandler.java | 149 ++++++++++++------ .../TestSimpleContainerDownloader.java | 13 +- .../main/proto/DatanodeClientProtocol.proto | 3 +- .../ScmServerDatanodeHeartbeatProtocol.proto | 1 + .../replication/ECMisReplicationHandler.java | 1 + .../ECUnderReplicationHandler.java | 1 + .../RatisMisReplicationHandler.java | 1 + .../RatisUnderReplicationHandler.java | 1 + .../replication/ReplicationManager.java | 1 + .../ozoneimpl/TestOzoneContainerWithTLS.java | 7 +- 30 files changed, 392 insertions(+), 226 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerDownloader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerDownloader.java index 8063a7187de5..28879ffde18d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerDownloader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerDownloader.java @@ -20,7 +20,6 @@ import java.io.Closeable; import java.nio.file.Path; import java.util.List; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.protocol.DatanodeDetails; /** @@ -33,7 +32,8 @@ */ public interface ContainerDownloader extends Closeable { - Pair getContainerDataFromReplicas(long containerId, + Path getContainerDataFromReplicas(long containerId, List sources, Path downloadDir, CopyContainerCompression compression); + } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java index f72f8b92ea00..879ffe563994 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java @@ -146,11 +146,11 @@ private static void deleteFileQuietely(Path tarFilePath) { } } - HddsVolume chooseNextVolume() throws IOException { + HddsVolume chooseNextVolume(long spaceReserved) throws IOException { // Choose volume that can hold both container in tmp and dest directory return volumeChoosingPolicy.chooseVolume( StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()), - getDefaultReplicationSpace()); + spaceReserved); } public static Path getUntarDirectory(HddsVolume hddsVolume) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerReplicationSource.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerReplicationSource.java index 65cb9b1f49d0..ad0e1cddde6d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerReplicationSource.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerReplicationSource.java @@ -48,17 +48,4 @@ void copyData(long containerId, OutputStream destination, CopyContainerCompression compression) throws IOException; - /** - * Get the actual size of the container in bytes. - * This is used for space reservation during replication. - * - * @param containerId ID of the container - * @return actual container size in bytes, or null if size cannot be determined - * @throws IOException if container is not found or size cannot be retrieved - */ - default Long getContainerSize(long containerId) throws IOException { - // Default implementation returns null for backward compatibility - return null; - } - } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java index b5c1ea17c434..420bb034b58e 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java @@ -27,6 +27,6 @@ */ public interface ContainerUploader { OutputStream startUpload(long containerId, DatanodeDetails target, - CompletableFuture callback, CopyContainerCompression compression, Long containerSize) - throws IOException; + CompletableFuture callback, CopyContainerCompression compression, + Long replicateSize) throws IOException; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/CopyContainerResponseStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/CopyContainerResponseStream.java index 6b078f7a6447..61cecf1255b1 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/CopyContainerResponseStream.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/CopyContainerResponseStream.java @@ -27,36 +27,22 @@ class CopyContainerResponseStream extends GrpcOutputStream { - private final Long containerSize; - CopyContainerResponseStream( CallStreamObserver streamObserver, long containerId, int bufferSize) { - this(streamObserver, containerId, bufferSize, null); - } - - CopyContainerResponseStream( - CallStreamObserver streamObserver, - long containerId, int bufferSize, Long containerSize) { super(streamObserver, containerId, bufferSize); - this.containerSize = containerSize; } @Override protected void sendPart(boolean eof, int length, ByteString data) { - CopyContainerResponseProto.Builder responseBuilder = + CopyContainerResponseProto response = CopyContainerResponseProto.newBuilder() .setContainerID(getContainerId()) .setData(data) .setEof(eof) .setReadOffset(getWrittenBytes()) - .setLen(length); - - // Include container size in first response - if (getWrittenBytes() == 0 && containerSize != null) { - responseBuilder.setActualContainerSize(containerSize); - } - - getStreamObserver().onNext(responseBuilder.build()); + .setLen(length) + .build(); + getStreamObserver().onNext(response); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java index dab2c562df5d..7950567ac886 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java @@ -21,7 +21,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.List; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; @@ -72,35 +71,30 @@ public void replicate(ReplicationTask task) { LOG.info("Starting replication of container {} from {} using {}", containerID, sourceDatanodes, compression); HddsVolume targetVolume = null; - long spaceReserved = containerImporter.getDefaultReplicationSpace(); + + // Use replicate size from command if available, otherwise use default + long spaceReserved = 0; + Long replicateSize = task.getReplicateSize(); + if (replicateSize != null) { + spaceReserved = containerImporter.getRequiredReplicationSpace(replicateSize); + } else { + // Fallback to default (backward compatibility) + spaceReserved = containerImporter.getDefaultReplicationSpace(); + } try { - targetVolume = containerImporter.chooseNextVolume(); + targetVolume = containerImporter.chooseNextVolume(spaceReserved); + // Wait for the download. This thread pool is limiting the parallel // downloads, so it's ok to block here and wait for the full download. - Pair downloadResult = + Path tarFilePath = downloader.getContainerDataFromReplicas(containerID, sourceDatanodes, ContainerImporter.getUntarDirectory(targetVolume), compression); - if (downloadResult == null || downloadResult.getLeft() == null) { + if (tarFilePath == null) { task.setStatus(Status.FAILED); return; } - Path tarFilePath = downloadResult.getLeft(); - Long actualContainerSize = downloadResult.getRight(); - - if (actualContainerSize != null) { - long actualSpaceNeeded = containerImporter.getRequiredReplicationSpace(actualContainerSize); - long spaceAdjustment = actualSpaceNeeded - spaceReserved; - - if (spaceAdjustment > 0) { - targetVolume.incCommittedBytes(spaceAdjustment); - spaceReserved = actualSpaceNeeded; - LOG.info("Container {} space adjusted by {} bytes (actual size: {} bytes)", - containerID, spaceAdjustment, actualContainerSize); - } - } - long bytes = Files.size(tarFilePath); LOG.info("Container {} is downloaded with size {}, starting to import.", containerID, bytes); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java index 34e8710a0b97..6923e8786263 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java @@ -54,8 +54,8 @@ public GrpcContainerUploader( @Override public OutputStream startUpload(long containerId, DatanodeDetails target, - CompletableFuture callback, CopyContainerCompression compression, Long containerSize) - throws IOException { + CompletableFuture callback, CopyContainerCompression compression, + Long replicateSize) throws IOException { GrpcReplicationClient client = createReplicationClient(target, compression); try { // gRPC runtime always provides implementation of CallStreamObserver @@ -68,7 +68,7 @@ public OutputStream startUpload(long containerId, DatanodeDetails target, (CallStreamObserver) client.upload( responseObserver), responseObserver); return new SendContainerOutputStream(requestStream, containerId, - GrpcReplicationService.BUFFER_SIZE, compression, containerSize) { + GrpcReplicationService.BUFFER_SIZE, compression, replicateSize) { @Override public void close() throws IOException { try { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java index 6ccb6bcf1f9a..7b9b24071f93 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java @@ -26,7 +26,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.CopyContainerRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.CopyContainerResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerRequest; @@ -98,7 +97,7 @@ public GrpcReplicationClient( LOG.debug("{}: created", this); } - public CompletableFuture> download(long containerId, Path dir) { + public CompletableFuture download(long containerId, Path dir) { CopyContainerRequestProto request = CopyContainerRequestProto.newBuilder() .setContainerID(containerId) @@ -107,7 +106,7 @@ public CompletableFuture> download(long containerId, Path dir) .setCompression(compression.toProto()) .build(); - CompletableFuture> response = new CompletableFuture<>(); + CompletableFuture response = new CompletableFuture<>(); Path destinationPath = dir .resolve(ContainerUtils.getContainerTarName(containerId)); @@ -152,14 +151,12 @@ public String toString() { public static class StreamDownloader implements StreamObserver { - private final CompletableFuture> response; + private final CompletableFuture response; private final long containerId; private final OutputStream stream; - private Long actualContainerSize = null; private final Path outputPath; - private boolean firstChunk = true; - public StreamDownloader(long containerId, CompletableFuture> response, + public StreamDownloader(long containerId, CompletableFuture response, Path outputPath) { this.response = response; this.containerId = containerId; @@ -178,18 +175,6 @@ public StreamDownloader(long containerId, CompletableFuture> re @Override public void onNext(CopyContainerResponseProto chunk) { try { - // Capture container size from first chunk - if (firstChunk) { - firstChunk = false; - if (chunk.hasActualContainerSize()) { - actualContainerSize = chunk.getActualContainerSize(); - LOG.info("Container {} actual size received: {} bytes", - containerId, actualContainerSize); - } else { - LOG.info("Container {} actual size not provided, will use default", containerId); - } - } - chunk.getData().writeTo(stream); } catch (IOException e) { LOG.error("Failed to write the stream buffer to {} for container {}", @@ -226,7 +211,7 @@ public void onCompleted() { try { stream.close(); LOG.info("Container {} is downloaded to {}", containerId, outputPath); - response.complete(Pair.of(outputPath, actualContainerSize)); + response.complete(outputPath); } catch (IOException e) { LOG.error("Downloaded container {} OK, but failed to close {}", containerId, outputPath, e); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java index 9d640ab77a67..10cba29845f3 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationService.java @@ -126,19 +126,11 @@ public void download(CopyContainerRequestProto request, "with compression {}", containerID, compression); OutputStream outputStream = null; try { - // Get container size before creating stream - Long containerSize = null; - try { - containerSize = source.getContainerSize(containerID); - } catch (Exception e) { - LOG.warn("Could not get container size for {}, proceeding without size info", containerID, e); - } - outputStream = new CopyContainerResponseStream( // gRPC runtime always provides implementation of CallStreamObserver // that allows flow control. (CallStreamObserver) responseObserver, - containerID, BUFFER_SIZE, containerSize); + containerID, BUFFER_SIZE); source.copyData(containerID, outputStream, compression); } catch (IOException e) { LOG.warn("Error streaming container {}", containerID, e); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/OnDemandContainerReplicationSource.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/OnDemandContainerReplicationSource.java index 0ab637582a5b..422dd370d1fd 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/OnDemandContainerReplicationSource.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/OnDemandContainerReplicationSource.java @@ -61,14 +61,4 @@ public void copyData(long containerId, OutputStream destination, container.getContainerType(), containerId, destination, new TarContainerPacker(compression)); } - - @Override - public Long getContainerSize(long containerId) throws IOException { - Container container = controller.getContainer(containerId); - if (container == null) { - throw new StorageContainerException("Container " + containerId + - " is not found.", CONTAINER_NOT_FOUND); - } - return container.getContainerData().getBytesUsed(); - } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java index 5c17d667e1ac..35a076cd749f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java @@ -56,21 +56,13 @@ public void replicate(ReplicationTask task) { LOG.info("Starting replication of container {} to {} using {}", containerID, target, compression); - // Get container size before starting upload (same as pull replication) - Long containerSize = null; - try { - containerSize = source.getContainerSize(containerID); - LOG.info("Container {} actual size: {} bytes", containerID, containerSize); - } catch (Exception e) { - LOG.warn("Could not get container size for {}, proceeding without size info", containerID, e); - } - source.prepare(containerID); CountingOutputStream output = null; try { + Long replicateSize = task.getReplicateSize(); output = new CountingOutputStream( - uploader.startUpload(containerID, target, fut, compression, containerSize)); + uploader.startUpload(containerID, target, fut, compression, replicateSize)); source.copyData(containerID, output, compression); fut.get(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java index f61e219678ec..efdbfd5f02ec 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java @@ -101,6 +101,10 @@ public List getSources() { return cmd.getSourceDatanodes(); } + public Long getReplicateSize() { + return cmd.getReplicateSize(); + } + @Override protected Object getCommandForDebug() { return debugString; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java index ced206bce7a3..408efad5bd4d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java @@ -27,14 +27,15 @@ class SendContainerOutputStream extends GrpcOutputStream { private final CopyContainerCompression compression; - private final Long containerSize; + private final Long replicateSize; SendContainerOutputStream( CallStreamObserver streamObserver, - long containerId, int bufferSize, CopyContainerCompression compression, Long containerSize) { + long containerId, int bufferSize, CopyContainerCompression compression, + Long replicateSize) { super(streamObserver, containerId, bufferSize); this.compression = compression; - this.containerSize = containerSize; + this.replicateSize = replicateSize; } @Override @@ -44,10 +45,10 @@ protected void sendPart(boolean eof, int length, ByteString data) { .setData(data) .setOffset(getWrittenBytes()) .setCompression(compression.toProto()); - - // Include container size in first request - if (getWrittenBytes() == 0 && containerSize != null) { - requestBuilder.setActualContainerSize(containerSize); + + // Include replicate size in the first request + if (getWrittenBytes() == 0 && replicateSize != null) { + requestBuilder.setReplicateSize(replicateSize); } getStreamObserver().onNext(requestBuilder.build()); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java index a8e0e66a5e98..d4d62db5b2b7 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java @@ -54,7 +54,6 @@ class SendContainerRequestHandler private Path path; private CopyContainerCompression compression; private final ZeroCopyMessageMarshaller marshaller; - private Long actualContainerSize = null; private long spaceReserved = 0; private boolean firstRequest = true; @@ -87,9 +86,15 @@ public void onNext(SendContainerRequest req) { if (containerId == -1) { containerId = req.getContainerID(); - spaceReserved = importer.getDefaultReplicationSpace(); + + // Use replicate size if available, otherwise fall back to default + if (firstRequest && req.hasReplicateSize()) { + spaceReserved = importer.getRequiredReplicationSpace(req.getReplicateSize()); + } else { + spaceReserved = importer.getDefaultReplicationSpace(); + } - volume = importer.chooseNextVolume(); + volume = importer.chooseNextVolume(spaceReserved); Path dir = ContainerImporter.getUntarDirectory(volume); Files.createDirectories(dir); @@ -99,22 +104,8 @@ public void onNext(SendContainerRequest req) { LOG.info("Accepting container {}", req.getContainerID()); } - - if (firstRequest) { - firstRequest = false; - if (req.hasActualContainerSize()) { - actualContainerSize = req.getActualContainerSize(); - long actualSpaceNeeded = importer.getRequiredReplicationSpace(actualContainerSize); - long spaceAdjustment = actualSpaceNeeded - spaceReserved; - - if (spaceAdjustment > 0) { - volume.incCommittedBytes(spaceAdjustment); - spaceReserved = actualSpaceNeeded; - LOG.info("Container {} space adjusted by {} bytes (actual size: {} bytes)", - containerId, spaceAdjustment, actualContainerSize); - } - } - } + + firstRequest = false; assertSame(containerId, req.getContainerID(), "containerID"); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java index 6bf5423ce0f9..145d63680c23 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java @@ -25,7 +25,6 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name; @@ -57,7 +56,7 @@ public SimpleContainerDownloader( } @Override - public Pair getContainerDataFromReplicas( + public Path getContainerDataFromReplicas( long containerId, List sourceDatanodes, Path downloadDir, CopyContainerCompression compression) { @@ -74,7 +73,7 @@ public Pair getContainerDataFromReplicas( GrpcReplicationClient client = null; try { client = createReplicationClient(datanode, compression); - CompletableFuture> result = + CompletableFuture result = downloadContainer(client, containerId, downloadDir); return result.get(); } catch (InterruptedException e) { @@ -128,7 +127,7 @@ protected GrpcReplicationClient createReplicationClient( } @VisibleForTesting - protected CompletableFuture> downloadContainer( + protected CompletableFuture downloadContainer( GrpcReplicationClient client, long containerId, Path downloadDir) { return client.download(containerId, downloadDir); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java index 826af4fdd362..bf633aa33f2c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java @@ -43,6 +43,9 @@ public final class ReplicateContainerCommand private ReplicationCommandPriority priority = ReplicationCommandPriority.NORMAL; + // Actual container size in bytes + private Long replicateSize; + public static ReplicateContainerCommand fromSources(long containerID, List sourceDatanodes) { return new ReplicateContainerCommand(containerID, sourceDatanodes, null); @@ -82,6 +85,14 @@ public void setPriority(ReplicationCommandPriority priority) { this.priority = priority; } + public void setReplicateSize(Long replicateSize) { + this.replicateSize = replicateSize; + } + + public Long getReplicateSize() { + return replicateSize; + } + @Override public Type getType() { return SCMCommandProto.Type.replicateContainerCommand; @@ -105,6 +116,12 @@ public ReplicateContainerCommandProto getProto() { builder.setTarget(targetDatanode.getProtoBufMessage()); } builder.setPriority(priority); + + // Add replicate size if available + if (replicateSize != null) { + builder.setReplicateSize(replicateSize); + } + return builder.build(); } @@ -131,6 +148,12 @@ public static ReplicateContainerCommand getFromProtobuf( if (protoMessage.hasPriority()) { cmd.setPriority(protoMessage.getPriority()); } + + // Add replicate size handling for backward compatibility + if (protoMessage.hasReplicateSize()) { + cmd.setReplicateSize(protoMessage.getReplicateSize()); + } + return cmd; } @@ -164,6 +187,9 @@ public String toString() { .append(", deadlineMsSinceEpoch: ").append(getDeadline()); sb.append(", containerId=").append(getContainerID()); sb.append(", replicaIndex=").append(getReplicaIndex()); + if (replicateSize != null) { + sb.append(", replicateSize=").append(replicateSize); + } if (targetDatanode != null) { sb.append(", targetNode=").append(targetDatanode); } else { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java index c690b50d6425..b48ba9d9a570 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java @@ -22,17 +22,22 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; +import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy; @@ -41,11 +46,14 @@ import org.apache.hadoop.ozone.container.common.volume.StorageVolume; import org.apache.hadoop.ozone.container.common.volume.VolumeChoosingPolicyFactory; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; +import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; import org.apache.ozone.test.GenericTestUtils; 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.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; /** * Test for DownloadAndImportReplicator. @@ -71,19 +79,82 @@ void setup() throws IOException { StorageVolume.VolumeType.DATA_VOLUME, null); ContainerImporter importer = new ContainerImporter(conf, containerSet, mock(ContainerController.class), volumeSet, volumeChoosingPolicy); + importer = spy(importer); downloader = mock(SimpleContainerDownloader.class); replicator = new DownloadAndImportReplicator(conf, containerSet, importer, downloader); containerMaxSize = (long) conf.getStorageSize( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); + + // Mock the space reservation logic to be independent of the + // importer's implementation details. + doAnswer(invocation -> 2 * (long) invocation.getArgument(0)) + .when(importer).getRequiredReplicationSpace(anyLong()); + when(importer.getDefaultReplicationSpace()).thenReturn(2 * containerMaxSize); + + } + + /** + * Provides a stream of different container sizes for tests. + */ + static Stream replicateSizeProvider() { + return Stream.of( + Arguments.of("Null replicate size (fallback to default)", null), + Arguments.of("Normal 2GB", 2L * 1024L * 1024L * 1024L), + Arguments.of("Overallocated 20GB", 20L * 1024L * 1024L * 1024L) + ); + } + + @ParameterizedTest(name = "for {0}") + @MethodSource("replicateSizeProvider") + public void testSpaceReservedAndReleasedOnSuccess(String testName, Long replicateSize) + throws IOException { + // GIVEN + long containerId = 1; + HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); + long initialCommittedBytes = volume.getCommittedBytes(); + long expectedReservedSpace = replicateSize != null ? + importer.getRequiredReplicationSpace(replicateSize) : + importer.getDefaultReplicationSpace(); + + // Mock downloader to check reservation and return a dummy path + Path dummyPath = tempDir.toPath().resolve("dummy.tar"); + Files.createFile(dummyPath); + when(downloader.getContainerDataFromReplicas(anyLong(), any(), any(), any())) + .thenAnswer(invocation -> { + // Check that space was reserved before download attempt + assertEquals(initialCommittedBytes + expectedReservedSpace, + volume.getCommittedBytes()); + return dummyPath; + }); + // Mock the import itself to avoid file system operations + doAnswer(invocation -> null).when(importer) + .importContainer(anyLong(), any(), any(), any()); + + ReplicationTask task = createTask(containerId, replicateSize); + + // WHEN + // The replicator should reserve space, "download", "import", and then + // release the space in the finally block. + replicator.replicate(task); + + // THEN + assertEquals(AbstractReplicationTask.Status.DONE, task.getStatus()); + assertEquals(initialCommittedBytes, volume.getCommittedBytes(), + "Committed space should be released on success"); } - @Test - public void testCommitSpaceReleasedOnReplicationFailure() throws Exception { + @ParameterizedTest(name = "for {0}") + @MethodSource("replicateSizeProvider") + public void testCommitSpaceReleasedOnReplicationFailure(String testName, Long replicateSize) + throws Exception { long containerId = 1; HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); + long expectedReservedSpace = replicateSize != null ? + importer.getRequiredReplicationSpace(replicateSize) : + importer.getDefaultReplicationSpace(); // Mock downloader to throw exception Semaphore semaphore = new Semaphore(1); @@ -93,8 +164,8 @@ public void testCommitSpaceReleasedOnReplicationFailure() throws Exception { throw new IOException("Download failed"); }); - ReplicationTask task = new ReplicationTask(containerId, - Collections.singletonList(mock(DatanodeDetails.class)), replicator); + // Create task with the parameterized replicateSize + ReplicationTask task = createTask(containerId, replicateSize); // Acquire semaphore so that container import will pause before downloading. semaphore.acquire(); @@ -106,7 +177,7 @@ public void testCommitSpaceReleasedOnReplicationFailure() throws Exception { GenericTestUtils.waitFor(() -> volume.getCommittedBytes() > initialCommittedBytes, 1000, 50000); - assertEquals(volume.getCommittedBytes(), initialCommittedBytes + 2 * containerMaxSize); + assertEquals(volume.getCommittedBytes(), initialCommittedBytes + expectedReservedSpace); semaphore.release(); GenericTestUtils.waitFor(() -> @@ -116,4 +187,14 @@ public void testCommitSpaceReleasedOnReplicationFailure() throws Exception { // Verify commit space is released assertEquals(initialCommittedBytes, volume.getCommittedBytes()); } + + private ReplicationTask createTask(long containerId, Long replicateSize) { + ReplicateContainerCommand cmd = ReplicateContainerCommand.fromSources( + containerId, + Collections.singletonList(MockDatanodeDetails.randomDatanodeDetails())); + if (replicateSize != null) { + cmd.setReplicateSize(replicateSize); + } + return new ReplicationTask(cmd, replicator); + } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java index 22148db184c4..0172f123d9b5 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java @@ -150,7 +150,7 @@ public void init() throws Exception { }).when(importer).importContainer(anyLong(), any(), any(), any()); doReturn(true).when(importer).isAllowedContainerImport(eq( CONTAINER_ID)); - when(importer.chooseNextVolume()).thenReturn(new HddsVolume.Builder( + when(importer.chooseNextVolume(anyLong())).thenReturn(new HddsVolume.Builder( Files.createDirectory(tempDir.resolve("ImporterDir")).toString()).conf( conf).build()); @@ -173,7 +173,7 @@ public void testDownload() throws IOException { Path result = downloader.getContainerDataFromReplicas( CONTAINER_ID, Collections.singletonList(datanode), downloadDir, - CopyContainerCompression.NO_COMPRESSION).getLeft(); + CopyContainerCompression.NO_COMPRESSION); assertTrue(result.toString().startsWith(downloadDir.toString())); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java index 5259f4a20691..cda51deda20b 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java @@ -21,6 +21,7 @@ import static org.apache.hadoop.ozone.container.replication.CopyContainerCompression.NO_COMPRESSION; import static org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand.toTarget; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.AssertionsKt.assertNull; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.eq; @@ -29,18 +30,23 @@ import java.io.IOException; import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ThreadLocalRandom; import java.util.function.Consumer; +import java.util.stream.Stream; 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.ozone.container.replication.AbstractReplicationTask.Status; +import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; import org.apache.ozone.test.SpyOutputStream; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentCaptor; /** @@ -55,10 +61,35 @@ void setup() { conf = new OzoneConfiguration(); } - @ParameterizedTest - @EnumSource - void uploadCompletesNormally(CopyContainerCompression compression) - throws IOException { + /** + * Provides a stream of different container sizes for tests. + */ + static Stream replicateSize() { + return Stream.of( + Arguments.of("Normal 2GB", 2L * 1024L * 1024L * 1024L), + Arguments.of("Max Container Size 5GB", 5L * 1024L * 1024L * 1024L), + Arguments.of("Overallocated 20GB", 20L * 1024L * 1024L * 1024L) + ); + } + + /** + * Creates a Cartesian product of all compression types and container sizes. + */ + public static Stream compressionAndReplicateSize() { + List arguments = new ArrayList<>(); + for (CopyContainerCompression compression : CopyContainerCompression.values()) { + replicateSize().forEach(sizeArgs -> { + Object[] args = sizeArgs.get(); + arguments.add(Arguments.of(compression, args[0], args[1])); + }); + } + return arguments.stream(); + } + + @ParameterizedTest(name = "{1} with {0} compression") + @MethodSource("compressionAndReplicateSize") + void uploadCompletesNormallyWithReplicateSize( + CopyContainerCompression compression, String testName, Long replicateSize) throws IOException { // GIVEN compression.setOn(conf); long containerID = randomContainerID(); @@ -66,9 +97,12 @@ void uploadCompletesNormally(CopyContainerCompression compression) Consumer> completion = fut -> fut.complete(null); SpyOutputStream output = new SpyOutputStream(NULL_OUTPUT_STREAM); + ArgumentCaptor replicateSizeCaptor = ArgumentCaptor.forClass(Long.class); ContainerReplicator subject = createSubject(containerID, target, - output, completion, compression); - ReplicationTask task = new ReplicationTask(toTarget(containerID, target), + output, completion, compression, replicateSizeCaptor); + ReplicateContainerCommand cmd = toTarget(containerID, target); + cmd.setReplicateSize(replicateSize); + ReplicationTask task = new ReplicationTask(cmd, subject); // WHEN @@ -77,19 +111,58 @@ void uploadCompletesNormally(CopyContainerCompression compression) // THEN assertEquals(Status.DONE, task.getStatus()); output.assertClosedExactlyOnce(); + + // Verify the task also has the correct size + assertEquals(replicateSize, task.getReplicateSize()); } + /** + * Test that verifies backward compatibility - when replicateSize is null, + * the uploader still receives null (target will handle fallback to default). + */ @Test - void uploadFailsWithException() throws IOException { + void uploadCompletesWithNullReplicateSize() throws IOException { + // GIVEN + long containerID = randomContainerID(); + DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); + Consumer> completion = + fut -> fut.complete(null); + SpyOutputStream output = new SpyOutputStream(NULL_OUTPUT_STREAM); + ArgumentCaptor replicateSizeCaptor = ArgumentCaptor.forClass(Long.class); + ContainerReplicator subject = createSubject(containerID, target, + output, completion, NO_COMPRESSION, replicateSizeCaptor); + ReplicateContainerCommand cmd = toTarget(containerID, target); + // replicate size is not set - it is null + ReplicationTask task = new ReplicationTask(cmd, + subject); + + // WHEN + subject.replicate(task); + + // THEN + assertEquals(Status.DONE, task.getStatus()); + output.assertClosedExactlyOnce(); + + // The push replicator should pass null to uploader + assertNull(replicateSizeCaptor.getValue()); + assertNull(task.getReplicateSize()); + } + + @ParameterizedTest(name = "for {0}") + @MethodSource("replicateSize") + void uploadFailsWithException(String testName, long replicateSize) throws IOException { // GIVEN long containerID = randomContainerID(); DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); SpyOutputStream output = new SpyOutputStream(NULL_OUTPUT_STREAM); Consumer> completion = fut -> fut.completeExceptionally(new Exception("testing")); + ArgumentCaptor replicateSizeCaptor = ArgumentCaptor.forClass(Long.class); ContainerReplicator subject = createSubject(containerID, target, - output, completion, NO_COMPRESSION); - ReplicationTask task = new ReplicationTask(toTarget(containerID, target), + output, completion, NO_COMPRESSION, replicateSizeCaptor); + ReplicateContainerCommand cmd = toTarget(containerID, target); + cmd.setReplicateSize(replicateSize); + ReplicationTask task = new ReplicationTask(cmd, subject); // WHEN @@ -100,8 +173,9 @@ void uploadFailsWithException() throws IOException { output.assertClosedExactlyOnce(); } - @Test - void packFailsWithException() throws IOException { + @ParameterizedTest(name = "for {0}") + @MethodSource("replicateSize") + void packFailsWithException(String testName, long replicateSize) throws IOException { // GIVEN long containerID = randomContainerID(); DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); @@ -109,9 +183,12 @@ void packFailsWithException() throws IOException { Consumer> completion = fut -> { throw new RuntimeException(); }; + ArgumentCaptor replicateSizeCaptor = ArgumentCaptor.forClass(Long.class); ContainerReplicator subject = createSubject(containerID, target, - output, completion, NO_COMPRESSION); - ReplicationTask task = new ReplicationTask(toTarget(containerID, target), + output, completion, NO_COMPRESSION, replicateSizeCaptor); + ReplicateContainerCommand cmd = toTarget(containerID, target); + cmd.setReplicateSize(replicateSize); + ReplicationTask task = new ReplicationTask(cmd, subject); // WHEN @@ -129,7 +206,7 @@ private static long randomContainerID() { private ContainerReplicator createSubject( long containerID, DatanodeDetails target, OutputStream outputStream, Consumer> completion, - CopyContainerCompression compression + CopyContainerCompression compression, ArgumentCaptor replicateSizeCaptor ) throws IOException { ContainerReplicationSource source = mock(ContainerReplicationSource.class); ContainerUploader uploader = mock(ContainerUploader.class); @@ -140,7 +217,7 @@ private ContainerReplicator createSubject( when( uploader.startUpload(eq(containerID), eq(target), - futureArgument.capture(), compressionArgument.capture(), any(Long.class) + futureArgument.capture(), eq(compression), replicateSizeCaptor.capture() )) .thenReturn(outputStream); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java index e9c0328930cc..8dee27e488b1 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java @@ -73,7 +73,6 @@ import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -338,7 +337,7 @@ public void testDownloadAndImportReplicatorFailure(ContainerLayoutVersion layout when( moc.getContainerDataFromReplicas(anyLong(), anyList(), any(Path.class), any())) - .thenReturn(Pair.of(res, null)); + .thenReturn(res); final String testDir = tempFile.getPath(); MutableVolumeSet volumeSet = mock(MutableVolumeSet.class); @@ -408,7 +407,7 @@ public void testReplicationImportReserveSpace(ContainerLayoutVersion layout) when( moc.getContainerDataFromReplicas(anyLong(), anyList(), any(Path.class), any())) - .thenReturn(Pair.of(tarFile.toPath(), null)); + .thenReturn(tarFile.toPath()); ContainerImporter importer = new ContainerImporter(conf, set, controllerMock, volumeSet, volumeChoosingPolicy); @@ -583,7 +582,7 @@ public void testMultipleReplication(ContainerLayoutVersion layout, SimpleContainerDownloader moc = mock(SimpleContainerDownloader.class); Path res = Paths.get("file:/tmp/no-such-file"); when(moc.getContainerDataFromReplicas(anyLong(), anyList(), - any(Path.class), any())).thenReturn(Pair.of(res, null)); + any(Path.class), any())).thenReturn(res); final String testDir = tempFile.getPath(); MutableVolumeSet volumeSet = mock(MutableVolumeSet.class); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java index 961ed9b48c73..0448d116a94e 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java @@ -21,6 +21,7 @@ import static org.apache.hadoop.ozone.container.replication.CopyContainerCompression.NO_COMPRESSION; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doAnswer; @@ -31,6 +32,7 @@ import java.io.File; import java.io.IOException; +import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; @@ -51,6 +53,9 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; /** * Test for {@link SendContainerRequestHandler}. @@ -85,6 +90,24 @@ void setup() throws IOException { containerMaxSize = (long) conf.getStorageSize( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); + + // Mock the space reservation logic to be independent of the + // importer's implementation details. + doAnswer(invocation -> 2 * (long) invocation.getArgument(0)) + .when(importer).getRequiredReplicationSpace(anyLong()); + when(importer.getDefaultReplicationSpace()).thenReturn(2 * containerMaxSize); + } + + /** + * Provides stream of different container sizes for tests. + */ + public static Stream replicateSizeProvider() { + return Stream.of( + Arguments.of("Null replicate size (fallback to default)", null), + Arguments.of("Zero Size", 0L), + Arguments.of("Normal 2GB", 2L * 1024L * 1024L * 1024L), + Arguments.of("Overallocated 20GB", 20L * 1024L * 1024L * 1024L) + ); } @Test @@ -104,36 +127,27 @@ void testReceiveDataForExistingContainer() throws Exception { ((StorageContainerException) arg).getResult()); return null; }).when(responseObserver).onError(any()); - ByteString data = ByteString.copyFromUtf8("test"); - ContainerProtos.SendContainerRequest request - = ContainerProtos.SendContainerRequest.newBuilder() - .setContainerID(containerId) - .setData(data) - .setOffset(0) - .setCompression(NO_COMPRESSION.toProto()) - .build(); - sendContainerRequestHandler.onNext(request); + + sendContainerRequestHandler.onNext(createRequest(containerId, + ByteString.copyFromUtf8("test"), 0, null)); } - @Test - public void testSpaceReservedAndReleasedWhenRequestCompleted() throws Exception { + @ParameterizedTest(name = "for {0}") + @MethodSource("replicateSizeProvider") + public void testSpaceReservedAndReleasedWhenRequestCompleted(String testName, Long replicateSize) throws Exception { long containerId = 1; HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); + long expectedReservedSpace = replicateSize != null ? + importer.getRequiredReplicationSpace(replicateSize) : + importer.getDefaultReplicationSpace(); - // Create request - ContainerProtos.SendContainerRequest request = ContainerProtos.SendContainerRequest.newBuilder() - .setContainerID(containerId) - .setData(ByteString.EMPTY) - .setOffset(0) - .setCompression(CopyContainerCompression.NO_COMPRESSION.toProto()) - .build(); - - // Execute request - sendContainerRequestHandler.onNext(request); + // Create and execute the first request to reserve space + sendContainerRequestHandler.onNext( + createRequest(containerId, ByteString.EMPTY, 0, replicateSize)); // Verify commit space is reserved - assertEquals(volume.getCommittedBytes(), initialCommittedBytes + 2 * containerMaxSize); + assertEquals(volume.getCommittedBytes(), initialCommittedBytes + expectedReservedSpace); // complete the request sendContainerRequestHandler.onCompleted(); @@ -142,44 +156,50 @@ public void testSpaceReservedAndReleasedWhenRequestCompleted() throws Exception assertEquals(volume.getCommittedBytes(), initialCommittedBytes); } - @Test - public void testSpaceReservedAndReleasedWhenOnNextFails() throws Exception { + @ParameterizedTest(name = "for {0}") + @MethodSource("replicateSizeProvider") + public void testSpaceReservedAndReleasedWhenOnNextFails(String testName, Long replicateSize) throws Exception { long containerId = 1; HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); + long expectedReservedSpace = replicateSize != null ? + importer.getRequiredReplicationSpace(replicateSize) : + importer.getDefaultReplicationSpace(); - // Create request - ContainerProtos.SendContainerRequest request = createRequest(containerId, ByteString.copyFromUtf8("test"), 0); - - // Execute request - sendContainerRequestHandler.onNext(request); + ByteString data = ByteString.copyFromUtf8("test"); + // Execute first request to reserve space + sendContainerRequestHandler.onNext( + createRequest(containerId, data, 0, replicateSize)); // Verify commit space is reserved - assertEquals(volume.getCommittedBytes(), initialCommittedBytes + 2 * containerMaxSize); + assertEquals(volume.getCommittedBytes(), initialCommittedBytes + expectedReservedSpace); // mock the importer is not allowed to import this container when(importer.isAllowedContainerImport(containerId)).thenReturn(false); - sendContainerRequestHandler.onNext(request); + sendContainerRequestHandler.onNext(createRequest(containerId, data, 0, + replicateSize)); // Verify commit space is released assertEquals(volume.getCommittedBytes(), initialCommittedBytes); } - @Test - public void testSpaceReservedAndReleasedWhenOnCompletedFails() throws Exception { + @ParameterizedTest(name = "for {0}") + @MethodSource("replicateSizeProvider") + public void testSpaceReservedAndReleasedWhenOnCompletedFails(String testName, Long replicateSize) throws Exception { long containerId = 1; HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); - - // Create request - ContainerProtos.SendContainerRequest request = createRequest(containerId, ByteString.copyFromUtf8("test"), 0); + long expectedReservedSpace = replicateSize != null ? + importer.getRequiredReplicationSpace(replicateSize) : + importer.getDefaultReplicationSpace(); // Execute request - sendContainerRequestHandler.onNext(request); + sendContainerRequestHandler.onNext(createRequest(containerId, + ByteString.copyFromUtf8("test"), 0, replicateSize)); // Verify commit space is reserved - assertEquals(volume.getCommittedBytes(), initialCommittedBytes + 2 * containerMaxSize); + assertEquals(volume.getCommittedBytes(), initialCommittedBytes + expectedReservedSpace); doThrow(new IOException("Failed")).when(importer).importContainer(anyLong(), any(), any(), any()); @@ -189,12 +209,51 @@ public void testSpaceReservedAndReleasedWhenOnCompletedFails() throws Exception assertEquals(volume.getCommittedBytes(), initialCommittedBytes); } - private ContainerProtos.SendContainerRequest createRequest(long containerId, ByteString data, int offset) { - return ContainerProtos.SendContainerRequest.newBuilder() - .setContainerID(containerId) - .setData(data) - .setOffset(offset) - .setCompression(CopyContainerCompression.NO_COMPRESSION.toProto()) - .build(); + /** + * Test that verifies the actual space calculation difference between + * overallocated containers and default containers. + */ + @Test + public void testOverallocatedReservesMoreSpace() { + long containerId1 = 1; + long containerId2 = 2; + long overallocatedSize = 10L * 1024L * 1024L * 1024L; // 10GB + HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); + long initialCommittedBytes = volume.getCommittedBytes(); + // Test overallocated container (10GB) + SendContainerRequestHandler handler1 = new SendContainerRequestHandler(importer, responseObserver, null); + handler1.onNext(createRequest(containerId1, ByteString.EMPTY, 0, overallocatedSize)); + + long overallocatedReservation = volume.getCommittedBytes() - initialCommittedBytes; + handler1.onCompleted(); // Release space + + // Test default container (null replicateSize) + SendContainerRequestHandler handler2 = new SendContainerRequestHandler(importer, responseObserver, null); + handler2.onNext(createRequest(containerId2, ByteString.EMPTY, 0, null)); + + long defaultReservation = volume.getCommittedBytes() - initialCommittedBytes; + handler2.onCompleted(); // Release space + + // Verify overallocated container reserves more space + assertTrue(overallocatedReservation > defaultReservation); + + // Verify specific calculations + assertEquals(2 * overallocatedSize, overallocatedReservation); + assertEquals(2 * containerMaxSize, defaultReservation); + } + + private ContainerProtos.SendContainerRequest createRequest( + long containerId, ByteString data, int offset, Long replicateSize) { + ContainerProtos.SendContainerRequest.Builder builder = + ContainerProtos.SendContainerRequest.newBuilder() + .setContainerID(containerId) + .setData(data) + .setOffset(offset) + .setCompression(NO_COMPRESSION.toProto()); + + if (replicateSize != null) { + builder.setReplicateSize(replicateSize); + } + return builder.build(); } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java index e94e4e1a4e59..076fb17c711b 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSimpleContainerDownloader.java @@ -32,7 +32,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; @@ -57,7 +56,7 @@ public void testGetContainerDataFromReplicasHappyPath() throws Exception { //WHEN Path result = downloader.getContainerDataFromReplicas(1L, datanodes, - tempDir, NO_COMPRESSION).getLeft(); + tempDir, NO_COMPRESSION); //THEN assertEquals(datanodes.get(0).getUuidString(), @@ -78,7 +77,7 @@ public void testGetContainerDataFromReplicasDirectFailure() //WHEN final Path result = downloader.getContainerDataFromReplicas(1L, datanodes, - tempDir, NO_COMPRESSION).getLeft(); + tempDir, NO_COMPRESSION); //THEN //first datanode is failed, second worked @@ -99,7 +98,7 @@ public void testGetContainerDataFromReplicasAsyncFailure() throws Exception { //WHEN final Path result = downloader.getContainerDataFromReplicas(1L, datanodes, - tempDir, NO_COMPRESSION).getLeft(); + tempDir, NO_COMPRESSION); //THEN //first datanode is failed, second worked @@ -124,7 +123,7 @@ public void testRandomSelection() throws Exception { //returned. for (int i = 0; i < 10000; i++) { Path path = downloader.getContainerDataFromReplicas(1L, datanodes, - tempDir, NO_COMPRESSION).getLeft(); + tempDir, NO_COMPRESSION); if (path.toString().equals(datanodes.get(1).getUuidString())) { return; } @@ -207,7 +206,7 @@ protected GrpcReplicationClient createReplicationClient( } @Override - protected CompletableFuture> downloadContainer( + protected CompletableFuture downloadContainer( GrpcReplicationClient client, long containerId, Path downloadPath) { @@ -226,7 +225,7 @@ protected CompletableFuture> downloadContainer( //path includes the dn id to make it possible to assert. return CompletableFuture.completedFuture( - Pair.of(Paths.get(datanode.getUuidString()), null)); + Paths.get(datanode.getUuidString())); } } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 225773ca9878..ebf0050c097f 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -541,7 +541,6 @@ message CopyContainerResponseProto { required bool eof = 4; required bytes data = 5; optional int64 checksum = 6; - optional uint64 actualContainerSize = 7; } message SendContainerRequest { @@ -550,7 +549,7 @@ message SendContainerRequest { required bytes data = 3; optional int64 checksum = 4; optional CopyContainerCompressProto compression = 5; - optional int64 actualContainerSize = 6; + optional int64 replicateSize = 6; } message SendContainerResponse { diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto index e48ed4d1c595..0d93a51238bf 100644 --- a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto +++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto @@ -420,6 +420,7 @@ message ReplicateContainerCommandProto { optional int32 replicaIndex = 4; optional DatanodeDetailsProto target = 5; optional ReplicationCommandPriority priority = 6 [default = NORMAL]; + optional int64 replicateSize = 7; } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECMisReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECMisReplicationHandler.java index 1333efea5c35..68277faa10c3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECMisReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECMisReplicationHandler.java @@ -83,6 +83,7 @@ protected int sendReplicateCommands( // For EC containers, we need to track the replica index which is // to be replicated, so add it to the command. cmd.setReplicaIndex(replica.getReplicaIndex()); + cmd.setReplicateSize(containerInfo.getUsedBytes()); replicationManager.sendDatanodeCommand(cmd, containerInfo, target); } commandsSent++; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java index 158c802479f0..f5be77dc4f0f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java @@ -611,6 +611,7 @@ private void createReplicateCommand( // For EC containers, we need to track the replica index which is // to be replicated, so add it to the command. replicateCommand.setReplicaIndex(replica.getReplicaIndex()); + replicateCommand.setReplicateSize(container.getUsedBytes()); replicationManager.sendDatanodeCommand(replicateCommand, container, target); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisMisReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisMisReplicationHandler.java index e15598ccfe8c..e82b6f2b2008 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisMisReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisMisReplicationHandler.java @@ -73,6 +73,7 @@ protected int sendReplicateCommands( } else { ReplicateContainerCommand cmd = ReplicateContainerCommand .fromSources(containerID, sources); + cmd.setReplicateSize(containerInfo.getUsedBytes()); replicationManager.sendDatanodeCommand(cmd, containerInfo, target); } commandsSent++; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java index 57eb29033e4c..4711358addb3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java @@ -475,6 +475,7 @@ private int sendReplicationCommands( ReplicateContainerCommand command = ReplicateContainerCommand.fromSources( containerInfo.getContainerID(), sources); + command.setReplicateSize(containerInfo.getUsedBytes()); replicationManager.sendDatanodeCommand(command, containerInfo, target); commandsSent++; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java index 33ab37a9387a..ec7d04e6618a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java @@ -529,6 +529,7 @@ public void sendThrottledReplicationCommand(ContainerInfo containerInfo, ReplicateContainerCommand cmd = ReplicateContainerCommand.toTarget(containerID, target); cmd.setReplicaIndex(replicaIndex); + cmd.setReplicateSize(containerInfo.getUsedBytes()); sendDatanodeCommand(cmd, containerInfo, source); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java index 8429146f691f..af42ce3b7527 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java @@ -59,7 +59,6 @@ import java.util.Date; import java.util.List; import java.util.UUID; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -403,10 +402,10 @@ private void assertDownloadContainerFails(long containerId, LogCapturer logCapture = captureLogs(SimpleContainerDownloader.class); SimpleContainerDownloader downloader = new SimpleContainerDownloader(conf, caClient); - Pair result = downloader.getContainerDataFromReplicas(containerId, + Path file = downloader.getContainerDataFromReplicas(containerId, sourceDatanodes, tempFolder.resolve("tmp"), NO_COMPRESSION); downloader.close(); - assertNull(result); + assertNull(file); assertThat(logCapture.getOutput()) .contains("java.security.cert.CertificateExpiredException"); } @@ -417,7 +416,7 @@ private void assertDownloadContainerWorks(List containers, SimpleContainerDownloader downloader = new SimpleContainerDownloader(conf, caClient); Path file = downloader.getContainerDataFromReplicas(cId, sourceDatanodes, - tempFolder.resolve("tmp"), NO_COMPRESSION).getLeft(); + tempFolder.resolve("tmp"), NO_COMPRESSION); downloader.close(); assertNotNull(file); } From be8b95c35f7de2d0040a414ca6154cbee0364ac1 Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Wed, 3 Sep 2025 13:53:11 +0530 Subject: [PATCH 03/10] fixed findbug --- .../replication/TestDownloadAndImportReplicator.java | 5 ++--- .../replication/TestSendContainerRequestHandler.java | 4 ++-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java index b48ba9d9a570..8b221ac5122b 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java @@ -89,10 +89,9 @@ void setup() throws IOException { // Mock the space reservation logic to be independent of the // importer's implementation details. - doAnswer(invocation -> 2 * (long) invocation.getArgument(0)) - .when(importer).getRequiredReplicationSpace(anyLong()); + when(importer.getRequiredReplicationSpace(anyLong())) + .thenAnswer(invocation -> 2 * (long) invocation.getArgument(0)); when(importer.getDefaultReplicationSpace()).thenReturn(2 * containerMaxSize); - } /** diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java index 0448d116a94e..db164f7d5b19 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java @@ -93,8 +93,8 @@ void setup() throws IOException { // Mock the space reservation logic to be independent of the // importer's implementation details. - doAnswer(invocation -> 2 * (long) invocation.getArgument(0)) - .when(importer).getRequiredReplicationSpace(anyLong()); + when(importer.getRequiredReplicationSpace(anyLong())) + .thenAnswer(invocation -> 2 * (long) invocation.getArgument(0)); when(importer.getDefaultReplicationSpace()).thenReturn(2 * containerMaxSize); } From 533708539f0a48ee7321e467a1fa27c95787a01e Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Thu, 4 Sep 2025 09:38:58 +0530 Subject: [PATCH 04/10] variables name refactoring --- .../replication/ContainerImporter.java | 4 ++-- .../DownloadAndImportReplicator.java | 10 ++++----- .../SendContainerRequestHandler.java | 21 ++++++++----------- 3 files changed, 16 insertions(+), 19 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java index 879ffe563994..17831bb5814f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java @@ -146,11 +146,11 @@ private static void deleteFileQuietely(Path tarFilePath) { } } - HddsVolume chooseNextVolume(long spaceReserved) throws IOException { + HddsVolume chooseNextVolume(long spaceToReserve) throws IOException { // Choose volume that can hold both container in tmp and dest directory return volumeChoosingPolicy.chooseVolume( StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()), - spaceReserved); + spaceToReserve); } public static Path getUntarDirectory(HddsVolume hddsVolume) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java index 7950567ac886..0d3fdcecf898 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java @@ -73,17 +73,17 @@ public void replicate(ReplicationTask task) { HddsVolume targetVolume = null; // Use replicate size from command if available, otherwise use default - long spaceReserved = 0; + long spaceToReserve = 0; Long replicateSize = task.getReplicateSize(); if (replicateSize != null) { - spaceReserved = containerImporter.getRequiredReplicationSpace(replicateSize); + spaceToReserve = containerImporter.getRequiredReplicationSpace(replicateSize); } else { // Fallback to default (backward compatibility) - spaceReserved = containerImporter.getDefaultReplicationSpace(); + spaceToReserve = containerImporter.getDefaultReplicationSpace(); } try { - targetVolume = containerImporter.chooseNextVolume(spaceReserved); + targetVolume = containerImporter.chooseNextVolume(spaceToReserve); // Wait for the download. This thread pool is limiting the parallel // downloads, so it's ok to block here and wait for the full download. @@ -110,7 +110,7 @@ public void replicate(ReplicationTask task) { task.setStatus(Status.FAILED); } finally { if (targetVolume != null) { - targetVolume.incCommittedBytes(-spaceReserved); + targetVolume.incCommittedBytes(-spaceToReserve); } } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java index d4d62db5b2b7..eebb0a9451ab 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java @@ -54,8 +54,7 @@ class SendContainerRequestHandler private Path path; private CopyContainerCompression compression; private final ZeroCopyMessageMarshaller marshaller; - private long spaceReserved = 0; - private boolean firstRequest = true; + private long spaceToReserve = 0; SendContainerRequestHandler( ContainerImporter importer, @@ -88,13 +87,13 @@ public void onNext(SendContainerRequest req) { containerId = req.getContainerID(); // Use replicate size if available, otherwise fall back to default - if (firstRequest && req.hasReplicateSize()) { - spaceReserved = importer.getRequiredReplicationSpace(req.getReplicateSize()); + if (req.hasReplicateSize()) { + spaceToReserve = importer.getRequiredReplicationSpace(req.getReplicateSize()); } else { - spaceReserved = importer.getDefaultReplicationSpace(); + spaceToReserve = importer.getDefaultReplicationSpace(); } - volume = importer.chooseNextVolume(spaceReserved); + volume = importer.chooseNextVolume(spaceToReserve); Path dir = ContainerImporter.getUntarDirectory(volume); Files.createDirectories(dir); @@ -104,8 +103,6 @@ public void onNext(SendContainerRequest req) { LOG.info("Accepting container {}", req.getContainerID()); } - - firstRequest = false; assertSame(containerId, req.getContainerID(), "containerID"); @@ -129,8 +126,8 @@ public void onError(Throwable t) { deleteTarball(); responseObserver.onError(t); } finally { - if (volume != null && spaceReserved > 0) { - volume.incCommittedBytes(-spaceReserved); + if (volume != null && spaceToReserve > 0) { + volume.incCommittedBytes(-spaceToReserve); } } } @@ -158,8 +155,8 @@ public void onCompleted() { responseObserver.onError(t); } } finally { - if (volume != null && spaceReserved > 0) { - volume.incCommittedBytes(-spaceReserved); + if (volume != null && spaceToReserve > 0) { + volume.incCommittedBytes(-spaceToReserve); } } } From 7e528ae68ba92581f0c8e395cff107ad273dfefa Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Wed, 10 Sep 2025 11:54:55 +0530 Subject: [PATCH 05/10] update integration test and some refactoring --- .../replication/ContainerImporter.java | 17 ++++++++ .../DownloadAndImportReplicator.java | 12 ++---- .../SendContainerRequestHandler.java | 7 +-- .../TestDownloadAndImportReplicator.java | 6 --- .../TestSendContainerRequestHandler.java | 8 +--- .../replication/TestContainerReplication.java | 43 ++++++++++++++++--- 6 files changed, 59 insertions(+), 34 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java index 17831bb5814f..7ebbb4b29304 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java @@ -186,4 +186,21 @@ public long getDefaultReplicationSpace() { public long getRequiredReplicationSpace(long actualContainerSize) { return HddsServerUtil.requiredReplicationSpace(actualContainerSize); } + + /** + * Get space to reserve for replication. If replicateSize is provided, + * calculate required space based on that, otherwise return default + * replication space. + * + * @param replicateSize the size of the container to replicate in bytes + * (can be null) + * @return space to reserve for replication + */ + public long getSpaceToReserve(Long replicateSize) { + if (replicateSize != null) { + return getRequiredReplicationSpace(replicateSize); + } else { + return getDefaultReplicationSpace(); + } + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java index 0d3fdcecf898..17776e95c8db 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java @@ -44,6 +44,7 @@ public class DownloadAndImportReplicator implements ContainerReplicator { private final ContainerDownloader downloader; private final ContainerImporter containerImporter; private final ContainerSet containerSet; + private Long spaceToReserve; public DownloadAndImportReplicator( ConfigurationSource conf, ContainerSet containerSet, @@ -73,14 +74,7 @@ public void replicate(ReplicationTask task) { HddsVolume targetVolume = null; // Use replicate size from command if available, otherwise use default - long spaceToReserve = 0; - Long replicateSize = task.getReplicateSize(); - if (replicateSize != null) { - spaceToReserve = containerImporter.getRequiredReplicationSpace(replicateSize); - } else { - // Fallback to default (backward compatibility) - spaceToReserve = containerImporter.getDefaultReplicationSpace(); - } + spaceToReserve = containerImporter.getSpaceToReserve(task.getReplicateSize()); try { targetVolume = containerImporter.chooseNextVolume(spaceToReserve); @@ -109,7 +103,7 @@ public void replicate(ReplicationTask task) { LOG.error("Container {} replication was unsuccessful.", containerID, e); task.setStatus(Status.FAILED); } finally { - if (targetVolume != null) { + if (targetVolume != null && spaceToReserve > 0) { targetVolume.incCommittedBytes(-spaceToReserve); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java index eebb0a9451ab..9222b22ae09e 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java @@ -87,11 +87,8 @@ public void onNext(SendContainerRequest req) { containerId = req.getContainerID(); // Use replicate size if available, otherwise fall back to default - if (req.hasReplicateSize()) { - spaceToReserve = importer.getRequiredReplicationSpace(req.getReplicateSize()); - } else { - spaceToReserve = importer.getDefaultReplicationSpace(); - } + spaceToReserve = importer.getSpaceToReserve( + req.hasReplicateSize() ? req.getReplicateSize() : null); volume = importer.chooseNextVolume(spaceToReserve); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java index 8b221ac5122b..33adee7f1da9 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java @@ -86,12 +86,6 @@ void setup() throws IOException { containerMaxSize = (long) conf.getStorageSize( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); - - // Mock the space reservation logic to be independent of the - // importer's implementation details. - when(importer.getRequiredReplicationSpace(anyLong())) - .thenAnswer(invocation -> 2 * (long) invocation.getArgument(0)); - when(importer.getDefaultReplicationSpace()).thenReturn(2 * containerMaxSize); } /** diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java index db164f7d5b19..cc0592b5b3e3 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java @@ -90,12 +90,6 @@ void setup() throws IOException { containerMaxSize = (long) conf.getStorageSize( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); - - // Mock the space reservation logic to be independent of the - // importer's implementation details. - when(importer.getRequiredReplicationSpace(anyLong())) - .thenAnswer(invocation -> 2 * (long) invocation.getArgument(0)); - when(importer.getDefaultReplicationSpace()).thenReturn(2 * containerMaxSize); } /** @@ -214,7 +208,7 @@ public void testSpaceReservedAndReleasedWhenOnCompletedFails(String testName, Lo * overallocated containers and default containers. */ @Test - public void testOverallocatedReservesMoreSpace() { + public void testOverAllocatedReservesMoreSpace() { long containerId1 = 1; long containerId2 = 2; long overallocatedSize = 10L * 1024L * 1024L * 1024L; // 10GB diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java index 68fecdb52d3e..73289639e3eb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java @@ -33,7 +33,9 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.function.ToLongFunction; +import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port; @@ -51,7 +53,8 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; /** * Tests ozone containers replication. @@ -88,9 +91,32 @@ static void tearDown() { IOUtils.closeQuietly(clientFactory, cluster); } - @ParameterizedTest - @EnumSource - void testPush(CopyContainerCompression compression) throws Exception { + /** + * Creates a Cartesian product of all compression types and replicate sizes. + */ + public static Stream compressionAndReplicateSize() { + List replicateSizes = Stream.of( + Arguments.of("Null replicate size (fallback to default)", null), + Arguments.of("Zero Size", 0L), + Arguments.of("Normal 2GB", 2L * 1024L * 1024L * 1024L), + Arguments.of("Overallocated 20GB", 20L * 1024L * 1024L * 1024L) + ).collect(Collectors.toList()); + + // Create a Cartesian product of all compression types and the sizes + return Stream.of(CopyContainerCompression.values()) + .flatMap(compression -> + replicateSizes.stream().map(sizeArgs -> { + Object[] args = sizeArgs.get(); + // Arguments are: compression, testName, replicateSize + return Arguments.of(compression, args[0], args[1]); + }) + ); + } + + @ParameterizedTest(name = "{1} with {0} compression") + @MethodSource("compressionAndReplicateSize") + void testPushWithReplicateSize(CopyContainerCompression compression, String testName, Long replicateSize) + throws Exception { final int index = compression.ordinal(); DatanodeDetails source = cluster.getHddsDatanodes().get(index) .getDatanodeDetails(); @@ -98,14 +124,16 @@ void testPush(CopyContainerCompression compression) throws Exception { DatanodeDetails target = selectOtherNode(source); ReplicateContainerCommand cmd = ReplicateContainerCommand.toTarget(containerID, target); + cmd.setReplicateSize(replicateSize); queueAndWaitForCompletion(cmd, source, ReplicationSupervisor::getReplicationSuccessCount); } - @ParameterizedTest - @EnumSource - void testPull(CopyContainerCompression compression) throws Exception { + @ParameterizedTest(name = "{1} with {0} compression") + @MethodSource("compressionAndReplicateSize") + void testPullWithReplicateSize(CopyContainerCompression compression, String testName, Long replicateSize) + throws Exception { final int index = compression.ordinal(); DatanodeDetails target = cluster.getHddsDatanodes().get(index) .getDatanodeDetails(); @@ -114,6 +142,7 @@ void testPull(CopyContainerCompression compression) throws Exception { ReplicateContainerCommand cmd = ReplicateContainerCommand.fromSources(containerID, ImmutableList.of(source)); + cmd.setReplicateSize(replicateSize); queueAndWaitForCompletion(cmd, target, ReplicationSupervisor::getReplicationSuccessCount); From 4d26afa506e6084286cb48508589c7774c584af3 Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Wed, 10 Sep 2025 14:50:00 +0530 Subject: [PATCH 06/10] fixed findbugs error --- .../replication/TestDownloadAndImportReplicator.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java index 33adee7f1da9..d23fd2495503 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java @@ -36,7 +36,6 @@ import java.util.concurrent.Semaphore; import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; @@ -67,7 +66,6 @@ public class TestDownloadAndImportReplicator { private MutableVolumeSet volumeSet; private SimpleContainerDownloader downloader; private DownloadAndImportReplicator replicator; - private long containerMaxSize; @BeforeEach void setup() throws IOException { @@ -83,9 +81,6 @@ void setup() throws IOException { downloader = mock(SimpleContainerDownloader.class); replicator = new DownloadAndImportReplicator(conf, containerSet, importer, downloader); - containerMaxSize = (long) conf.getStorageSize( - ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, - ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); } /** From 6c8fb36808e645b065b39dc1df06081488abacdc Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Mon, 22 Sep 2025 11:30:14 +0530 Subject: [PATCH 07/10] removed size from ReplicateContainerCommandProto and changes done to push replicator only --- .../statemachine/DatanodeStateMachine.java | 2 +- .../replication/ContainerUploader.java | 3 +- .../DownloadAndImportReplicator.java | 10 +- .../replication/GrpcContainerUploader.java | 20 +++- .../container/replication/PushReplicator.java | 3 +- .../replication/ReplicationTask.java | 4 - .../SendContainerOutputStream.java | 12 +- .../SendContainerRequestHandler.java | 4 +- .../commands/ReplicateContainerCommand.java | 24 ---- .../TestDownloadAndImportReplicator.java | 97 +++------------ .../TestGrpcContainerUploader.java | 7 +- .../TestGrpcReplicationService.java | 2 +- .../replication/TestPushReplicator.java | 111 +++--------------- .../TestSendContainerRequestHandler.java | 42 +++---- .../main/proto/DatanodeClientProtocol.proto | 2 +- .../ScmServerDatanodeHeartbeatProtocol.proto | 1 - .../replication/ECMisReplicationHandler.java | 1 - .../ECUnderReplicationHandler.java | 1 - .../RatisMisReplicationHandler.java | 1 - .../RatisUnderReplicationHandler.java | 1 - .../replication/ReplicationManager.java | 1 - .../replication/TestContainerReplication.java | 43 ++----- 22 files changed, 94 insertions(+), 298 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java index 5275c047d43b..1bd888c84a61 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java @@ -206,7 +206,7 @@ public DatanodeStateMachine(HddsDatanodeService hddsDatanodeService, new SimpleContainerDownloader(conf, certClient)); ContainerReplicator pushReplicator = new PushReplicator(conf, new OnDemandContainerReplicationSource(container.getController()), - new GrpcContainerUploader(conf, certClient) + new GrpcContainerUploader(conf, certClient, container.getController()) ); pullReplicatorWithMetrics = new MeasuredReplicator(pullReplicator, "pull"); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java index 420bb034b58e..3b4de6377b85 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java @@ -27,6 +27,5 @@ */ public interface ContainerUploader { OutputStream startUpload(long containerId, DatanodeDetails target, - CompletableFuture callback, CopyContainerCompression compression, - Long replicateSize) throws IOException; + CompletableFuture callback, CopyContainerCompression compression) throws IOException; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java index 17776e95c8db..09aa1e8fad6b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java @@ -44,7 +44,6 @@ public class DownloadAndImportReplicator implements ContainerReplicator { private final ContainerDownloader downloader; private final ContainerImporter containerImporter; private final ContainerSet containerSet; - private Long spaceToReserve; public DownloadAndImportReplicator( ConfigurationSource conf, ContainerSet containerSet, @@ -72,12 +71,9 @@ public void replicate(ReplicationTask task) { LOG.info("Starting replication of container {} from {} using {}", containerID, sourceDatanodes, compression); HddsVolume targetVolume = null; - - // Use replicate size from command if available, otherwise use default - spaceToReserve = containerImporter.getSpaceToReserve(task.getReplicateSize()); try { - targetVolume = containerImporter.chooseNextVolume(spaceToReserve); + targetVolume = containerImporter.chooseNextVolume(containerImporter.getDefaultReplicationSpace()); // Wait for the download. This thread pool is limiting the parallel // downloads, so it's ok to block here and wait for the full download. @@ -103,8 +99,8 @@ public void replicate(ReplicationTask task) { LOG.error("Container {} replication was unsuccessful.", containerID, e); task.setStatus(Status.FAILED); } finally { - if (targetVolume != null && spaceToReserve > 0) { - targetVolume.incCommittedBytes(-spaceToReserve); + if (targetVolume != null) { + targetVolume.incCommittedBytes(-containerImporter.getDefaultReplicationSpace()); } } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java index 6923e8786263..7faceed75690 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java @@ -30,6 +30,8 @@ import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient; import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; import org.apache.ratis.thirdparty.io.grpc.stub.CallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.slf4j.Logger; @@ -45,17 +47,27 @@ public class GrpcContainerUploader implements ContainerUploader { private final SecurityConfig securityConfig; private final CertificateClient certClient; + private final ContainerController containerController; public GrpcContainerUploader( - ConfigurationSource conf, CertificateClient certClient) { + ConfigurationSource conf, CertificateClient certClient, + ContainerController containerController) { this.certClient = certClient; + this.containerController = containerController; securityConfig = new SecurityConfig(conf); } @Override public OutputStream startUpload(long containerId, DatanodeDetails target, - CompletableFuture callback, CopyContainerCompression compression, - Long replicateSize) throws IOException { + CompletableFuture callback, CopyContainerCompression compression) throws IOException { + + // Get container size from local datanode instead of using passed replicateSize + Long containerSize = null; + Container container = containerController.getContainer(containerId); + if (container != null) { + containerSize = container.getContainerData().getBytesUsed(); + } + GrpcReplicationClient client = createReplicationClient(target, compression); try { // gRPC runtime always provides implementation of CallStreamObserver @@ -68,7 +80,7 @@ public OutputStream startUpload(long containerId, DatanodeDetails target, (CallStreamObserver) client.upload( responseObserver), responseObserver); return new SendContainerOutputStream(requestStream, containerId, - GrpcReplicationService.BUFFER_SIZE, compression, replicateSize) { + GrpcReplicationService.BUFFER_SIZE, compression, containerSize) { @Override public void close() throws IOException { try { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java index 35a076cd749f..759aff722baf 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java @@ -60,9 +60,8 @@ public void replicate(ReplicationTask task) { CountingOutputStream output = null; try { - Long replicateSize = task.getReplicateSize(); output = new CountingOutputStream( - uploader.startUpload(containerID, target, fut, compression, replicateSize)); + uploader.startUpload(containerID, target, fut, compression)); source.copyData(containerID, output, compression); fut.get(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java index efdbfd5f02ec..f61e219678ec 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java @@ -101,10 +101,6 @@ public List getSources() { return cmd.getSourceDatanodes(); } - public Long getReplicateSize() { - return cmd.getReplicateSize(); - } - @Override protected Object getCommandForDebug() { return debugString; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java index 408efad5bd4d..3bb7e463d9d3 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java @@ -27,15 +27,15 @@ class SendContainerOutputStream extends GrpcOutputStream { private final CopyContainerCompression compression; - private final Long replicateSize; + private final Long size; SendContainerOutputStream( CallStreamObserver streamObserver, long containerId, int bufferSize, CopyContainerCompression compression, - Long replicateSize) { + Long size) { super(streamObserver, containerId, bufferSize); this.compression = compression; - this.replicateSize = replicateSize; + this.size = size; } @Override @@ -46,9 +46,9 @@ protected void sendPart(boolean eof, int length, ByteString data) { .setOffset(getWrittenBytes()) .setCompression(compression.toProto()); - // Include replicate size in the first request - if (getWrittenBytes() == 0 && replicateSize != null) { - requestBuilder.setReplicateSize(replicateSize); + // Include container size in the first request + if (getWrittenBytes() == 0 && size != null) { + requestBuilder.setSize(size); } getStreamObserver().onNext(requestBuilder.build()); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java index 9222b22ae09e..0824341127c3 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java @@ -86,9 +86,9 @@ public void onNext(SendContainerRequest req) { if (containerId == -1) { containerId = req.getContainerID(); - // Use replicate size if available, otherwise fall back to default + // Use container size if available, otherwise fall back to default spaceToReserve = importer.getSpaceToReserve( - req.hasReplicateSize() ? req.getReplicateSize() : null); + req.hasSize() ? req.getSize() : null); volume = importer.chooseNextVolume(spaceToReserve); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java index bf633aa33f2c..985d4163bae6 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java @@ -43,9 +43,6 @@ public final class ReplicateContainerCommand private ReplicationCommandPriority priority = ReplicationCommandPriority.NORMAL; - // Actual container size in bytes - private Long replicateSize; - public static ReplicateContainerCommand fromSources(long containerID, List sourceDatanodes) { return new ReplicateContainerCommand(containerID, sourceDatanodes, null); @@ -85,14 +82,6 @@ public void setPriority(ReplicationCommandPriority priority) { this.priority = priority; } - public void setReplicateSize(Long replicateSize) { - this.replicateSize = replicateSize; - } - - public Long getReplicateSize() { - return replicateSize; - } - @Override public Type getType() { return SCMCommandProto.Type.replicateContainerCommand; @@ -117,11 +106,6 @@ public ReplicateContainerCommandProto getProto() { } builder.setPriority(priority); - // Add replicate size if available - if (replicateSize != null) { - builder.setReplicateSize(replicateSize); - } - return builder.build(); } @@ -149,11 +133,6 @@ public static ReplicateContainerCommand getFromProtobuf( cmd.setPriority(protoMessage.getPriority()); } - // Add replicate size handling for backward compatibility - if (protoMessage.hasReplicateSize()) { - cmd.setReplicateSize(protoMessage.getReplicateSize()); - } - return cmd; } @@ -187,9 +166,6 @@ public String toString() { .append(", deadlineMsSinceEpoch: ").append(getDeadline()); sb.append(", containerId=").append(getContainerID()); sb.append(", replicaIndex=").append(getReplicaIndex()); - if (replicateSize != null) { - sb.append(", replicateSize=").append(replicateSize); - } if (targetDatanode != null) { sb.append(", targetNode=").append(targetDatanode); } else { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java index d23fd2495503..3b0811a15da7 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java @@ -22,21 +22,17 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; -import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy; @@ -45,14 +41,11 @@ import org.apache.hadoop.ozone.container.common.volume.StorageVolume; import org.apache.hadoop.ozone.container.common.volume.VolumeChoosingPolicyFactory; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; -import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; import org.apache.ozone.test.GenericTestUtils; 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.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; /** * Test for DownloadAndImportReplicator. @@ -66,6 +59,7 @@ public class TestDownloadAndImportReplicator { private MutableVolumeSet volumeSet; private SimpleContainerDownloader downloader; private DownloadAndImportReplicator replicator; + private long containerMaxSize; @BeforeEach void setup() throws IOException { @@ -77,72 +71,19 @@ void setup() throws IOException { StorageVolume.VolumeType.DATA_VOLUME, null); ContainerImporter importer = new ContainerImporter(conf, containerSet, mock(ContainerController.class), volumeSet, volumeChoosingPolicy); - importer = spy(importer); downloader = mock(SimpleContainerDownloader.class); replicator = new DownloadAndImportReplicator(conf, containerSet, importer, downloader); + containerMaxSize = (long) conf.getStorageSize( + ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, + ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); } - /** - * Provides a stream of different container sizes for tests. - */ - static Stream replicateSizeProvider() { - return Stream.of( - Arguments.of("Null replicate size (fallback to default)", null), - Arguments.of("Normal 2GB", 2L * 1024L * 1024L * 1024L), - Arguments.of("Overallocated 20GB", 20L * 1024L * 1024L * 1024L) - ); - } - - @ParameterizedTest(name = "for {0}") - @MethodSource("replicateSizeProvider") - public void testSpaceReservedAndReleasedOnSuccess(String testName, Long replicateSize) - throws IOException { - // GIVEN - long containerId = 1; - HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); - long initialCommittedBytes = volume.getCommittedBytes(); - long expectedReservedSpace = replicateSize != null ? - importer.getRequiredReplicationSpace(replicateSize) : - importer.getDefaultReplicationSpace(); - - // Mock downloader to check reservation and return a dummy path - Path dummyPath = tempDir.toPath().resolve("dummy.tar"); - Files.createFile(dummyPath); - when(downloader.getContainerDataFromReplicas(anyLong(), any(), any(), any())) - .thenAnswer(invocation -> { - // Check that space was reserved before download attempt - assertEquals(initialCommittedBytes + expectedReservedSpace, - volume.getCommittedBytes()); - return dummyPath; - }); - // Mock the import itself to avoid file system operations - doAnswer(invocation -> null).when(importer) - .importContainer(anyLong(), any(), any(), any()); - - ReplicationTask task = createTask(containerId, replicateSize); - - // WHEN - // The replicator should reserve space, "download", "import", and then - // release the space in the finally block. - replicator.replicate(task); - - // THEN - assertEquals(AbstractReplicationTask.Status.DONE, task.getStatus()); - assertEquals(initialCommittedBytes, volume.getCommittedBytes(), - "Committed space should be released on success"); - } - - @ParameterizedTest(name = "for {0}") - @MethodSource("replicateSizeProvider") - public void testCommitSpaceReleasedOnReplicationFailure(String testName, Long replicateSize) - throws Exception { + @Test + public void testCommitSpaceReleasedOnReplicationFailure() throws Exception { long containerId = 1; HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); - long expectedReservedSpace = replicateSize != null ? - importer.getRequiredReplicationSpace(replicateSize) : - importer.getDefaultReplicationSpace(); // Mock downloader to throw exception Semaphore semaphore = new Semaphore(1); @@ -152,8 +93,8 @@ public void testCommitSpaceReleasedOnReplicationFailure(String testName, Long re throw new IOException("Download failed"); }); - // Create task with the parameterized replicateSize - ReplicationTask task = createTask(containerId, replicateSize); + ReplicationTask task = new ReplicationTask(containerId, + Collections.singletonList(mock(DatanodeDetails.class)), replicator); // Acquire semaphore so that container import will pause before downloading. semaphore.acquire(); @@ -163,26 +104,16 @@ public void testCommitSpaceReleasedOnReplicationFailure(String testName, Long re // Wait such that first container import reserve space GenericTestUtils.waitFor(() -> - volume.getCommittedBytes() > initialCommittedBytes, + volume.getCommittedBytes() > initialCommittedBytes, 1000, 50000); - assertEquals(volume.getCommittedBytes(), initialCommittedBytes + expectedReservedSpace); + assertEquals(volume.getCommittedBytes(), initialCommittedBytes + 2 * containerMaxSize); semaphore.release(); GenericTestUtils.waitFor(() -> - volume.getCommittedBytes() == initialCommittedBytes, + volume.getCommittedBytes() == initialCommittedBytes, 1000, 50000); // Verify commit space is released assertEquals(initialCommittedBytes, volume.getCommittedBytes()); } - - private ReplicationTask createTask(long containerId, Long replicateSize) { - ReplicateContainerCommand cmd = ReplicateContainerCommand.fromSources( - containerId, - Collections.singletonList(MockDatanodeDetails.randomDatanodeDetails())); - if (replicateSize != null) { - cmd.setReplicateSize(replicateSize); - } - return new ReplicationTask(cmd, replicator); - } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcContainerUploader.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcContainerUploader.java index 490f15ccf44f..b10b412b12f0 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcContainerUploader.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcContainerUploader.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerRequest; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerResponse; +import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; import org.apache.ratis.thirdparty.io.grpc.stub.CallStreamObserver; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.junit.jupiter.api.Test; @@ -116,8 +117,8 @@ void immediateError() throws Exception { private static GrpcContainerUploader createSubject( GrpcReplicationClient client) { - - return new GrpcContainerUploader(new InMemoryConfiguration(), null) { + return new GrpcContainerUploader(new InMemoryConfiguration(), null, + mock(ContainerController.class)) { @Override protected GrpcReplicationClient createReplicationClient( DatanodeDetails target, CopyContainerCompression compression) { @@ -129,7 +130,7 @@ protected GrpcReplicationClient createReplicationClient( private static OutputStream startUpload(GrpcContainerUploader subject, CompletableFuture callback) throws IOException { DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); - return subject.startUpload(1, target, callback, NO_COMPRESSION, null); + return subject.startUpload(1, target, callback, NO_COMPRESSION); } /** diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java index 0172f123d9b5..8b831fa06466 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java @@ -193,7 +193,7 @@ public void testUpload() { ContainerReplicationSource source = new OnDemandContainerReplicationSource(containerController); - GrpcContainerUploader uploader = new GrpcContainerUploader(conf, null); + GrpcContainerUploader uploader = new GrpcContainerUploader(conf, null, containerController); PushReplicator pushReplicator = new PushReplicator(conf, source, uploader); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java index cda51deda20b..f804dd08f323 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java @@ -21,7 +21,6 @@ import static org.apache.hadoop.ozone.container.replication.CopyContainerCompression.NO_COMPRESSION; import static org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand.toTarget; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.AssertionsKt.assertNull; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.eq; @@ -30,23 +29,18 @@ import java.io.IOException; import java.io.OutputStream; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ThreadLocalRandom; import java.util.function.Consumer; -import java.util.stream.Stream; 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.ozone.container.replication.AbstractReplicationTask.Status; -import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; import org.apache.ozone.test.SpyOutputStream; 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.junit.jupiter.params.provider.EnumSource; import org.mockito.ArgumentCaptor; /** @@ -61,35 +55,10 @@ void setup() { conf = new OzoneConfiguration(); } - /** - * Provides a stream of different container sizes for tests. - */ - static Stream replicateSize() { - return Stream.of( - Arguments.of("Normal 2GB", 2L * 1024L * 1024L * 1024L), - Arguments.of("Max Container Size 5GB", 5L * 1024L * 1024L * 1024L), - Arguments.of("Overallocated 20GB", 20L * 1024L * 1024L * 1024L) - ); - } - - /** - * Creates a Cartesian product of all compression types and container sizes. - */ - public static Stream compressionAndReplicateSize() { - List arguments = new ArrayList<>(); - for (CopyContainerCompression compression : CopyContainerCompression.values()) { - replicateSize().forEach(sizeArgs -> { - Object[] args = sizeArgs.get(); - arguments.add(Arguments.of(compression, args[0], args[1])); - }); - } - return arguments.stream(); - } - - @ParameterizedTest(name = "{1} with {0} compression") - @MethodSource("compressionAndReplicateSize") - void uploadCompletesNormallyWithReplicateSize( - CopyContainerCompression compression, String testName, Long replicateSize) throws IOException { + @ParameterizedTest + @EnumSource + void uploadCompletesNormally(CopyContainerCompression compression) + throws IOException { // GIVEN compression.setOn(conf); long containerID = randomContainerID(); @@ -97,12 +66,9 @@ void uploadCompletesNormallyWithReplicateSize( Consumer> completion = fut -> fut.complete(null); SpyOutputStream output = new SpyOutputStream(NULL_OUTPUT_STREAM); - ArgumentCaptor replicateSizeCaptor = ArgumentCaptor.forClass(Long.class); ContainerReplicator subject = createSubject(containerID, target, - output, completion, compression, replicateSizeCaptor); - ReplicateContainerCommand cmd = toTarget(containerID, target); - cmd.setReplicateSize(replicateSize); - ReplicationTask task = new ReplicationTask(cmd, + output, completion, compression); + ReplicationTask task = new ReplicationTask(toTarget(containerID, target), subject); // WHEN @@ -111,58 +77,19 @@ void uploadCompletesNormallyWithReplicateSize( // THEN assertEquals(Status.DONE, task.getStatus()); output.assertClosedExactlyOnce(); - - // Verify the task also has the correct size - assertEquals(replicateSize, task.getReplicateSize()); } - /** - * Test that verifies backward compatibility - when replicateSize is null, - * the uploader still receives null (target will handle fallback to default). - */ @Test - void uploadCompletesWithNullReplicateSize() throws IOException { - // GIVEN - long containerID = randomContainerID(); - DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); - Consumer> completion = - fut -> fut.complete(null); - SpyOutputStream output = new SpyOutputStream(NULL_OUTPUT_STREAM); - ArgumentCaptor replicateSizeCaptor = ArgumentCaptor.forClass(Long.class); - ContainerReplicator subject = createSubject(containerID, target, - output, completion, NO_COMPRESSION, replicateSizeCaptor); - ReplicateContainerCommand cmd = toTarget(containerID, target); - // replicate size is not set - it is null - ReplicationTask task = new ReplicationTask(cmd, - subject); - - // WHEN - subject.replicate(task); - - // THEN - assertEquals(Status.DONE, task.getStatus()); - output.assertClosedExactlyOnce(); - - // The push replicator should pass null to uploader - assertNull(replicateSizeCaptor.getValue()); - assertNull(task.getReplicateSize()); - } - - @ParameterizedTest(name = "for {0}") - @MethodSource("replicateSize") - void uploadFailsWithException(String testName, long replicateSize) throws IOException { + void uploadFailsWithException() throws IOException { // GIVEN long containerID = randomContainerID(); DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); SpyOutputStream output = new SpyOutputStream(NULL_OUTPUT_STREAM); Consumer> completion = fut -> fut.completeExceptionally(new Exception("testing")); - ArgumentCaptor replicateSizeCaptor = ArgumentCaptor.forClass(Long.class); ContainerReplicator subject = createSubject(containerID, target, - output, completion, NO_COMPRESSION, replicateSizeCaptor); - ReplicateContainerCommand cmd = toTarget(containerID, target); - cmd.setReplicateSize(replicateSize); - ReplicationTask task = new ReplicationTask(cmd, + output, completion, NO_COMPRESSION); + ReplicationTask task = new ReplicationTask(toTarget(containerID, target), subject); // WHEN @@ -173,9 +100,8 @@ void uploadFailsWithException(String testName, long replicateSize) throws IOExce output.assertClosedExactlyOnce(); } - @ParameterizedTest(name = "for {0}") - @MethodSource("replicateSize") - void packFailsWithException(String testName, long replicateSize) throws IOException { + @Test + void packFailsWithException() throws IOException { // GIVEN long containerID = randomContainerID(); DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); @@ -183,12 +109,9 @@ void packFailsWithException(String testName, long replicateSize) throws IOExcept Consumer> completion = fut -> { throw new RuntimeException(); }; - ArgumentCaptor replicateSizeCaptor = ArgumentCaptor.forClass(Long.class); ContainerReplicator subject = createSubject(containerID, target, - output, completion, NO_COMPRESSION, replicateSizeCaptor); - ReplicateContainerCommand cmd = toTarget(containerID, target); - cmd.setReplicateSize(replicateSize); - ReplicationTask task = new ReplicationTask(cmd, + output, completion, NO_COMPRESSION); + ReplicationTask task = new ReplicationTask(toTarget(containerID, target), subject); // WHEN @@ -206,8 +129,7 @@ private static long randomContainerID() { private ContainerReplicator createSubject( long containerID, DatanodeDetails target, OutputStream outputStream, Consumer> completion, - CopyContainerCompression compression, ArgumentCaptor replicateSizeCaptor - ) throws IOException { + CopyContainerCompression compression) throws IOException { ContainerReplicationSource source = mock(ContainerReplicationSource.class); ContainerUploader uploader = mock(ContainerUploader.class); ArgumentCaptor> futureArgument = @@ -217,8 +139,7 @@ private ContainerReplicator createSubject( when( uploader.startUpload(eq(containerID), eq(target), - futureArgument.capture(), eq(compression), replicateSizeCaptor.capture() - )) + futureArgument.capture(), compressionArgument.capture())) .thenReturn(outputStream); doAnswer(invocation -> { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java index cc0592b5b3e3..885eb7398941 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java @@ -95,7 +95,7 @@ void setup() throws IOException { /** * Provides stream of different container sizes for tests. */ - public static Stream replicateSizeProvider() { + public static Stream sizeProvider() { return Stream.of( Arguments.of("Null replicate size (fallback to default)", null), Arguments.of("Zero Size", 0L), @@ -127,18 +127,18 @@ void testReceiveDataForExistingContainer() throws Exception { } @ParameterizedTest(name = "for {0}") - @MethodSource("replicateSizeProvider") - public void testSpaceReservedAndReleasedWhenRequestCompleted(String testName, Long replicateSize) throws Exception { + @MethodSource("sizeProvider") + public void testSpaceReservedAndReleasedWhenRequestCompleted(String testName, Long size) throws Exception { long containerId = 1; HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); - long expectedReservedSpace = replicateSize != null ? - importer.getRequiredReplicationSpace(replicateSize) : + long expectedReservedSpace = size != null ? + importer.getRequiredReplicationSpace(size) : importer.getDefaultReplicationSpace(); // Create and execute the first request to reserve space sendContainerRequestHandler.onNext( - createRequest(containerId, ByteString.EMPTY, 0, replicateSize)); + createRequest(containerId, ByteString.EMPTY, 0, size)); // Verify commit space is reserved assertEquals(volume.getCommittedBytes(), initialCommittedBytes + expectedReservedSpace); @@ -151,19 +151,19 @@ public void testSpaceReservedAndReleasedWhenRequestCompleted(String testName, Lo } @ParameterizedTest(name = "for {0}") - @MethodSource("replicateSizeProvider") - public void testSpaceReservedAndReleasedWhenOnNextFails(String testName, Long replicateSize) throws Exception { + @MethodSource("sizeProvider") + public void testSpaceReservedAndReleasedWhenOnNextFails(String testName, Long size) throws Exception { long containerId = 1; HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); - long expectedReservedSpace = replicateSize != null ? - importer.getRequiredReplicationSpace(replicateSize) : + long expectedReservedSpace = size != null ? + importer.getRequiredReplicationSpace(size) : importer.getDefaultReplicationSpace(); ByteString data = ByteString.copyFromUtf8("test"); // Execute first request to reserve space sendContainerRequestHandler.onNext( - createRequest(containerId, data, 0, replicateSize)); + createRequest(containerId, data, 0, size)); // Verify commit space is reserved assertEquals(volume.getCommittedBytes(), initialCommittedBytes + expectedReservedSpace); @@ -172,25 +172,25 @@ public void testSpaceReservedAndReleasedWhenOnNextFails(String testName, Long re when(importer.isAllowedContainerImport(containerId)).thenReturn(false); sendContainerRequestHandler.onNext(createRequest(containerId, data, 0, - replicateSize)); + size)); // Verify commit space is released assertEquals(volume.getCommittedBytes(), initialCommittedBytes); } @ParameterizedTest(name = "for {0}") - @MethodSource("replicateSizeProvider") - public void testSpaceReservedAndReleasedWhenOnCompletedFails(String testName, Long replicateSize) throws Exception { + @MethodSource("sizeProvider") + public void testSpaceReservedAndReleasedWhenOnCompletedFails(String testName, Long size) throws Exception { long containerId = 1; HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); - long expectedReservedSpace = replicateSize != null ? - importer.getRequiredReplicationSpace(replicateSize) : + long expectedReservedSpace = size != null ? + importer.getRequiredReplicationSpace(size) : importer.getDefaultReplicationSpace(); // Execute request sendContainerRequestHandler.onNext(createRequest(containerId, - ByteString.copyFromUtf8("test"), 0, replicateSize)); + ByteString.copyFromUtf8("test"), 0, size)); // Verify commit space is reserved assertEquals(volume.getCommittedBytes(), initialCommittedBytes + expectedReservedSpace); @@ -221,7 +221,7 @@ public void testOverAllocatedReservesMoreSpace() { long overallocatedReservation = volume.getCommittedBytes() - initialCommittedBytes; handler1.onCompleted(); // Release space - // Test default container (null replicateSize) + // Test default container (null size) SendContainerRequestHandler handler2 = new SendContainerRequestHandler(importer, responseObserver, null); handler2.onNext(createRequest(containerId2, ByteString.EMPTY, 0, null)); @@ -237,7 +237,7 @@ public void testOverAllocatedReservesMoreSpace() { } private ContainerProtos.SendContainerRequest createRequest( - long containerId, ByteString data, int offset, Long replicateSize) { + long containerId, ByteString data, int offset, Long size) { ContainerProtos.SendContainerRequest.Builder builder = ContainerProtos.SendContainerRequest.newBuilder() .setContainerID(containerId) @@ -245,8 +245,8 @@ private ContainerProtos.SendContainerRequest createRequest( .setOffset(offset) .setCompression(NO_COMPRESSION.toProto()); - if (replicateSize != null) { - builder.setReplicateSize(replicateSize); + if (size != null) { + builder.setSize(size); } return builder.build(); } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index ebf0050c097f..99fcc6e271a6 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -549,7 +549,7 @@ message SendContainerRequest { required bytes data = 3; optional int64 checksum = 4; optional CopyContainerCompressProto compression = 5; - optional int64 replicateSize = 6; + optional int64 size = 6; } message SendContainerResponse { diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto index 0d93a51238bf..e48ed4d1c595 100644 --- a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto +++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto @@ -420,7 +420,6 @@ message ReplicateContainerCommandProto { optional int32 replicaIndex = 4; optional DatanodeDetailsProto target = 5; optional ReplicationCommandPriority priority = 6 [default = NORMAL]; - optional int64 replicateSize = 7; } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECMisReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECMisReplicationHandler.java index 68277faa10c3..1333efea5c35 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECMisReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECMisReplicationHandler.java @@ -83,7 +83,6 @@ protected int sendReplicateCommands( // For EC containers, we need to track the replica index which is // to be replicated, so add it to the command. cmd.setReplicaIndex(replica.getReplicaIndex()); - cmd.setReplicateSize(containerInfo.getUsedBytes()); replicationManager.sendDatanodeCommand(cmd, containerInfo, target); } commandsSent++; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java index f5be77dc4f0f..158c802479f0 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java @@ -611,7 +611,6 @@ private void createReplicateCommand( // For EC containers, we need to track the replica index which is // to be replicated, so add it to the command. replicateCommand.setReplicaIndex(replica.getReplicaIndex()); - replicateCommand.setReplicateSize(container.getUsedBytes()); replicationManager.sendDatanodeCommand(replicateCommand, container, target); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisMisReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisMisReplicationHandler.java index e82b6f2b2008..e15598ccfe8c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisMisReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisMisReplicationHandler.java @@ -73,7 +73,6 @@ protected int sendReplicateCommands( } else { ReplicateContainerCommand cmd = ReplicateContainerCommand .fromSources(containerID, sources); - cmd.setReplicateSize(containerInfo.getUsedBytes()); replicationManager.sendDatanodeCommand(cmd, containerInfo, target); } commandsSent++; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java index 4711358addb3..57eb29033e4c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java @@ -475,7 +475,6 @@ private int sendReplicationCommands( ReplicateContainerCommand command = ReplicateContainerCommand.fromSources( containerInfo.getContainerID(), sources); - command.setReplicateSize(containerInfo.getUsedBytes()); replicationManager.sendDatanodeCommand(command, containerInfo, target); commandsSent++; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java index ec7d04e6618a..33ab37a9387a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java @@ -529,7 +529,6 @@ public void sendThrottledReplicationCommand(ContainerInfo containerInfo, ReplicateContainerCommand cmd = ReplicateContainerCommand.toTarget(containerID, target); cmd.setReplicaIndex(replicaIndex); - cmd.setReplicateSize(containerInfo.getUsedBytes()); sendDatanodeCommand(cmd, containerInfo, source); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java index 73289639e3eb..68fecdb52d3e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java @@ -33,9 +33,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.function.ToLongFunction; -import java.util.stream.Collectors; import java.util.stream.IntStream; -import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port; @@ -53,8 +51,7 @@ import org.junit.jupiter.api.BeforeAll; 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.junit.jupiter.params.provider.EnumSource; /** * Tests ozone containers replication. @@ -91,32 +88,9 @@ static void tearDown() { IOUtils.closeQuietly(clientFactory, cluster); } - /** - * Creates a Cartesian product of all compression types and replicate sizes. - */ - public static Stream compressionAndReplicateSize() { - List replicateSizes = Stream.of( - Arguments.of("Null replicate size (fallback to default)", null), - Arguments.of("Zero Size", 0L), - Arguments.of("Normal 2GB", 2L * 1024L * 1024L * 1024L), - Arguments.of("Overallocated 20GB", 20L * 1024L * 1024L * 1024L) - ).collect(Collectors.toList()); - - // Create a Cartesian product of all compression types and the sizes - return Stream.of(CopyContainerCompression.values()) - .flatMap(compression -> - replicateSizes.stream().map(sizeArgs -> { - Object[] args = sizeArgs.get(); - // Arguments are: compression, testName, replicateSize - return Arguments.of(compression, args[0], args[1]); - }) - ); - } - - @ParameterizedTest(name = "{1} with {0} compression") - @MethodSource("compressionAndReplicateSize") - void testPushWithReplicateSize(CopyContainerCompression compression, String testName, Long replicateSize) - throws Exception { + @ParameterizedTest + @EnumSource + void testPush(CopyContainerCompression compression) throws Exception { final int index = compression.ordinal(); DatanodeDetails source = cluster.getHddsDatanodes().get(index) .getDatanodeDetails(); @@ -124,16 +98,14 @@ void testPushWithReplicateSize(CopyContainerCompression compression, String test DatanodeDetails target = selectOtherNode(source); ReplicateContainerCommand cmd = ReplicateContainerCommand.toTarget(containerID, target); - cmd.setReplicateSize(replicateSize); queueAndWaitForCompletion(cmd, source, ReplicationSupervisor::getReplicationSuccessCount); } - @ParameterizedTest(name = "{1} with {0} compression") - @MethodSource("compressionAndReplicateSize") - void testPullWithReplicateSize(CopyContainerCompression compression, String testName, Long replicateSize) - throws Exception { + @ParameterizedTest + @EnumSource + void testPull(CopyContainerCompression compression) throws Exception { final int index = compression.ordinal(); DatanodeDetails target = cluster.getHddsDatanodes().get(index) .getDatanodeDetails(); @@ -142,7 +114,6 @@ void testPullWithReplicateSize(CopyContainerCompression compression, String test ReplicateContainerCommand cmd = ReplicateContainerCommand.fromSources(containerID, ImmutableList.of(source)); - cmd.setReplicateSize(replicateSize); queueAndWaitForCompletion(cmd, target, ReplicationSupervisor::getReplicationSuccessCount); From e6c799ece114f453dec27c82b6541a6831b2ad1d Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Mon, 22 Sep 2025 11:46:52 +0530 Subject: [PATCH 08/10] resolved checkstyle failure --- .../ozone/container/replication/ContainerUploader.java | 3 ++- .../container/replication/DownloadAndImportReplicator.java | 4 ++-- .../ozone/protocol/commands/ReplicateContainerCommand.java | 2 -- .../replication/TestDownloadAndImportReplicator.java | 4 ++-- .../ozone/container/replication/TestPushReplicator.java | 6 ++++-- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java index 3b4de6377b85..55874511ba7c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java @@ -27,5 +27,6 @@ */ public interface ContainerUploader { OutputStream startUpload(long containerId, DatanodeDetails target, - CompletableFuture callback, CopyContainerCompression compression) throws IOException; + CompletableFuture callback, CopyContainerCompression compression) + throws IOException; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java index 09aa1e8fad6b..2457b592b141 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java @@ -73,7 +73,8 @@ public void replicate(ReplicationTask task) { HddsVolume targetVolume = null; try { - targetVolume = containerImporter.chooseNextVolume(containerImporter.getDefaultReplicationSpace()); + targetVolume = containerImporter.chooseNextVolume( + containerImporter.getDefaultReplicationSpace()); // Wait for the download. This thread pool is limiting the parallel // downloads, so it's ok to block here and wait for the full download. @@ -84,7 +85,6 @@ public void replicate(ReplicationTask task) { task.setStatus(Status.FAILED); return; } - long bytes = Files.size(tarFilePath); LOG.info("Container {} is downloaded with size {}, starting to import.", containerID, bytes); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java index 985d4163bae6..826af4fdd362 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java @@ -105,7 +105,6 @@ public ReplicateContainerCommandProto getProto() { builder.setTarget(targetDatanode.getProtoBufMessage()); } builder.setPriority(priority); - return builder.build(); } @@ -132,7 +131,6 @@ public static ReplicateContainerCommand getFromProtobuf( if (protoMessage.hasPriority()) { cmd.setPriority(protoMessage.getPriority()); } - return cmd; } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java index 3b0811a15da7..c690b50d6425 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java @@ -104,13 +104,13 @@ public void testCommitSpaceReleasedOnReplicationFailure() throws Exception { // Wait such that first container import reserve space GenericTestUtils.waitFor(() -> - volume.getCommittedBytes() > initialCommittedBytes, + volume.getCommittedBytes() > initialCommittedBytes, 1000, 50000); assertEquals(volume.getCommittedBytes(), initialCommittedBytes + 2 * containerMaxSize); semaphore.release(); GenericTestUtils.waitFor(() -> - volume.getCommittedBytes() == initialCommittedBytes, + volume.getCommittedBytes() == initialCommittedBytes, 1000, 50000); // Verify commit space is released diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java index f804dd08f323..a4463410cea5 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java @@ -129,7 +129,8 @@ private static long randomContainerID() { private ContainerReplicator createSubject( long containerID, DatanodeDetails target, OutputStream outputStream, Consumer> completion, - CopyContainerCompression compression) throws IOException { + CopyContainerCompression compression + ) throws IOException { ContainerReplicationSource source = mock(ContainerReplicationSource.class); ContainerUploader uploader = mock(ContainerUploader.class); ArgumentCaptor> futureArgument = @@ -139,7 +140,8 @@ private ContainerReplicator createSubject( when( uploader.startUpload(eq(containerID), eq(target), - futureArgument.capture(), compressionArgument.capture())) + futureArgument.capture(), compressionArgument.capture() + )) .thenReturn(outputStream); doAnswer(invocation -> { From 7e42b8b40d04819bbe3017f74ee75536eef7b968 Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Wed, 24 Sep 2025 11:53:58 +0530 Subject: [PATCH 09/10] added integration test --- .../replication/ContainerImporter.java | 1 + .../replication/GrpcContainerUploader.java | 2 + .../replication/TestContainerReplication.java | 144 ++++++++++++++++++ 3 files changed, 147 insertions(+) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java index 7ebbb4b29304..a0d81e0160ea 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java @@ -148,6 +148,7 @@ private static void deleteFileQuietely(Path tarFilePath) { HddsVolume chooseNextVolume(long spaceToReserve) throws IOException { // Choose volume that can hold both container in tmp and dest directory + LOG.info("Choosing volume to reserve space : {}", spaceToReserve); return volumeChoosingPolicy.chooseVolume( StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()), spaceToReserve); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java index 7faceed75690..1d0b466d11dd 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java @@ -65,6 +65,8 @@ public OutputStream startUpload(long containerId, DatanodeDetails target, Long containerSize = null; Container container = containerController.getContainer(containerId); if (container != null) { + LOG.info("Starting upload of container {} to {} with size {}", + containerId, target, container.getContainerData().getBytesUsed()); containerSize = container.getContainerData().getBytesUsed(); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java index 68fecdb52d3e..65e89fbfc100 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java @@ -24,6 +24,7 @@ import static org.apache.hadoop.hdds.scm.pipeline.MockPipeline.createPipeline; import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.createContainer; import static org.apache.ozone.test.GenericTestUtils.waitFor; +import static org.junit.jupiter.api.Assertions.assertEquals; import com.google.common.collect.ImmutableList; import java.io.IOException; @@ -34,9 +35,14 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.ToLongFunction; import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; @@ -44,14 +50,21 @@ import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; +import org.apache.ozone.test.GenericTestUtils; +import org.apache.ozone.test.GenericTestUtils.LogCapturer; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; /** * Tests ozone containers replication. @@ -157,6 +170,68 @@ void pushUnknownContainer() throws Exception { ReplicationSupervisor::getReplicationFailureCount); } + /** + * Provides stream of different container sizes for tests. + */ + public static Stream sizeProvider() { + return Stream.of( + Arguments.of("Normal 2MB", 2L * 1024L * 1024L), + Arguments.of("Overallocated 6MB", 6L * 1024L * 1024L) + ); + } + + /** + * Tests push replication of a container with over-allocated size. + * The target datanode will need to reserve double the container size, + * which is greater than the configured max container size. + */ + @ParameterizedTest(name = "for {0}") + @MethodSource("sizeProvider") + void testPushWithOverAllocatedContainer(String testName, Long containerSize) + throws Exception { + LogCapturer grpcLog = LogCapturer.captureLogs(GrpcContainerUploader.class); + LogCapturer containerImporterLog = LogCapturer.captureLogs(ContainerImporter.class); + + DatanodeDetails source = cluster.getHddsDatanodes().get(0) + .getDatanodeDetails(); + + long containerID = createOverAllocatedContainer(source, containerSize); + + DatanodeDetails target = selectOtherNode(source); + + // Get the original container size from source + Container sourceContainer = getContainer(source, containerID); + long originalSize = sourceContainer.getContainerData().getBytesUsed(); + + // Verify container is created with expected size + assertEquals(originalSize, containerSize); + + // Create replication command to push container to target + ReplicateContainerCommand cmd = + ReplicateContainerCommand.toTarget(containerID, target); + + // Execute push replication + queueAndWaitForCompletion(cmd, source, + ReplicationSupervisor::getReplicationSuccessCount); + + GenericTestUtils.waitFor(() -> { + String grpcLogs = grpcLog.getOutput(); + String containerImporterLogOutput = containerImporterLog.getOutput(); + + return grpcLogs.contains("Starting upload of container " + + containerID + " to " + target + " with size " + originalSize) && + containerImporterLogOutput.contains("Choosing volume to reserve space : " + + originalSize * 2); + }, 100, 1000); + + // Verify container was successfully replicated to target + Container targetContainer = getContainer(target, containerID); + long replicatedSize = targetContainer.getContainerData().getBytesUsed(); + + // verify sizes match exactly + assertEquals(originalSize, replicatedSize); + } + /** * Queues {@code cmd} in {@code dn}'s state machine, and waits until the * command is completed, as indicated by {@code counter} having been @@ -194,6 +269,8 @@ private static OzoneConfiguration createConfiguration() { OzoneConfiguration conf = new OzoneConfiguration(); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 6, TimeUnit.SECONDS); + conf.setStorageSize(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, 5, StorageUnit.MB); + conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 1, StorageUnit.MB); ReplicationManagerConfiguration repConf = conf.getObject(ReplicationManagerConfiguration.class); @@ -212,4 +289,71 @@ private static long createNewClosedContainer(DatanodeDetails dn) } } + private static long createOverAllocatedContainer(DatanodeDetails dn, Long targetDataSize) throws Exception { + long containerID = CONTAINER_ID.incrementAndGet(); + try (XceiverClientSpi client = clientFactory.acquireClient( + createPipeline(singleton(dn)))) { + + // Create the container + createContainer(client, containerID, null); + + int chunkSize = 1 * 1024 * 1024; // 1MB chunks + long totalBytesWritten = 0; + + // Write data in chunks until we reach target size + while (totalBytesWritten < targetDataSize) { + BlockID blockID = ContainerTestHelper.getTestBlockID(containerID); + + // Calculate remaining bytes and adjust chunk size if needed + long remainingBytes = targetDataSize - totalBytesWritten; + int currentChunkSize = (int) Math.min(chunkSize, remainingBytes); + + // Create a write chunk request with current chunk size + ContainerProtos.ContainerCommandRequestProto writeChunkRequest = + ContainerTestHelper.getWriteChunkRequest( + createPipeline(singleton(dn)), blockID, currentChunkSize); + + // Send write chunk command + client.sendCommand(writeChunkRequest); + + // Create and send put block command + ContainerProtos.ContainerCommandRequestProto putBlockRequest = + ContainerTestHelper.getPutBlockRequest(writeChunkRequest); + client.sendCommand(putBlockRequest); + + totalBytesWritten += currentChunkSize; + } + + // Close the container + ContainerProtos.CloseContainerRequestProto closeRequest = + ContainerProtos.CloseContainerRequestProto.newBuilder().build(); + ContainerProtos.ContainerCommandRequestProto closeContainerRequest = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.CloseContainer) + .setContainerID(containerID) + .setCloseContainer(closeRequest) + .setDatanodeUuid(dn.getUuidString()) + .build(); + client.sendCommand(closeContainerRequest); + + return containerID; + } + } + + /** + * Gets the container from the specified datanode. + */ + private Container getContainer(DatanodeDetails datanode, long containerID) { + for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes()) { + if (datanode.equals(datanodeService.getDatanodeDetails())) { + Container container = datanodeService.getDatanodeStateMachine().getContainer() + .getContainerSet().getContainer(containerID); + if (container != null) { + return container; + } + } + } + throw new AssertionError("Container " + containerID + " not found on " + datanode); + } + } From a444beca1ab3e4efdba9a726d40ea391d8f694ca Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Tue, 7 Oct 2025 12:33:10 +0530 Subject: [PATCH 10/10] changed log level to debug and used containerMaxSize --- .../hadoop/ozone/container/replication/ContainerImporter.java | 2 +- .../ozone/container/replication/GrpcContainerUploader.java | 2 +- .../container/replication/TestSendContainerRequestHandler.java | 2 +- .../ozone/container/replication/TestContainerReplication.java | 3 +++ 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java index a0d81e0160ea..7b42006b2293 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java @@ -148,7 +148,7 @@ private static void deleteFileQuietely(Path tarFilePath) { HddsVolume chooseNextVolume(long spaceToReserve) throws IOException { // Choose volume that can hold both container in tmp and dest directory - LOG.info("Choosing volume to reserve space : {}", spaceToReserve); + LOG.debug("Choosing volume to reserve space : {}", spaceToReserve); return volumeChoosingPolicy.chooseVolume( StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()), spaceToReserve); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java index 1d0b466d11dd..64adcb6c6168 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java @@ -65,7 +65,7 @@ public OutputStream startUpload(long containerId, DatanodeDetails target, Long containerSize = null; Container container = containerController.getContainer(containerId); if (container != null) { - LOG.info("Starting upload of container {} to {} with size {}", + LOG.debug("Starting upload of container {} to {} with size {}", containerId, target, container.getContainerData().getBytesUsed()); containerSize = container.getContainerData().getBytesUsed(); } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java index 885eb7398941..4fb801532f0b 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java @@ -211,7 +211,7 @@ public void testSpaceReservedAndReleasedWhenOnCompletedFails(String testName, Lo public void testOverAllocatedReservesMoreSpace() { long containerId1 = 1; long containerId2 = 2; - long overallocatedSize = 10L * 1024L * 1024L * 1024L; // 10GB + long overallocatedSize = containerMaxSize * 2; // 10GB HddsVolume volume = (HddsVolume) volumeSet.getVolumesList().get(0); long initialCommittedBytes = volume.getCommittedBytes(); // Test overallocated container (10GB) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java index 65e89fbfc100..968e331103e3 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerReplication.java @@ -65,6 +65,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.event.Level; /** * Tests ozone containers replication. @@ -189,6 +190,8 @@ public static Stream sizeProvider() { @MethodSource("sizeProvider") void testPushWithOverAllocatedContainer(String testName, Long containerSize) throws Exception { + GenericTestUtils.setLogLevel(GrpcContainerUploader.class, Level.DEBUG); + GenericTestUtils.setLogLevel(ContainerImporter.class, Level.DEBUG); LogCapturer grpcLog = LogCapturer.captureLogs(GrpcContainerUploader.class); LogCapturer containerImporterLog = LogCapturer.captureLogs(ContainerImporter.class);