This is an automated email from the ASF dual-hosted git repository.

adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 60bcdafa73 HDDS-10292. Extract configurator for size settings from 
MiniOzoneCluster (#6167)
60bcdafa73 is described below

commit 60bcdafa73affe68d3ebed5a74e7b54a414efcf4
Author: Doroszlai, Attila <[email protected]>
AuthorDate: Tue Feb 6 21:57:31 2024 +0100

    HDDS-10292. Extract configurator for size settings from MiniOzoneCluster 
(#6167)
---
 .../apache/hadoop/ozone/MiniOzoneChaosCluster.java |   2 +-
 .../java/org/apache/hadoop/fs/ozone/TestHSync.java |  21 ++--
 .../apache/hadoop/fs/ozone/TestLeaseRecovery.java  |  21 ++--
 .../hadoop/fs/ozone/TestOzoneFSInputStream.java    |  14 ++-
 .../fs/ozone/TestOzoneFileSystemMetrics.java       |  14 ++-
 .../fs/ozone/TestOzoneFileSystemWithStreaming.java |  22 ++--
 .../hadoop/hdds/scm/storage/TestCommitWatcher.java |  17 +--
 .../hadoop/ozone/ClientConfigForTesting.java       | 135 +++++++++++++++++++++
 .../org/apache/hadoop/ozone/MiniOzoneCluster.java  |  71 -----------
 .../apache/hadoop/ozone/MiniOzoneClusterImpl.java  |  55 ---------
 .../client/rpc/AbstractTestECKeyOutputStream.java  |  16 ++-
 .../ozone/client/rpc/Test2WayCommitInRatis.java    |  13 +-
 .../client/rpc/TestBlockDataStreamOutput.java      |  17 +--
 .../ozone/client/rpc/TestBlockOutputStream.java    |  13 +-
 .../rpc/TestCloseContainerHandlingByClient.java    |   2 +-
 .../rpc/TestContainerStateMachineFlushDelay.java   |  16 ++-
 .../rpc/TestContainerStateMachineStream.java       |  17 +--
 .../client/rpc/TestDiscardPreallocatedBlocks.java  |   2 +-
 .../rpc/TestFailureHandlingByClientFlushDelay.java |  17 ++-
 .../client/rpc/TestOzoneAtRestEncryption.java      |  12 +-
 ...estOzoneClientRetriesOnExceptionFlushDelay.java |  16 ++-
 .../rpc/TestOzoneClientRetriesOnExceptions.java    |  13 +-
 .../client/rpc/TestOzoneRpcClientAbstract.java     |   9 +-
 .../ozone/client/rpc/TestWatchForCommit.java       |  12 +-
 .../ozone/client/rpc/read/TestInputStreamBase.java |  15 ++-
 .../ozone/container/TestECContainerRecovery.java   |  17 ++-
 .../src/test/resources/ozone-site.xml              |  34 ++++++
 27 files changed, 369 insertions(+), 244 deletions(-)

diff --git 
a/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java
 
b/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java
index f212570fc5..26f896663b 100644
--- 
a/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java
+++ 
b/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java
@@ -232,7 +232,7 @@ public class MiniOzoneChaosCluster extends 
MiniOzoneHAClusterImpl {
     protected void initializeConfiguration() throws IOException {
       super.initializeConfiguration();
 
-      OzoneClientConfig clientConfig = new OzoneClientConfig();
+      OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
       clientConfig.setStreamBufferFlushSize(8 * 1024 * 1024);
       clientConfig.setStreamBufferMaxSize(16 * 1024 * 1024);
       clientConfig.setStreamBufferSize(4 * 1024);
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java
index 559b8da498..05d297d38e 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestHSync.java
@@ -27,9 +27,6 @@ import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.hdds.utils.IOUtils;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoCodec;
 import org.apache.hadoop.crypto.CryptoOutputStream;
@@ -37,15 +34,19 @@ import org.apache.hadoop.crypto.Encryptor;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.hdds.utils.IOUtils;
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -116,17 +117,19 @@ public class TestHSync {
     CONF.setBoolean(OZONE_OM_RATIS_ENABLE_KEY, false);
     CONF.set(OZONE_DEFAULT_BUCKET_LAYOUT, layout.name());
     CONF.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);
-    cluster = MiniOzoneCluster.newBuilder(CONF)
-        .setNumDatanodes(5)
-        .setTotalPipelineNumLimit(10)
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(blockSize)
         .setChunkSize(chunkSize)
         .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setDataStreamBufferFlushize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .setDataStreamBufferFlushSize(maxFlushSize)
         .setDataStreamMinPacketSize(chunkSize)
-        .setDataStreamStreamWindowSize(5 * chunkSize)
+        .setDataStreamWindowSize(5 * chunkSize)
+        .applyTo(CONF);
+
+    cluster = MiniOzoneCluster.newBuilder(CONF)
+        .setNumDatanodes(5)
+        .setTotalPipelineNumLimit(10)
         .build();
     cluster.waitForClusterToBeReady();
     client = cluster.newClient();
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java
index 1e61c9026b..ca68aad455 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java
@@ -17,14 +17,15 @@
  */
 package org.apache.hadoop.fs.ozone;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.TestDataUtil;
@@ -91,17 +92,19 @@ public class TestLeaseRecovery {
     conf.setBoolean(OZONE_OM_RATIS_ENABLE_KEY, false);
     conf.setBoolean(OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);
     conf.set(OZONE_DEFAULT_BUCKET_LAYOUT, layout.name());
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
+        .setBlockSize(blockSize)
+        .setChunkSize(chunkSize)
+        .setStreamBufferFlushSize(flushSize)
+        .setStreamBufferMaxSize(maxFlushSize)
+        .setDataStreamBufferFlushSize(maxFlushSize)
+        .setDataStreamMinPacketSize(chunkSize)
+        .setDataStreamWindowSize(5 * chunkSize)
+        .applyTo(conf);
+
     cluster = MiniOzoneCluster.newBuilder(conf)
       .setNumDatanodes(5)
       .setTotalPipelineNumLimit(10)
-      .setBlockSize(blockSize)
-      .setChunkSize(chunkSize)
-      .setStreamBufferFlushSize(flushSize)
-      .setStreamBufferMaxSize(maxFlushSize)
-      .setDataStreamBufferFlushize(maxFlushSize)
-      .setStreamBufferSizeUnit(StorageUnit.BYTES)
-      .setDataStreamMinPacketSize(chunkSize)
-      .setDataStreamStreamWindowSize(5 * chunkSize)
       .build();
     cluster.waitForClusterToBeReady();
     client = cluster.newClient();
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
index 2e00e8c2c6..c6893c57e9 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
@@ -35,9 +35,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.TestDataUtil;
@@ -83,12 +85,16 @@ public class TestOzoneFSInputStream {
     conf = new OzoneConfiguration();
     conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
         BucketLayout.LEGACY.name());
+
+    ClientConfigForTesting.newBuilder(StorageUnit.MB)
+        .setChunkSize(2)
+        .setBlockSize(8)
+        .setStreamBufferFlushSize(2)
+        .setStreamBufferMaxSize(4)
+        .applyTo(conf);
+
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(5)
-        .setChunkSize(2) // MB
-        .setBlockSize(8) // MB
-        .setStreamBufferFlushSize(2) // MB
-        .setStreamBufferMaxSize(4) // MB
         .build();
     cluster.waitForClusterToBeReady();
     client = cluster.newClient();
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemMetrics.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemMetrics.java
index d11ea5376d..5aba83bd41 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemMetrics.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemMetrics.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.ozone;
 
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.utils.IOUtils;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -25,6 +26,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.TestDataUtil;
@@ -72,12 +74,16 @@ public class TestOzoneFileSystemMetrics {
     conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
         BucketLayout.LEGACY.name());
     conf.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS, true);
+
+    ClientConfigForTesting.newBuilder(StorageUnit.MB)
+        .setChunkSize(2)
+        .setBlockSize(8)
+        .setStreamBufferFlushSize(2)
+        .setStreamBufferMaxSize(4)
+        .applyTo(conf);
+
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
-        .setChunkSize(2) // MB
-        .setBlockSize(8) // MB
-        .setStreamBufferFlushSize(2) // MB
-        .setStreamBufferMaxSize(4) // MB
         .build();
     cluster.waitForClusterToBeReady();
     client = cluster.newClient();
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemWithStreaming.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemWithStreaming.java
index 9d5009eeb3..2a6c8c456b 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemWithStreaming.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemWithStreaming.java
@@ -22,14 +22,15 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.concurrent.ThreadLocalRandom;
 
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.hdds.utils.IOUtils;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.TestDataUtil;
 import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -86,17 +87,20 @@ public class TestOzoneFileSystemWithStreaming {
     CONF.set(OZONE_FS_DATASTREAM_AUTO_THRESHOLD, AUTO_THRESHOLD + "B");
     CONF.setBoolean(OZONE_OM_RATIS_ENABLE_KEY, true);
     CONF.set(OZONE_DEFAULT_BUCKET_LAYOUT, layout.name());
-    cluster = MiniOzoneCluster.newBuilder(CONF)
-        .setNumDatanodes(5)
-        .setTotalPipelineNumLimit(10)
+
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(blockSize)
         .setChunkSize(chunkSize)
         .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setDataStreamBufferFlushize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .setDataStreamBufferFlushSize(maxFlushSize)
         .setDataStreamMinPacketSize(chunkSize)
-        .setDataStreamStreamWindowSize(5 * chunkSize)
+        .setDataStreamWindowSize(5 * chunkSize)
+        .applyTo(CONF);
+
+    cluster = MiniOzoneCluster.newBuilder(CONF)
+        .setNumDatanodes(5)
+        .setTotalPipelineNumLimit(10)
         .build();
     cluster.waitForClusterToBeReady();
     client = cluster.newClient();
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestCommitWatcher.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestCommitWatcher.java
index d4581d1ad4..563e0162ac 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestCommitWatcher.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestCommitWatcher.java
@@ -25,10 +25,10 @@ import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
@@ -45,6 +45,7 @@ import 
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import 
org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -127,21 +128,23 @@ public class TestCommitWatcher {
     ratisClientConfig.setWatchRequestTimeout(Duration.ofSeconds(10));
     conf.setFromObject(ratisClientConfig);
 
-    OzoneClientConfig clientConfig = new OzoneClientConfig();
+    OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
     clientConfig.setChecksumType(ChecksumType.NONE);
     conf.setFromObject(clientConfig);
 
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
+        .setBlockSize(blockSize)
+        .setChunkSize(chunkSize)
+        .setStreamBufferFlushSize(flushSize)
+        .setStreamBufferMaxSize(maxFlushSize)
+        .applyTo(conf);
+
     conf.setQuietMode(false);
     conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4,
         StorageUnit.MB);
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(5)
         .setTotalPipelineNumLimit(3)
-        .setBlockSize(blockSize)
-        .setChunkSize(chunkSize)
-        .setStreamBufferFlushSize(flushSize)
-        .setStreamBufferMaxSize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
         .build();
     cluster.waitForClusterToBeReady();
     //the easiest way to create an open container is creating a key
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ClientConfigForTesting.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ClientConfigForTesting.java
new file mode 100644
index 0000000000..d436a65dab
--- /dev/null
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ClientConfigForTesting.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone;
+
+import org.apache.hadoop.hdds.conf.MutableConfigurationSource;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+
+import static 
org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
+
+/**
+ * Helper for tests that want to set client stream properties.
+ */
+public final class ClientConfigForTesting {
+
+  private int chunkSize = 1024 * 1024;
+  private Long blockSize;
+  private Integer streamBufferSize;
+  private Long streamBufferFlushSize;
+  private Long dataStreamBufferFlushSize;
+  private Long dataStreamWindowSize;
+  private Long streamBufferMaxSize;
+  private Integer dataStreamMinPacketSize;
+  private final StorageUnit unit;
+
+  /**
+   * @param unit Defines the unit in which size properties will be passed to 
the builder.
+   * All sizes are stored internally converted to {@link StorageUnit#BYTES}.
+   */
+  public static ClientConfigForTesting newBuilder(StorageUnit unit) {
+    return new ClientConfigForTesting(unit);
+  }
+
+  private ClientConfigForTesting(StorageUnit unit) {
+    this.unit = unit;
+  }
+
+  public ClientConfigForTesting setChunkSize(int size) {
+    chunkSize = (int) toBytes(size);
+    return this;
+  }
+
+  public ClientConfigForTesting setBlockSize(long size) {
+    blockSize = toBytes(size);
+    return this;
+  }
+
+  @SuppressWarnings("unused") // kept for completeness
+  public ClientConfigForTesting setStreamBufferSize(int size) {
+    streamBufferSize = (int) toBytes(size);
+    return this;
+  }
+
+  public ClientConfigForTesting setStreamBufferFlushSize(long size) {
+    streamBufferFlushSize = toBytes(size);
+    return this;
+  }
+
+  public ClientConfigForTesting setStreamBufferMaxSize(long size) {
+    streamBufferMaxSize = toBytes(size);
+    return this;
+  }
+
+  public ClientConfigForTesting setDataStreamMinPacketSize(int size) {
+    dataStreamMinPacketSize = (int) toBytes(size);
+    return this;
+  }
+
+  public ClientConfigForTesting setDataStreamBufferFlushSize(long size) {
+    dataStreamBufferFlushSize = toBytes(size);
+    return this;
+  }
+
+  public ClientConfigForTesting setDataStreamWindowSize(long size) {
+    dataStreamWindowSize = toBytes(size);
+    return this;
+  }
+
+  public void applyTo(MutableConfigurationSource conf) {
+    if (streamBufferSize == null) {
+      streamBufferSize = chunkSize;
+    }
+    if (streamBufferFlushSize == null) {
+      streamBufferFlushSize = (long) chunkSize;
+    }
+    if (streamBufferMaxSize == null) {
+      streamBufferMaxSize = 2 * streamBufferFlushSize;
+    }
+    if (dataStreamBufferFlushSize == null) {
+      dataStreamBufferFlushSize = 4L * chunkSize;
+    }
+    if (dataStreamMinPacketSize == null) {
+      dataStreamMinPacketSize = chunkSize / 4;
+    }
+    if (dataStreamWindowSize == null) {
+      dataStreamWindowSize = 8L * chunkSize;
+    }
+    if (blockSize == null) {
+      blockSize = 2 * streamBufferMaxSize;
+    }
+
+    OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+    clientConfig.setStreamBufferSize(streamBufferSize);
+    clientConfig.setStreamBufferMaxSize(streamBufferMaxSize);
+    clientConfig.setStreamBufferFlushSize(streamBufferFlushSize);
+    clientConfig.setDataStreamBufferFlushSize(dataStreamBufferFlushSize);
+    clientConfig.setDataStreamMinPacketSize(dataStreamMinPacketSize);
+    clientConfig.setStreamWindowSize(dataStreamWindowSize);
+
+    conf.setFromObject(clientConfig);
+    conf.setStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, chunkSize, 
StorageUnit.BYTES);
+    conf.setStorageSize(OZONE_SCM_BLOCK_SIZE, blockSize, StorageUnit.BYTES);
+  }
+
+  private long toBytes(long value) {
+    return Math.round(unit.toBytes(value));
+  }
+
+}
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
index 9afc8ce634..e864cae00b 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
@@ -20,11 +20,9 @@ package org.apache.hadoop.ozone;
 import java.io.IOException;
 import java.util.List;
 import java.util.Optional;
-import java.util.OptionalInt;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -325,15 +323,6 @@ public interface MiniOzoneCluster extends AutoCloseable {
     protected String omId = UUID.randomUUID().toString();
     
     protected Optional<String> datanodeReservedSpace = Optional.empty();
-    protected Optional<Integer> chunkSize = Optional.empty();
-    protected OptionalInt streamBufferSize = OptionalInt.empty();
-    protected Optional<Long> streamBufferFlushSize = Optional.empty();
-    protected Optional<Long> dataStreamBufferFlushSize = Optional.empty();
-    protected Optional<Long> datastreamWindowSize = Optional.empty();
-    protected Optional<Long> streamBufferMaxSize = Optional.empty();
-    protected OptionalInt dataStreamMinPacketSize = OptionalInt.empty();
-    protected Optional<Long> blockSize = Optional.empty();
-    protected Optional<StorageUnit> streamBufferSizeUnit = Optional.empty();
     protected boolean includeRecon = false;
 
 
@@ -489,61 +478,6 @@ public interface MiniOzoneCluster extends AutoCloseable {
       return this;
     }
 
-    /**
-     * Sets the chunk size.
-     *
-     * @return MiniOzoneCluster.Builder
-     */
-    public Builder setChunkSize(int size) {
-      chunkSize = Optional.of(size);
-      return this;
-    }
-
-    /**
-     * Sets the flush size for stream buffer.
-     *
-     * @return MiniOzoneCluster.Builder
-     */
-    public Builder setStreamBufferFlushSize(long size) {
-      streamBufferFlushSize = Optional.of(size);
-      return this;
-    }
-
-    /**
-     * Sets the max size for stream buffer.
-     *
-     * @return MiniOzoneCluster.Builder
-     */
-    public Builder setStreamBufferMaxSize(long size) {
-      streamBufferMaxSize = Optional.of(size);
-      return this;
-    }
-
-    public Builder setDataStreamBufferFlushize(long size) {
-      dataStreamBufferFlushSize = Optional.of(size);
-      return this;
-    }
-
-    public Builder setDataStreamMinPacketSize(int size) {
-      dataStreamMinPacketSize = OptionalInt.of(size);
-      return this;
-    }
-
-    public Builder setDataStreamStreamWindowSize(long size) {
-      datastreamWindowSize = Optional.of(size);
-      return this;
-    }
-
-    /**
-     * Sets the block size for stream buffer.
-     *
-     * @return MiniOzoneCluster.Builder
-     */
-    public Builder setBlockSize(long size) {
-      blockSize = Optional.of(size);
-      return this;
-    }
-
     public Builder setNumOfOzoneManagers(int numOMs) {
       this.numOfOMs = numOMs;
       return this;
@@ -554,11 +488,6 @@ public interface MiniOzoneCluster extends AutoCloseable {
       return this;
     }
 
-    public Builder setStreamBufferSizeUnit(StorageUnit unit) {
-      this.streamBufferSizeUnit = Optional.of(unit);
-      return this;
-    }
-
     public Builder setOMServiceId(String serviceId) {
       this.omServiceId = serviceId;
       return this;
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index ec1b118330..400ae3ee2c 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -26,8 +26,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-import java.util.Optional;
-import java.util.OptionalInt;
 import java.util.Set;
 import java.util.UUID;
 import java.util.Iterator;
@@ -36,7 +34,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.DFSConfigKeysLegacy;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
@@ -45,7 +42,6 @@ import org.apache.hadoop.hdds.conf.ConfigurationTarget;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.HddsTestUtils;
 import org.apache.hadoop.hdds.scm.ha.SCMHANodeDetails;
@@ -655,58 +651,7 @@ public class MiniOzoneClusterImpl implements 
MiniOzoneCluster {
       Files.createDirectories(metaDir);
       conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.toString());
       // conf.setBoolean(ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY, true);
-      if (!chunkSize.isPresent()) {
-        //set it to 1MB by default in tests
-        chunkSize = Optional.of(1);
-      }
-      if (!streamBufferSize.isPresent()) {
-        streamBufferSize = OptionalInt.of(chunkSize.get());
-      }
-      if (!streamBufferFlushSize.isPresent()) {
-        streamBufferFlushSize = Optional.of((long) chunkSize.get());
-      }
-      if (!streamBufferMaxSize.isPresent()) {
-        streamBufferMaxSize = Optional.of(2 * streamBufferFlushSize.get());
-      }
-      if (!dataStreamBufferFlushSize.isPresent()) {
-        dataStreamBufferFlushSize = Optional.of((long) 4 * chunkSize.get());
-      }
-      if (!dataStreamMinPacketSize.isPresent()) {
-        dataStreamMinPacketSize = OptionalInt.of(chunkSize.get() / 4);
-      }
-      if (!datastreamWindowSize.isPresent()) {
-        datastreamWindowSize = Optional.of((long) 8 * chunkSize.get());
-      }
-      if (!blockSize.isPresent()) {
-        blockSize = Optional.of(2 * streamBufferMaxSize.get());
-      }
-
-      if (!streamBufferSizeUnit.isPresent()) {
-        streamBufferSizeUnit = Optional.of(StorageUnit.MB);
-      }
 
-      OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
-      clientConfig.setStreamBufferSize(
-          (int) Math.round(
-              
streamBufferSizeUnit.get().toBytes(streamBufferSize.getAsInt())));
-      clientConfig.setStreamBufferMaxSize(Math.round(
-          streamBufferSizeUnit.get().toBytes(streamBufferMaxSize.get())));
-      clientConfig.setStreamBufferFlushSize(Math.round(
-          streamBufferSizeUnit.get().toBytes(streamBufferFlushSize.get())));
-      clientConfig.setDataStreamBufferFlushSize(Math.round(
-          
streamBufferSizeUnit.get().toBytes(dataStreamBufferFlushSize.get())));
-      clientConfig.setDataStreamMinPacketSize((int) Math.round(
-          streamBufferSizeUnit.get()
-              .toBytes(dataStreamMinPacketSize.getAsInt())));
-      clientConfig.setStreamWindowSize(Math.round(
-          streamBufferSizeUnit.get().toBytes(datastreamWindowSize.get())));
-      conf.setFromObject(clientConfig);
-
-      conf.setStorageSize(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
-          chunkSize.get(), streamBufferSizeUnit.get());
-
-      conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 
blockSize.get(),
-          streamBufferSizeUnit.get());
       // MiniOzoneCluster should have global pipeline upper limit.
       conf.setInt(ScmConfigKeys.OZONE_SCM_RATIS_PIPELINE_LIMIT,
           pipelineNumLimit >= DEFAULT_PIPELINE_LIMIT ?
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java
index 3478489edd..0b0b2586c9 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/AbstractTestECKeyOutputStream.java
@@ -16,13 +16,13 @@
  */
 package org.apache.hadoop.ozone.client.rpc;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -31,6 +31,7 @@ import 
org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.BucketArgs;
@@ -120,11 +121,16 @@ abstract class AbstractTestECKeyOutputStream {
         TimeUnit.SECONDS);
     conf.setBoolean(OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED,
         zeroCopyEnabled);
-    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(10)
-        .setTotalPipelineNumLimit(10).setBlockSize(blockSize)
-        .setChunkSize(chunkSize).setStreamBufferFlushSize(flushSize)
+
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
+        .setBlockSize(blockSize)
+        .setChunkSize(chunkSize)
+        .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES).build();
+        .applyTo(conf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(10)
+        .setTotalPipelineNumLimit(10).build();
     cluster.waitForClusterToBeReady();
     client = OzoneClientFactory.getRpcClient(conf);
     objectStore = client.getObjectStore();
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/Test2WayCommitInRatis.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/Test2WayCommitInRatis.java
index ce89e679df..e7c8be170c 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/Test2WayCommitInRatis.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/Test2WayCommitInRatis.java
@@ -17,10 +17,10 @@
 
 package org.apache.hadoop.ozone.client.rpc;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.protocolPB.
     StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -100,14 +101,16 @@ public class Test2WayCommitInRatis {
     raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(10));
     conf.setFromObject(raftClientConfig);
 
-    conf.setQuietMode(false);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(3)
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(blockSize)
         .setChunkSize(chunkSize)
         .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .applyTo(conf);
+
+    conf.setQuietMode(false);
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(3)
         .build();
     cluster.waitForClusterToBeReady();
     // the easiest way to create an open container is creating a key
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockDataStreamOutput.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockDataStreamOutput.java
index d3caf62387..8bb791bb10 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockDataStreamOutput.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockDataStreamOutput.java
@@ -17,9 +17,9 @@
 
 package org.apache.hadoop.ozone.client.rpc;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientMetrics;
 import org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput;
 import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.ObjectStore;
@@ -92,17 +93,19 @@ public class TestBlockDataStreamOutput {
     conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4,
         StorageUnit.MB);
 
-    cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(5)
-        .setTotalPipelineNumLimit(3)
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(blockSize)
         .setChunkSize(chunkSize)
         .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setDataStreamBufferFlushize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .setDataStreamBufferFlushSize(maxFlushSize)
         .setDataStreamMinPacketSize(chunkSize)
-        .setDataStreamStreamWindowSize(5 * chunkSize)
+        .setDataStreamWindowSize(5 * chunkSize)
+        .applyTo(conf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(5)
+        .setTotalPipelineNumLimit(3)
         .build();
     cluster.waitForClusterToBeReady();
     //the easiest way to create an open container is creating a key
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java
index 74686d363c..1e9cefbaa4 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java
@@ -24,12 +24,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.lang3.RandomUtils;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.hdds.scm.XceiverClientMetrics;
 import org.apache.hadoop.hdds.scm.storage.BufferPool;
 import org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneClient;
@@ -105,14 +106,16 @@ class TestBlockOutputStream {
     ratisClientConfig.setWatchRequestTimeout(Duration.ofSeconds(30));
     conf.setFromObject(ratisClientConfig);
 
-    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(5)
-        .setTotalPipelineNumLimit(3)
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(BLOCK_SIZE)
         .setChunkSize(CHUNK_SIZE)
         .setStreamBufferFlushSize(FLUSH_SIZE)
         .setStreamBufferMaxSize(MAX_FLUSH_SIZE)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .applyTo(conf);
+
+    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(5)
+        .setTotalPipelineNumLimit(3)
         .build();
     cluster.waitForClusterToBeReady();
 
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
index f2a5748bff..9609dea048 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
@@ -89,7 +89,7 @@ public class TestCloseContainerHandlingByClient {
     chunkSize = (int) OzoneConsts.MB;
     blockSize = 4 * chunkSize;
 
-    OzoneClientConfig config = new OzoneClientConfig();
+    OzoneClientConfig config = conf.getObject(OzoneClientConfig.class);
     config.setChecksumType(ChecksumType.NONE);
     conf.setFromObject(config);
 
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFlushDelay.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFlushDelay.java
index 32fc9ba5c9..23ab89b80c 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFlushDelay.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFlushDelay.java
@@ -17,16 +17,17 @@
 
 package org.apache.hadoop.ozone.client.rpc;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import 
org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClientTestImpl;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.ObjectStore;
@@ -108,13 +109,16 @@ public class TestContainerStateMachineFlushDelay {
     conf.set(OzoneConfigKeys.OZONE_SCM_CLOSE_CONTAINER_WAIT_DURATION, "2s");
     conf.set(ScmConfigKeys.OZONE_SCM_PIPELINE_SCRUB_INTERVAL, "2s");
     conf.set(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, "5s");
+
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
+        .setBlockSize(blockSize)
+        .setChunkSize(chunkSize)
+        .setStreamBufferFlushSize(flushSize)
+        .setStreamBufferMaxSize(maxFlushSize)
+        .applyTo(conf);
+
     cluster =
         MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1)
-            .setBlockSize(blockSize)
-            .setChunkSize(chunkSize)
-            .setStreamBufferFlushSize(flushSize)
-            .setStreamBufferMaxSize(maxFlushSize)
-            .setStreamBufferSizeUnit(StorageUnit.BYTES)
             .setHbInterval(200)
             .setCertificateClient(new CertificateClientTestImpl(conf))
             .setSecretKeyClient(new SecretKeyTestClient())
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineStream.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineStream.java
index ab2fbeadb6..97a3047bfd 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineStream.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineStream.java
@@ -17,14 +17,15 @@
 
 package org.apache.hadoop.ozone.client.rpc;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -112,18 +113,20 @@ public class TestContainerStateMachineStream {
     raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(10));
     conf.setFromObject(raftClientConfig);
 
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
+        .setDataStreamMinPacketSize(1024)
+        .setBlockSize(BLOCK_SIZE)
+        .setChunkSize(CHUNK_SIZE)
+        .setStreamBufferFlushSize(FLUSH_SIZE)
+        .setStreamBufferMaxSize(MAX_FLUSH_SIZE)
+        .applyTo(conf);
+
     conf.setLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY, 1);
     conf.setQuietMode(false);
     cluster =
         MiniOzoneCluster.newBuilder(conf)
             .setNumDatanodes(3)
             .setHbInterval(200)
-            .setDataStreamMinPacketSize(1024)
-            .setBlockSize(BLOCK_SIZE)
-            .setChunkSize(CHUNK_SIZE)
-            .setStreamBufferFlushSize(FLUSH_SIZE)
-            .setStreamBufferMaxSize(MAX_FLUSH_SIZE)
-            .setStreamBufferSizeUnit(StorageUnit.BYTES)
             .build();
     cluster.waitForClusterToBeReady();
     cluster.waitForPipelineTobeReady(HddsProtos.ReplicationFactor.ONE, 60000);
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDiscardPreallocatedBlocks.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDiscardPreallocatedBlocks.java
index c689a692ae..4789158682 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDiscardPreallocatedBlocks.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDiscardPreallocatedBlocks.java
@@ -85,7 +85,7 @@ public class TestDiscardPreallocatedBlocks {
     chunkSize = (int) OzoneConsts.MB;
     blockSize = 4 * chunkSize;
 
-    OzoneClientConfig config = new OzoneClientConfig();
+    OzoneClientConfig config = conf.getObject(OzoneClientConfig.class);
     config.setChecksumType(ChecksumType.NONE);
     conf.setFromObject(config);
 
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClientFlushDelay.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClientFlushDelay.java
index 41438996c2..fadc06bd57 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClientFlushDelay.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClientFlushDelay.java
@@ -17,13 +17,13 @@
 
 package org.apache.hadoop.ozone.client.rpc;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdds.utils.IOUtils;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.StaticMapping;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.ObjectStore;
@@ -129,14 +130,18 @@ public class TestFailureHandlingByClientFlushDelay {
     StaticMapping.addNodeToRack(NetUtils.normalizeHostNames(
             Collections.singleton(HddsUtils.getHostName(conf))).get(0),
         "/rack1");
-    cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(10)
-        .setTotalPipelineNumLimit(15)
-        .setChunkSize(chunkSize)
+
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(blockSize)
+        .setChunkSize(chunkSize)
         .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES).build();
+        .applyTo(conf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(10)
+        .setTotalPipelineNumLimit(15)
+        .build();
     cluster.waitForClusterToBeReady();
     //the easiest way to create an open container is creating a key
     client = OzoneClientFactory.getRpcClient(conf);
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneAtRestEncryption.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneAtRestEncryption.java
index bc328d5315..919654d82a 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneAtRestEncryption.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneAtRestEncryption.java
@@ -34,7 +34,6 @@ import java.util.TreeMap;
 import java.util.UUID;
 
 import com.google.common.cache.Cache;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.server.MiniKMS;
@@ -44,11 +43,13 @@ import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import 
org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
 import 
org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClientTestImpl;
 import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.BucketArgs;
@@ -140,11 +141,14 @@ class TestOzoneAtRestEncryption {
     conf.set(OZONE_METADATA_DIRS, testDir.getAbsolutePath());
     CertificateClientTestImpl certificateClientTest =
         new CertificateClientTestImpl(conf);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(10)
+
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(BLOCK_SIZE)
         .setChunkSize(CHUNK_SIZE)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .applyTo(conf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(10)
         .setCertificateClient(certificateClientTest)
         .setSecretKeyClient(new SecretKeyTestClient())
         .build();
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptionFlushDelay.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptionFlushDelay.java
index 925cfd9d95..c3e8a8d461 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptionFlushDelay.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptionFlushDelay.java
@@ -20,10 +20,10 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.UUID;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.ObjectStore;
@@ -88,7 +89,7 @@ public class TestOzoneClientRetriesOnExceptionFlushDelay {
     maxFlushSize = 2 * flushSize;
     blockSize = 2 * maxFlushSize;
 
-    OzoneClientConfig config = new OzoneClientConfig();
+    OzoneClientConfig config = conf.getObject(OzoneClientConfig.class);
     config.setChecksumType(ChecksumType.NONE);
     config.setMaxRetryCount(3);
     conf.setFromObject(config);
@@ -98,14 +99,17 @@ public class TestOzoneClientRetriesOnExceptionFlushDelay {
     conf.set(ScmConfigKeys.OZONE_SCM_PIPELINE_SCRUB_INTERVAL, "2s");
     conf.set(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, "5s");
     conf.setQuietMode(false);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(5)
-        .setTotalPipelineNumLimit(3)
+
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(blockSize)
         .setChunkSize(chunkSize)
         .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .applyTo(conf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(5)
+        .setTotalPipelineNumLimit(3)
         .build();
     cluster.waitForClusterToBeReady();
     //the easiest way to create an open container is creating a key
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptions.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptions.java
index 72b6880c17..cd99382f30 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptions.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptions.java
@@ -22,10 +22,10 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
@@ -38,6 +38,7 @@ import 
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenExcep
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.ObjectStore;
@@ -108,14 +109,16 @@ public class TestOzoneClientRetriesOnExceptions {
     conf.set(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, "5s");
     conf.setQuietMode(false);
 
-    cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(5)
-        .setTotalPipelineNumLimit(3)
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(blockSize)
         .setChunkSize(chunkSize)
         .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .applyTo(conf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(5)
+        .setTotalPipelineNumLimit(3)
         .build();
     cluster.waitForClusterToBeReady();
     //the easiest way to create an open container is creating a key
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
index 2adb310552..a87d05321e 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
@@ -38,7 +38,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Stream;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec;
@@ -49,6 +48,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 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.StorageType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -62,6 +62,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import 
org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OmUtils;
@@ -202,10 +203,14 @@ public abstract class TestOzoneRpcClientAbstract {
     // Reduce long wait time in MiniOzoneClusterImpl#waitForHddsDatanodesStop
     //  for testZReadKeyWithUnhealthyContainerReplica.
     conf.set("ozone.scm.stale.node.interval", "10s");
+
+    ClientConfigForTesting.newBuilder(StorageUnit.MB)
+        .setDataStreamMinPacketSize(1)
+        .applyTo(conf);
+
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(14)
         .setTotalPipelineNumLimit(10)
-        .setDataStreamMinPacketSize(1) // 1MB
         .build();
     cluster.waitForClusterToBeReady();
     ozClient = OzoneClientFactory.getRpcClient(conf);
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestWatchForCommit.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestWatchForCommit.java
index 69a1032015..f8e9b552e3 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestWatchForCommit.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestWatchForCommit.java
@@ -28,10 +28,10 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
@@ -48,6 +48,7 @@ import 
org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolCli
 import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -140,13 +141,16 @@ public class TestWatchForCommit {
     conf.setFromObject(raftClientConfig);
 
     conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 30, TimeUnit.SECONDS);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(9)
+
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(blockSize)
         .setChunkSize(chunkSize)
         .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .applyTo(conf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(9)
         .build();
     cluster.waitForClusterToBeReady();
     cluster.waitForPipelineTobeReady(HddsProtos.ReplicationFactor.THREE, 
60000);
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
index 22ad4f036c..9f5d04c56f 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
@@ -20,11 +20,12 @@ import java.time.Duration;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import 
org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
@@ -45,7 +46,7 @@ abstract class TestInputStreamBase {
       ContainerLayoutVersion containerLayout) throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
 
-    OzoneClientConfig config = new OzoneClientConfig();
+    OzoneClientConfig config = conf.getObject(OzoneClientConfig.class);
     config.setBytesPerChecksum(BYTES_PER_CHECKSUM);
     conf.setFromObject(config);
 
@@ -63,14 +64,16 @@ abstract class TestInputStreamBase {
     repConf.setInterval(Duration.ofSeconds(1));
     conf.setFromObject(repConf);
 
-    return MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(5)
-        .setTotalPipelineNumLimit(5)
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
         .setBlockSize(BLOCK_SIZE)
         .setChunkSize(CHUNK_SIZE)
         .setStreamBufferFlushSize(FLUSH_SIZE)
         .setStreamBufferMaxSize(MAX_FLUSH_SIZE)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES)
+        .applyTo(conf);
+
+    return MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(5)
+        .setTotalPipelineNumLimit(5)
         .build();
   }
 
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestECContainerRecovery.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestECContainerRecovery.java
index 5a7e404f50..e045b48bda 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestECContainerRecovery.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestECContainerRecovery.java
@@ -16,11 +16,11 @@
  */
 package org.apache.hadoop.ozone.container;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
 import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
@@ -34,6 +34,7 @@ import 
org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.ozone.ClientConfigForTesting;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -139,11 +140,17 @@ public class TestECContainerRecovery {
         TimeUnit.MILLISECONDS);
     conf.setTimeDuration(HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL, 1,
         TimeUnit.SECONDS);
-    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(10)
-        .setTotalPipelineNumLimit(10).setBlockSize(blockSize)
-        .setChunkSize(chunkSize).setStreamBufferFlushSize(flushSize)
+
+    ClientConfigForTesting.newBuilder(StorageUnit.BYTES)
+        .setBlockSize(blockSize)
+        .setChunkSize(chunkSize)
+        .setStreamBufferFlushSize(flushSize)
         .setStreamBufferMaxSize(maxFlushSize)
-        .setStreamBufferSizeUnit(StorageUnit.BYTES).build();
+        .applyTo(conf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(10)
+        .setTotalPipelineNumLimit(10)
+        .build();
     cluster.waitForClusterToBeReady();
     client = OzoneClientFactory.getRpcClient(conf);
     objectStore = client.getObjectStore();
diff --git a/hadoop-ozone/integration-test/src/test/resources/ozone-site.xml 
b/hadoop-ozone/integration-test/src/test/resources/ozone-site.xml
index 0d63a34057..4e79ae97fc 100644
--- a/hadoop-ozone/integration-test/src/test/resources/ozone-site.xml
+++ b/hadoop-ozone/integration-test/src/test/resources/ozone-site.xml
@@ -78,4 +78,38 @@
     <name>ozone.scm.ha.ratis.log.appender.queue.byte-limit</name>
     <value>4MB</value>
   </property>
+
+  <property>
+    <name>ozone.scm.chunk.size</name>
+    <value>1MB</value>
+  </property>
+  <property>
+    <name>ozone.scm.block.size</name>
+    <value>4MB</value>
+  </property>
+  <property>
+    <name>ozone.client.stream.buffer.flush.size</name>
+    <value>1MB</value>
+  </property>
+  <property>
+    <name>ozone.client.stream.buffer.max.size</name>
+    <value>2MB</value>
+  </property>
+  <property>
+    <name>ozone.client.stream.buffer.size</name>
+    <value>1MB</value>
+  </property>
+  <property>
+    <name>ozone.client.datastream.buffer.flush.size</name>
+    <value>4MB</value>
+  </property>
+  <property>
+    <name>ozone.client.datastream.min.packet.size</name>
+    <value>256KB</value>
+  </property>
+  <property>
+    <name>ozone.client.datastream.window.size</name>
+    <value>8MB</value>
+  </property>
+
 </configuration>


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to