From 1734ace35f1c92ff37ccf7f8545b4d74ecbc1cca Mon Sep 17 00:00:00 2001 From: Shashikant Banerjee Date: Wed, 21 Nov 2018 01:07:38 +0530 Subject: [PATCH] Revert "HDDS-835. Use storageSize instead of Long for buffer size configs in Ozone Client. Contributed by Shashikant Banerjee." This reverts commit c74783036d8a2a1d2ce071a888d14e0ccff9263e. --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 3 +- .../apache/hadoop/hdds/scm/ScmConfigKeys.java | 9 +++--- .../apache/hadoop/ozone/OzoneConfigKeys.java | 21 ++++++------- .../org/apache/hadoop/ozone/OzoneConsts.java | 5 --- .../java/org/apache/ratis/RatisHelper.java | 3 +- .../src/main/resources/ozone-default.xml | 6 ++-- .../transport/server/XceiverServerGrpc.java | 5 ++- .../server/ratis/XceiverServerRatis.java | 14 +++------ .../replication/GrpcReplicationClient.java | 4 +-- .../client/io/ChunkGroupOutputStream.java | 7 +++-- .../hadoop/ozone/client/rpc/RpcClient.java | 31 +++++++++---------- .../hadoop/ozone/MiniOzoneClusterImpl.java | 17 +++++----- .../TestCloseContainerHandlingByClient.java | 4 +-- .../rpc/TestFailureHandlingByClient.java | 5 +++ .../om/TestMultipleContainerReadWrite.java | 5 ++- .../storage/DistributedStorageHandler.java | 29 ++++++++--------- .../hadoop/ozone/om/KeyManagerImpl.java | 9 +++--- .../ozone/web/ozShell/keys/GetKeyHandler.java | 7 ++--- .../ozone/web/ozShell/keys/PutKeyHandler.java | 6 ++-- .../fs/ozone/TestOzoneFSInputStream.java | 4 +-- 20 files changed, 84 insertions(+), 110 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index a824c291a4..5592c1d0c1 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,7 +32,6 @@ 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 @@ private void connectToDatanode(DatanodeDetails dn) { 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); 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 f04d12f7aa..896caed1c2 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 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"; 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 8d5c180954..8a5762f3d7 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 = - "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 = 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 7dd9fe30d5..f8f31c80e8 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,11 +146,6 @@ public static Versioning getVersioning(boolean versioning) { 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 7e454d60ec..1ff769573b 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,7 +22,6 @@ 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 @@ static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader, 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) diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 94e134d3d8..865335c8bc 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -623,11 +623,11 @@ - ozone.scm.block.size - 256MB + ozone.scm.block.size.in.mb + 256 OZONE, SCM - 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. 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 be00c8ae09..992f3cb553 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,7 +31,6 @@ 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 XceiverServerGrpc(DatanodeDetails datanodeDetails, Configuration conf, 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); 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 3e54700a06..434d330ff6 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,7 +20,6 @@ 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.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 @@ private RaftProperties newRaftProperties(Configuration conf) { 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)); 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 768d2667c1..8149e2c83f 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 @@ .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 GrpcReplicationClient(String host, 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; 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 d2a4443f3b..5dbe9f65d2 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,6 +24,7 @@ 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 ChunkGroupOutputStream(OpenKeySession handler, 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); 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 3a0f47580e..98481e138e 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,7 +20,6 @@ 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 RpcClient(Configuration conf) throws IOException { 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, 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 ceea3ee3f3..324e17ba69 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,7 +24,6 @@ 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 @@ private void initializeConfiguration() throws IOException { 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 @@ private void initializeConfiguration() throws IOException { 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(); } 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 d268aebefc..abd60a1a5c 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,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 static void init() throws Exception { 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 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 923300fd75..dc6747f89e 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,6 +22,7 @@ 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 void init() throws Exception { 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 d0f4ce22b3..3848cda2bf 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,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(); 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 fd100553de..6a433b58de 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,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 DistributedStorageHandler(OzoneConfiguration conf, 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, 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 a41dae6bfa..98a27bf9e5 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,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.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_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 KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient, 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( 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 4e86699935..be9342d315 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,7 +24,6 @@ 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 Void call() throws Exception { 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( 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 f86af3fc98..b2ab68fc08 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,7 +22,6 @@ 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 Void call() throws Exception { .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; 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 913bb3ebd5..d3bc857632 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,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();