瀏覽代碼

Revert "HDDS-835. Use storageSize instead of Long for buffer size configs in Ozone Client. Contributed by Shashikant Banerjee."

This reverts commit c74783036d8a2a1d2ce071a888d14e0ccff9263e.
Shashikant Banerjee 6 年之前
父節點
當前提交
1734ace35f
共有 20 個文件被更改,包括 84 次插入110 次删除
  1. 1 2
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
  2. 5 4
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
  3. 10 11
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
  4. 0 5
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
  5. 1 2
      hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java
  6. 3 3
      hadoop-hdds/common/src/main/resources/ozone-default.xml
  7. 2 3
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
  8. 5 9
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
  9. 2 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java
  10. 4 3
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
  11. 14 17
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
  12. 7 10
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
  13. 1 3
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
  14. 5 0
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java
  15. 2 3
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java
  16. 13 16
      hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
  17. 4 5
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
  18. 2 5
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java
  19. 2 4
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java
  20. 1 3
      hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java

+ 1 - 2
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.util.Time;
 import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
 import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder;
@@ -104,7 +103,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
     LOG.debug("Connecting to server Port : " + dn.getIpAddress());
     ManagedChannel channel =
         NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext()
-            .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
+            .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
             .build();
     XceiverClientProtocolServiceStub asyncStub =
         XceiverClientProtocolServiceGrpc.newStub(channel);

+ 5 - 4
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java

