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

This commit is contained in:
Shashikant Banerjee 2018-11-21 10:51:50 +05:30
parent f994b526a0
commit 14e1a0a3d6
20 changed files with 116 additions and 90 deletions

View File

@ -32,6 +32,7 @@
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 @@ private void connectToDatanode(DatanodeDetails dn) {
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);

View File

@ -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";

View File

@ -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 =

View File

@ -146,6 +146,11 @@ 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.

View File

@ -22,6 +22,7 @@
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 @@ static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
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)

View File

@ -175,7 +175,7 @@
</property>
<property>
<name>dfs.container.ratis.segment.size</name>
<value>16384</value>
<value>16KB</value>
<tag>OZONE, RATIS, PERFORMANCE</tag>
<description>The size of the raft segment used by Apache Ratis on datanodes.
(16 KB by default)
@ -183,7 +183,7 @@
</property>
<property>
<name>dfs.container.ratis.segment.preallocated.size</name>
<value>134217728</value>
<value>128MB</value>
<tag>OZONE, RATIS, PERFORMANCE</tag>
<description>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 @@
</property>
<property>
<name>ozone.client.stream.buffer.flush.size</name>
<value>64</value>
<value>64MB</value>
<tag>OZONE, CLIENT</tag>
<description>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 @@
</property>
<property>
<name>ozone.client.stream.buffer.max.size</name>
<value>128</value>
<value>128MB</value>
<tag>OZONE, CLIENT</tag>
<description>Size in mb which determines at what buffer position ,
write call be blocked till acknowledgement of the fisrt partial flush
@ -623,17 +623,17 @@
</description>
</property>
<property>
<name>ozone.scm.block.size.in.mb</name>
<value>256</value>
<name>ozone.scm.block.size</name>
<value>256MB</value>
<tag>OZONE, SCM</tag>
<description>
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.
</description>
</property>
<property>
<name>ozone.scm.chunk.size</name>
<value>16777216</value>
<value>16MB</value>
<tag>OZONE, SCM, CONTAINER, PERFORMANCE</tag>
<description>
The chunk size for reading/writing chunk operations in bytes.

View File

@ -31,6 +31,7 @@
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 XceiverServerGrpc(DatanodeDetails datanodeDetails, Configuration conf,
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);

View File

@ -20,6 +20,7 @@
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.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 @@ private RaftProperties newRaftProperties(Configuration conf) {
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));

View File

@ -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(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
.build();
client = IntraDatanodeProtocolServiceGrpc.newStub(channel);
this.workingDirectory = workingDir;

View File

@ -24,7 +24,6 @@
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 ChunkGroupOutputStream(OpenKeySession handler,
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);

View File

@ -20,6 +20,7 @@
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 RpcClient(Configuration conf) throws IOException {
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,

View File

@ -24,6 +24,7 @@
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 @@ 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()) {
@ -403,13 +405,14 @@ private void initializeConfiguration() throws IOException {
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();
}

View File

@ -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 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.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

View File

@ -22,7 +22,6 @@
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 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();

View File

@ -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();

View File

@ -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 DistributedStorageHandler(OzoneConfiguration conf,
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,

View File

@ -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.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_IN_MB;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
/**
* Implementation of keyManager.
@ -87,8 +87,9 @@ public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient,
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(

View File

@ -24,6 +24,7 @@
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 Void call() throws Exception {
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(

View File

@ -22,6 +22,7 @@
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 Void call() throws Exception {
.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;

View File

@ -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();