diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 5592c1d0c17..a824c291a46 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -32,6 +32,7 @@ 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; @@ -103,7 +104,7 @@ public class XceiverClientGrpc extends XceiverClientSpi { LOG.debug("Connecting to server Port : " + dn.getIpAddress()); ManagedChannel channel = NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext() - .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE) + .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE) .build(); XceiverClientProtocolServiceStub asyncStub = XceiverClientProtocolServiceGrpc.newStub(channel); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java index 896caed1c20..6733b8e8d35 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java +++ b/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 int DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT = - 16 * 1024; + public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT = + "16KB"; public static final String DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY = "dfs.container.ratis.segment.preallocated.size"; - public static final int - DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT = 128 * 1024 * 1024; + public static final String + DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT = "128MB"; public static final String DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT = "dfs.container.ratis.statemachinedata.sync.timeout"; @@ -136,8 +136,7 @@ public final class ScmConfigKeys { // TODO : this is copied from OzoneConsts, may need to move to a better place 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_CHUNK_SIZE_DEFAULT = "16MB"; public static final String OZONE_SCM_CLIENT_PORT_KEY = "ozone.scm.client.port"; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java index 8a5762f3d75..8d5c1809540 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java +++ b/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_IN_MB = - "ozone.scm.block.size.in.mb"; - public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256; + public static final String OZONE_SCM_BLOCK_SIZE = + "ozone.scm.block.size"; + public static final String OZONE_SCM_BLOCK_SIZE_DEFAULT = "256MB"; /** * Ozone administrator users delimited by comma. @@ -115,12 +115,14 @@ public final class OzoneConfigKeys { public static final String OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE = "ozone.client.stream.buffer.flush.size"; - public static final long OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE_DEFAULT = 64; + public static final String OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE_DEFAULT = + "64MB"; public static final String OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE = "ozone.client.stream.buffer.max.size"; - public static final long OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE_DEFAULT = 128; + public static final String OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE_DEFAULT = + "128MB"; public static final String OZONE_CLIENT_WATCH_REQUEST_TIMEOUT = "ozone.client.watch.request.timeout"; @@ -231,12 +233,13 @@ 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 int DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT + public static final String 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 int DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT - = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT; + public static final String + 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 @@ -246,8 +249,6 @@ 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 = diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java index f8f31c80e87..7dd9fe30d50 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java @@ -146,6 +146,11 @@ 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. diff --git a/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java index 1ff769573b9..7e454d60ec3 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java @@ -22,6 +22,7 @@ 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; @@ -155,7 +156,7 @@ public interface RatisHelper { RaftConfigKeys.Rpc.setType(properties, rpcType); GrpcConfigKeys.setMessageSizeMax(properties, - SizeInBytes.valueOf(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)); + SizeInBytes.valueOf(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)); return RaftClient.newBuilder() .setRaftGroup(group) diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 865335c8bcf..767a86596e7 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -175,7 +175,7 @@ dfs.container.ratis.segment.size - 16384 + 16KB OZONE, RATIS, PERFORMANCE The size of the raft segment used by Apache Ratis on datanodes. (16 KB by default) @@ -183,7 +183,7 @@ dfs.container.ratis.segment.preallocated.size - 134217728 + 128MB OZONE, RATIS, PERFORMANCE The size of the buffer which is preallocated for raft segment used by Apache Ratis on datanodes.(128 MB by default) @@ -336,7 +336,7 @@ ozone.client.stream.buffer.flush.size - 64 + 64MB OZONE, CLIENT Size in mb which determines at what buffer position , a partial flush will be initiated during write. It should be ideally a mutiple @@ -345,7 +345,7 @@ ozone.client.stream.buffer.max.size - 128 + 128MB OZONE, CLIENT Size in mb which determines at what buffer position , write call be blocked till acknowledgement of the fisrt partial flush @@ -623,17 +623,17 @@ - ozone.scm.block.size.in.mb - 256 + ozone.scm.block.size + 256MB OZONE, SCM - The default size of a scm block in bytes. This is maps to the default + The default size of a scm block. This is maps to the default Ozone block size. ozone.scm.chunk.size - 16777216 + 16MB OZONE, SCM, CONTAINER, PERFORMANCE The chunk size for reading/writing chunk operations in bytes. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java index 992f3cb553b..be00c8ae098 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java @@ -31,6 +31,7 @@ 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; @@ -90,12 +91,12 @@ public final class XceiverServerGrpc implements XceiverServerSpi { datanodeDetails.setPort( DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port)); server = ((NettyServerBuilder) ServerBuilder.forPort(port)) - .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE) + .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE) .addService(new GrpcXceiverService(dispatcher)) .build(); NettyServerBuilder nettyServerBuilder = ((NettyServerBuilder) ServerBuilder.forPort(port)) - .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE) + .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE) .addService(new GrpcXceiverService(dispatcher)); for (BindableService service : additionalServices) { nettyServerBuilder.addService(service); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java index 434d330ff68..3e54700a061 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java @@ -20,6 +20,7 @@ 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; @@ -33,6 +34,7 @@ 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 @@ -154,23 +156,25 @@ public final class XceiverServerRatis implements XceiverServerSpi { RaftConfigKeys.Rpc.setType(properties, rpc); // set raft segment size - final int raftSegmentSize = conf.getInt( + final int raftSegmentSize = (int)conf.getStorageSize( OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY, - OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT); + OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT, + StorageUnit.BYTES); RaftServerConfigKeys.Log.setSegmentSizeMax(properties, SizeInBytes.valueOf(raftSegmentSize)); // set raft segment pre-allocated size - final int raftSegmentPreallocatedSize = conf.getInt( + final int raftSegmentPreallocatedSize = (int) conf.getStorageSize( OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY, - OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT); + OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT, + StorageUnit.BYTES); 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 = OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE; + final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE; RaftServerConfigKeys.Log.setWriteBufferSize(properties, SizeInBytes.valueOf(maxChunkSize)); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java index 8149e2c83f2..768d2667c1b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java @@ -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(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE) + .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE) .build(); client = IntraDatanodeProtocolServiceGrpc.newStub(channel); this.workingDirectory = workingDir; diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java index 5dbe9f65d2a..d2a4443f3be 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java @@ -24,7 +24,6 @@ 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; @@ -159,9 +158,9 @@ public class ChunkGroupOutputStream extends OutputStream { this.xceiverClientManager = xceiverClientManager; this.chunkSize = chunkSize; this.requestID = requestId; - this.streamBufferFlushSize = bufferFlushSize * OzoneConsts.MB; - this.streamBufferMaxSize = bufferMaxSize * OzoneConsts.MB; - this.blockSize = size * OzoneConsts.MB; + this.streamBufferFlushSize = bufferFlushSize; + this.streamBufferMaxSize = bufferMaxSize; + this.blockSize = size; this.watchTimeout = watchTimeout; Preconditions.checkState(chunkSize > 0); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 98481e138ea..3a0f47580eb 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -20,6 +20,7 @@ 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; @@ -139,26 +140,28 @@ public class RpcClient implements ClientProtocol { this.xceiverClientManager = new XceiverClientManager(conf); - int configuredChunkSize = conf.getInt( - ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, - ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT); - if(configuredChunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) { + 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) { LOG.warn("The chunk size ({}) is not allowed to be more than" + " the maximum size ({})," + " resetting to the maximum size.", - configuredChunkSize, ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE); - chunkSize = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE; + configuredChunkSize, OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE); + chunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE; } else { chunkSize = configuredChunkSize; } - 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); + 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); watchTimeout = conf.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT_DEFAULT, 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 324e17ba695..ceea3ee3f3e 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 @@ -24,6 +24,7 @@ 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; @@ -392,6 +393,7 @@ 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()) { @@ -403,13 +405,14 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { if (!blockSize.isPresent()) { blockSize = Optional.of(2 * streamBufferMaxSize.get()); } - 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()); + 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); configureTrace(); } 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 abd60a1a5cc..d268aebefc0 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 @@ -17,6 +17,7 @@ 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; @@ -94,7 +95,8 @@ 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.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, (4)); + conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4, + StorageUnit.MB); cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(7).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/TestFailureHandlingByClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java index dc6747f89e4..923300fd751 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestFailureHandlingByClient.java @@ -22,7 +22,6 @@ 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; @@ -83,15 +82,11 @@ 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(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java index 3848cda2bff..d0f4ce22b3c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java @@ -17,6 +17,7 @@ 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; @@ -69,8 +70,8 @@ public class TestMultipleContainerReadWrite { @BeforeClass public static void init() throws Exception { conf = new OzoneConfiguration(); - // set to as small as 100 bytes per block. - conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, 1); + conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 1, + StorageUnit.MB); conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 5); cluster = MiniOzoneCluster.newBuilder(conf).build(); cluster.waitForClusterToBeReady(); diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java index 6a433b58dee..fd100553dee 100644 --- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java +++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java @@ -19,6 +19,7 @@ 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; @@ -100,27 +101,29 @@ public final class DistributedStorageHandler implements StorageHandler { this.storageContainerLocationClient = storageContainerLocation; this.xceiverClientManager = new XceiverClientManager(conf); - chunkSize = conf.getInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, - ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT); + chunkSize = (int)conf.getStorageSize(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, + ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES); 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 > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) { + if(chunkSize > OzoneConsts.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, ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE); - chunkSize = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE; + chunkSize, OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE); + chunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE; } - 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); + 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); watchTimeout = conf.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT_DEFAULT, diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java index 98a27bf9e5c..a41dae6bfab 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java @@ -17,6 +17,7 @@ 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; @@ -25,7 +26,6 @@ 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_IN_MB; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE; /** * Implementation of keyManager. @@ -87,8 +87,9 @@ public class KeyManagerImpl implements KeyManager { String omId) { this.scmBlockClient = scmBlockClient; this.metadataManager = metadataManager; - this.scmBlockSize = conf.getLong(OZONE_SCM_BLOCK_SIZE_IN_MB, - OZONE_SCM_BLOCK_SIZE_DEFAULT) * OzoneConsts.MB; + this.scmBlockSize = (long) conf + .getStorageSize(OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT, + StorageUnit.BYTES); this.useRatis = conf.getBoolean(DFS_CONTAINER_RATIS_ENABLED_KEY, DFS_CONTAINER_RATIS_ENABLED_DEFAULT); this.preallocateMax = conf.getLong( diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java index be9342d3156..4e866999356 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java @@ -24,6 +24,7 @@ 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; @@ -94,8 +95,10 @@ public class GetKeyHandler extends Handler { OzoneInputStream keyInputStream = bucket.readKey(keyName); if (dataFilePath != null) { FileOutputStream outputStream = new FileOutputStream(dataFile); - IOUtils.copyBytes(keyInputStream, outputStream, new OzoneConfiguration() - .getInt(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT)); + IOUtils.copyBytes(keyInputStream, outputStream, + (int) new OzoneConfiguration() + .getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, + OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES)); outputStream.close(); } else { throw new OzoneClientException( diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java index b2ab68fc087..f86af3fc98d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java @@ -22,6 +22,7 @@ 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; @@ -106,8 +107,9 @@ public class PutKeyHandler extends Handler { .createKey(keyName, dataFile.length(), replicationType, replicationFactor); FileInputStream fileInputStream = new FileInputStream(dataFile); - IOUtils.copyBytes(fileInputStream, outputStream, - conf.getInt(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT)); + IOUtils.copyBytes(fileInputStream, outputStream, (int) conf + .getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT, + StorageUnit.BYTES)); outputStream.close(); fileInputStream.close(); return null; diff --git a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java index d3bc857632d..913bb3ebd56 100644 --- a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java +++ b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java @@ -19,6 +19,7 @@ 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; @@ -64,7 +65,8 @@ public class TestOzoneFSInputStream { @BeforeClass public static void init() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); - conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, 10); + conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 10, + StorageUnit.MB); cluster = MiniOzoneCluster.newBuilder(conf) .setNumDatanodes(10) .build();