@@ -73,12 +73,12 @@ public final class ScmConfigKeys {
       = 10;
   public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY =
       "dfs.container.ratis.segment.size";
-  public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT =
-      "16KB";
+  public static final int DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT =
+      16 * 1024;
   public static final String DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY =
       "dfs.container.ratis.segment.preallocated.size";
-  public static final String
-      DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT = "128MB";
+  public static final int
+      DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT = 128 * 1024 * 1024;
   public static final String
       DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT =
       "dfs.container.ratis.statemachinedata.sync.timeout";
@@ -137,6 +137,7 @@ public final class ScmConfigKeys {
   public static final String OZONE_SCM_CHUNK_SIZE_KEY = "ozone.scm.chunk.size";
   // 16 MB by default
   public static final int OZONE_SCM_CHUNK_SIZE_DEFAULT = 16 * 1024 * 1024;
+  public static final int OZONE_SCM_CHUNK_MAX_SIZE = 32 * 1024 * 1024;
 
   public static final String OZONE_SCM_CLIENT_PORT_KEY =
       "ozone.scm.client.port";

+ 10 - 11
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java

@@ -95,9 +95,9 @@ public final class OzoneConfigKeys {
       "ozone.container.cache.size";
   public static final int OZONE_CONTAINER_CACHE_DEFAULT = 1024;
 
-  public static final String OZONE_SCM_BLOCK_SIZE =
-      "ozone.scm.block.size";
-  public static final String OZONE_SCM_BLOCK_SIZE_DEFAULT = "256MB";
+  public static final String OZONE_SCM_BLOCK_SIZE_IN_MB =
+      "ozone.scm.block.size.in.mb";
+  public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256;
 
   /**
    * Ozone administrator users delimited by comma.
@@ -115,14 +115,12 @@ public final class OzoneConfigKeys {
   public static final String OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE =
       "ozone.client.stream.buffer.flush.size";
 
-  public static final String OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE_DEFAULT =
-      "64MB";
+  public static final long OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE_DEFAULT = 64;
 
   public static final String OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE =
       "ozone.client.stream.buffer.max.size";
 
-  public static final String OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE_DEFAULT =
-      "128MB";
+  public static final long OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE_DEFAULT = 128;
 
   public static final String OZONE_CLIENT_WATCH_REQUEST_TIMEOUT =
       "ozone.client.watch.request.timeout";
@@ -233,13 +231,12 @@ public final class OzoneConfigKeys {
       = ScmConfigKeys.DFS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_DEFAULT;
   public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY
       = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY;
-  public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT
+  public static final int DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT
       = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT;
   public static final String DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY
       = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY;
-  public static final String
-      DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT =
-      ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT;
+  public static final int DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT
+      = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT;
 
   // config settings to enable stateMachineData write timeout
   public static final String
@@ -249,6 +246,8 @@ public final class OzoneConfigKeys {
       DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT =
       ScmConfigKeys.DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT;
 
+  public static final int DFS_CONTAINER_CHUNK_MAX_SIZE
+      = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE;
   public static final String DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR =
       "dfs.container.ratis.datanode.storage.dir";
   public static final String DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY =

+ 0 - 5
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java

@@ -146,11 +146,6 @@ public final class OzoneConsts {
   public static final String OM_S3_PREFIX ="S3:";
   public static final String OM_S3_VOLUME_PREFIX = "s3";
 
-  /**
-   *   Max chunk size limit.
-   */
-  public static final int OZONE_SCM_CHUNK_MAX_SIZE = 32 * 1024 * 1024;
-
 
   /**
    * Max OM Quota size of 1024 PB.

+ 1 - 2
hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java

@@ -22,7 +22,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.grpc.GrpcConfigKeys;
@@ -156,7 +155,7 @@ public interface RatisHelper {
     RaftConfigKeys.Rpc.setType(properties, rpcType);
 
     GrpcConfigKeys.setMessageSizeMax(properties,
-        SizeInBytes.valueOf(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE));
+        SizeInBytes.valueOf(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE));
 
     return RaftClient.newBuilder()
         .setRaftGroup(group)

+ 3 - 3
hadoop-hdds/common/src/main/resources/ozone-default.xml

@@ -623,11 +623,11 @@
     </description>
   </property>
   <property>
-    <name>ozone.scm.block.size</name>
-    <value>256MB</value>
+    <name>ozone.scm.block.size.in.mb</name>
+    <value>256</value>
     <tag>OZONE, SCM</tag>
     <description>
-      The default size of a scm block. This is maps to the default
+      The default size of a scm block in bytes. This is maps to the default
       Ozone block size.
     </description>
   </property>

+ 2 - 3
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java

@@ -31,7 +31,6 @@ import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
 import org.apache.hadoop.hdds.scm.container.common.helpers.
     StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
 
 import org.apache.ratis.thirdparty.io.grpc.BindableService;
@@ -91,12 +90,12 @@ public final class XceiverServerGrpc implements XceiverServerSpi {
     datanodeDetails.setPort(
         DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port));
     server = ((NettyServerBuilder) ServerBuilder.forPort(port))
-        .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
+        .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
         .addService(new GrpcXceiverService(dispatcher))
         .build();
     NettyServerBuilder nettyServerBuilder =
         ((NettyServerBuilder) ServerBuilder.forPort(port))
-            .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
+            .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
             .addService(new GrpcXceiverService(dispatcher));
     for (BindableService service : additionalServices) {
       nettyServerBuilder.addService(service);

+ 5 - 9
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common.transport.server.ratis;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
@@ -34,7 +33,6 @@ import org.apache.hadoop.hdds.protocol.proto
 import org.apache.hadoop.hdds.scm.HddsServerUtil;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.common.transport.server
@@ -156,25 +154,23 @@ public final class XceiverServerRatis implements XceiverServerSpi {
     RaftConfigKeys.Rpc.setType(properties, rpc);
 
     // set raft segment size
-    final int raftSegmentSize = (int)conf.getStorageSize(
+    final int raftSegmentSize = conf.getInt(
         OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY,
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT,
-        StorageUnit.BYTES);
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT);
     RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
         SizeInBytes.valueOf(raftSegmentSize));
 
     // set raft segment pre-allocated size
-    final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
+    final int raftSegmentPreallocatedSize = conf.getInt(
         OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
-        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
-        StorageUnit.BYTES);
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT);
     RaftServerConfigKeys.Log.Appender.setBufferCapacity(properties,
         SizeInBytes.valueOf(raftSegmentPreallocatedSize));
     RaftServerConfigKeys.Log.setPreallocatedSize(properties,
         SizeInBytes.valueOf(raftSegmentPreallocatedSize));
 
     // Set max write buffer size, which is the scm chunk size
-    final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
+    final int maxChunkSize = OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE;
     RaftServerConfigKeys.Log.setWriteBufferSize(properties,
         SizeInBytes.valueOf(maxChunkSize));
 

+ 2 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java

@@ -33,9 +33,9 @@ import org.apache.hadoop.hdds.protocol.datanode.proto
     .IntraDatanodeProtocolServiceGrpc;
 import org.apache.hadoop.hdds.protocol.datanode.proto
     .IntraDatanodeProtocolServiceGrpc.IntraDatanodeProtocolServiceStub;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
 import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder;
 import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
@@ -61,7 +61,7 @@ public class GrpcReplicationClient {
 
     channel = NettyChannelBuilder.forAddress(host, port)
         .usePlaintext()
-        .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
+        .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
         .build();
     client = IntraDatanodeProtocolServiceGrpc.newStub(channel);
     this.workingDirectory = workingDir;

+ 4 - 3
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
@@ -158,9 +159,9 @@ public class ChunkGroupOutputStream extends OutputStream {
     this.xceiverClientManager = xceiverClientManager;
     this.chunkSize = chunkSize;
     this.requestID = requestId;
-    this.streamBufferFlushSize = bufferFlushSize;
-    this.streamBufferMaxSize = bufferMaxSize;
-    this.blockSize = size;
+    this.streamBufferFlushSize = bufferFlushSize * OzoneConsts.MB;
+    this.streamBufferMaxSize = bufferMaxSize * OzoneConsts.MB;
+    this.blockSize = size * OzoneConsts.MB;
     this.watchTimeout = watchTimeout;
 
     Preconditions.checkState(chunkSize > 0);

+ 14 - 17
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.client.rpc;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
@@ -140,28 +139,26 @@ public class RpcClient implements ClientProtocol {
 
     this.xceiverClientManager = new XceiverClientManager(conf);
 
-    int configuredChunkSize = (int) conf
-        .getStorageSize(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
-            ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES);
-    if(configuredChunkSize > OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE) {
+    int configuredChunkSize = conf.getInt(
+        ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
+        ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT);
+    if(configuredChunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) {
       LOG.warn("The chunk size ({}) is not allowed to be more than"
               + " the maximum size ({}),"
               + " resetting to the maximum size.",
-          configuredChunkSize, OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE);
-      chunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
+          configuredChunkSize, ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE);
+      chunkSize = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE;
     } else {
       chunkSize = configuredChunkSize;
     }
-    streamBufferFlushSize = (long) conf
-        .getStorageSize(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE,
-            OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE_DEFAULT,
-            StorageUnit.BYTES);
-    streamBufferMaxSize = (long) conf
-        .getStorageSize(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE,
-            OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE_DEFAULT,
-            StorageUnit.BYTES);
-    blockSize = (long) conf.getStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
-        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
+    streamBufferFlushSize =
+        conf.getLong(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE,
+            OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE_DEFAULT);
+    streamBufferMaxSize =
+        conf.getLong(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE,
+            OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE_DEFAULT);
+    blockSize = conf.getLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB,
+        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT);
     watchTimeout =
         conf.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT,
             OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT_DEFAULT,

+ 7 - 10
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java

@@ -24,7 +24,6 @@ import java.util.Optional;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -393,7 +392,6 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
       Files.createDirectories(metaDir);
       conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.toString());
       if (!chunkSize.isPresent()) {
-        //set it to 1MB by default in tests
         chunkSize = Optional.of(1);
       }
       if (!streamBufferFlushSize.isPresent()) {
@@ -405,14 +403,13 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
       if (!blockSize.isPresent()) {
         blockSize = Optional.of(2 * streamBufferMaxSize.get());
       }
-      conf.setStorageSize(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
-          chunkSize.get(), StorageUnit.MB);
-      conf.setStorageSize(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE,
-          streamBufferFlushSize.get(), StorageUnit.MB);
-      conf.setStorageSize(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE,
-          streamBufferMaxSize.get(), StorageUnit.MB);
-      conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, blockSize.get(),
-          StorageUnit.MB);
+      conf.setInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
+          (int) (chunkSize.get() * OzoneConsts.MB));
+      conf.setLong(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE,
+          streamBufferFlushSize.get());
+      conf.setLong(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE,
+          streamBufferMaxSize.get());
+      conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, blockSize.get());
       configureTrace();
     }
 

+ 1 - 3
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java

@@ -17,7 +17,6 @@
 
 package org.apache.hadoop.ozone.client.rpc;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -95,8 +94,7 @@ public class TestCloseContainerHandlingByClient {
     conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
     conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
     conf.setQuietMode(false);
-    conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4,
-        StorageUnit.MB);
+    conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, (4));
     cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(7).build();
     cluster.waitForClusterToBeReady();
     //the easiest way to create an open container is creating a key

+ 5 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java

@@ -22,6 +22,7 @@ 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.container.ContainerID;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
@@ -82,11 +83,15 @@ public class TestFailureHandlingByClient {
     maxRetries = 100;
     chunkSize = (int) OzoneConsts.MB;
     blockSize = 4 * chunkSize;
+    conf.setInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, chunkSize);
+    conf.setInt(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE, 1);
+    conf.setInt(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE, 2);
     conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 5,
         TimeUnit.SECONDS);
     conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
     conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
     conf.setQuietMode(false);
+    conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, (4));
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(6).build();
     cluster.waitForClusterToBeReady();

+ 2 - 3
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java

@@ -17,7 +17,6 @@
 package org.apache.hadoop.ozone.om;
 
 import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -70,8 +69,8 @@ public class TestMultipleContainerReadWrite {
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
-    conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 1,
-        StorageUnit.MB);
+    // set to as small as 100 bytes per block.
+    conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, 1);
     conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 5);
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();

+ 13 - 16
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.ozone.web.storage;
 
 import com.google.common.base.Strings;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -101,29 +100,27 @@ public final class DistributedStorageHandler implements StorageHandler {
     this.storageContainerLocationClient = storageContainerLocation;
     this.xceiverClientManager = new XceiverClientManager(conf);
 
-    chunkSize = (int)conf.getStorageSize(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
-        ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES);
+    chunkSize = conf.getInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
+        ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT);
     userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS,
         OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
     groupRights = conf.getEnum(OMConfigKeys.OZONE_OM_GROUP_RIGHTS,
         OMConfigKeys.OZONE_OM_GROUP_RIGHTS_DEFAULT);
-    if(chunkSize > OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE) {
+    if(chunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) {
       LOG.warn("The chunk size ({}) is not allowed to be more than"
               + " the maximum size ({}),"
               + " resetting to the maximum size.",
-          chunkSize, OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE);
-      chunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
+          chunkSize, ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE);
+      chunkSize = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE;
     }
-    streamBufferFlushSize = (long) conf
-        .getStorageSize(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE,
-            OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE_DEFAULT,
-            StorageUnit.BYTES);
-    streamBufferMaxSize = (long) conf
-        .getStorageSize(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE,
-            OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE_DEFAULT,
-            StorageUnit.BYTES);
-    blockSize = (long) conf.getStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
-        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
+    streamBufferFlushSize =
+        conf.getLong(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE,
+            OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE_DEFAULT);
+    streamBufferMaxSize =
+        conf.getLong(OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE,
+            OzoneConfigKeys.OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE_DEFAULT);
+    blockSize = conf.getLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB,
+        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT);
     watchTimeout =
         conf.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT,
             OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT_DEFAULT,

+ 4 - 5
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java

@@ -17,7 +17,6 @@
 package org.apache.hadoop.ozone.om;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
@@ -26,6 +25,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
@@ -59,7 +59,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVI
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB;
 
 /**
  * Implementation of keyManager.
@@ -87,9 +87,8 @@ public class KeyManagerImpl implements KeyManager {
       String omId) {
     this.scmBlockClient = scmBlockClient;
     this.metadataManager = metadataManager;
-    this.scmBlockSize = (long) conf
-        .getStorageSize(OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT,
-            StorageUnit.BYTES);
+    this.scmBlockSize = conf.getLong(OZONE_SCM_BLOCK_SIZE_IN_MB,
+        OZONE_SCM_BLOCK_SIZE_DEFAULT) * OzoneConsts.MB;
     this.useRatis = conf.getBoolean(DFS_CONTAINER_RATIS_ENABLED_KEY,
         DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
     this.preallocateMax = conf.getLong(

+ 2 - 5
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java

@@ -24,7 +24,6 @@ import java.io.FileOutputStream;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -95,10 +94,8 @@ public class GetKeyHandler extends Handler {
     OzoneInputStream keyInputStream = bucket.readKey(keyName);
     if (dataFilePath != null) {
       FileOutputStream outputStream = new FileOutputStream(dataFile);
-      IOUtils.copyBytes(keyInputStream, outputStream,
-          (int) new OzoneConfiguration()
-              .getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY,
-                  OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES));
+      IOUtils.copyBytes(keyInputStream, outputStream, new OzoneConfiguration()
+          .getInt(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT));
       outputStream.close();
     } else {
       throw new OzoneClientException(

+ 2 - 4
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java

@@ -22,7 +22,6 @@ import java.io.File;
 import java.io.FileInputStream;
 
 import org.apache.hadoop.conf.Configuration;
-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;
@@ -107,9 +106,8 @@ public class PutKeyHandler extends Handler {
         .createKey(keyName, dataFile.length(), replicationType,
             replicationFactor);
     FileInputStream fileInputStream = new FileInputStream(dataFile);
-    IOUtils.copyBytes(fileInputStream, outputStream, (int) conf
-        .getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT,
-            StorageUnit.BYTES));
+    IOUtils.copyBytes(fileInputStream, outputStream,
+        conf.getInt(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT));
     outputStream.close();
     fileInputStream.close();
     return null;

+ 1 - 3
hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.fs.ozone;
 
 import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -65,8 +64,7 @@ public class TestOzoneFSInputStream {
   @BeforeClass
   public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 10,
-        StorageUnit.MB);
+    conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, 10);
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(10)
         .build();