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-20 23:01:02 +05:30
parent 10b5da85fa
commit c74783036d
20 changed files with 110 additions and 84 deletions

View File

@ -32,6 +32,7 @@
import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; 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()); LOG.debug("Connecting to server Port : " + dn.getIpAddress());
ManagedChannel channel = ManagedChannel channel =
NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext() NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext()
.maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE) .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
.build(); .build();
XceiverClientProtocolServiceStub asyncStub = XceiverClientProtocolServiceStub asyncStub =
XceiverClientProtocolServiceGrpc.newStub(channel); XceiverClientProtocolServiceGrpc.newStub(channel);

View File

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

View File

@ -95,9 +95,9 @@ public final class OzoneConfigKeys {
"ozone.container.cache.size"; "ozone.container.cache.size";
public static final int OZONE_CONTAINER_CACHE_DEFAULT = 1024; public static final int OZONE_CONTAINER_CACHE_DEFAULT = 1024;
public static final String OZONE_SCM_BLOCK_SIZE_IN_MB = public static final String OZONE_SCM_BLOCK_SIZE =
"ozone.scm.block.size.in.mb"; "ozone.scm.block.size";
public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256; public static final String OZONE_SCM_BLOCK_SIZE_DEFAULT = "256MB";
/** /**
* Ozone administrator users delimited by comma. * Ozone administrator users delimited by comma.
@ -115,12 +115,14 @@ public final class OzoneConfigKeys {
public static final String OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE = public static final String OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE =
"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 = public static final String OZONE_CLIENT_STREAM_BUFFER_MAX_SIZE =
"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 = public static final String OZONE_CLIENT_WATCH_REQUEST_TIMEOUT =
"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; = ScmConfigKeys.DFS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_DEFAULT;
public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY
= ScmConfigKeys.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; = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT;
public static final String DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY public static final String DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY
= ScmConfigKeys.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 public static final String
= ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT; DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT =
ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT;
// config settings to enable stateMachineData write timeout // config settings to enable stateMachineData write timeout
public static final String public static final String
@ -246,8 +249,6 @@ public final class OzoneConfigKeys {
DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT = DFS_CONTAINER_RATIS_STATEMACHINEDATA_SYNC_TIMEOUT_DEFAULT =
ScmConfigKeys.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 = public static final String DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR =
"dfs.container.ratis.datanode.storage.dir"; "dfs.container.ratis.datanode.storage.dir";
public static final String DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY = 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_PREFIX ="S3:";
public static final String OM_S3_VOLUME_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. * 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.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.ratis.client.RaftClient; import org.apache.ratis.client.RaftClient;
import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.grpc.GrpcConfigKeys;
@ -155,7 +156,7 @@ static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
RaftConfigKeys.Rpc.setType(properties, rpcType); RaftConfigKeys.Rpc.setType(properties, rpcType);
GrpcConfigKeys.setMessageSizeMax(properties, GrpcConfigKeys.setMessageSizeMax(properties,
SizeInBytes.valueOf(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)); SizeInBytes.valueOf(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE));
return RaftClient.newBuilder() return RaftClient.newBuilder()
.setRaftGroup(group) .setRaftGroup(group)

View File

@ -623,11 +623,11 @@
</description> </description>
</property> </property>
<property> <property>
<name>ozone.scm.block.size.in.mb</name> <name>ozone.scm.block.size</name>
<value>256</value> <value>256MB</value>
<tag>OZONE, SCM</tag> <tag>OZONE, SCM</tag>
<description> <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. Ozone block size.
</description> </description>
</property> </property>

View File

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

View File

@ -20,6 +20,7 @@
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration; 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.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandRequestProto; .ContainerCommandRequestProto;
@ -33,6 +34,7 @@
import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.ozone.OzoneConfigKeys; 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.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server import org.apache.hadoop.ozone.container.common.transport.server
@ -154,23 +156,25 @@ private RaftProperties newRaftProperties(Configuration conf) {
RaftConfigKeys.Rpc.setType(properties, rpc); RaftConfigKeys.Rpc.setType(properties, rpc);
// set raft segment size // 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_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT); OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT,
StorageUnit.BYTES);
RaftServerConfigKeys.Log.setSegmentSizeMax(properties, RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
SizeInBytes.valueOf(raftSegmentSize)); SizeInBytes.valueOf(raftSegmentSize));
// set raft segment pre-allocated size // 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_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT); OzoneConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
StorageUnit.BYTES);
RaftServerConfigKeys.Log.Appender.setBufferCapacity(properties, RaftServerConfigKeys.Log.Appender.setBufferCapacity(properties,
SizeInBytes.valueOf(raftSegmentPreallocatedSize)); SizeInBytes.valueOf(raftSegmentPreallocatedSize));
RaftServerConfigKeys.Log.setPreallocatedSize(properties, RaftServerConfigKeys.Log.setPreallocatedSize(properties,
SizeInBytes.valueOf(raftSegmentPreallocatedSize)); SizeInBytes.valueOf(raftSegmentPreallocatedSize));
// Set max write buffer size, which is the scm chunk size // 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, RaftServerConfigKeys.Log.setWriteBufferSize(properties,
SizeInBytes.valueOf(maxChunkSize)); SizeInBytes.valueOf(maxChunkSize));

View File

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

View File

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

View File

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

View File

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

View File

@ -17,6 +17,7 @@
package org.apache.hadoop.ozone.client.rpc; 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.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; 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(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
conf.setQuietMode(false); 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 = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(7).build();
cluster.waitForClusterToBeReady(); cluster.waitForClusterToBeReady();
//the easiest way to create an open container is creating a key //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.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID; 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.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneCluster;
@ -83,15 +82,11 @@ public void init() throws Exception {
maxRetries = 100; maxRetries = 100;
chunkSize = (int) OzoneConsts.MB; chunkSize = (int) OzoneConsts.MB;
blockSize = 4 * chunkSize; 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, conf.setTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WATCH_REQUEST_TIMEOUT, 5,
TimeUnit.SECONDS); TimeUnit.SECONDS);
conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS); conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
conf.setQuietMode(false); conf.setQuietMode(false);
conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, (4));
cluster = MiniOzoneCluster.newBuilder(conf) cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(6).build(); .setNumDatanodes(6).build();
cluster.waitForClusterToBeReady(); cluster.waitForClusterToBeReady();

View File

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

View File

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

View File

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

View File

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

View File

@ -22,6 +22,7 @@
import java.io.FileInputStream; import java.io.FileInputStream;
import org.apache.hadoop.conf.Configuration; 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.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@ -106,8 +107,9 @@ public Void call() throws Exception {
.createKey(keyName, dataFile.length(), replicationType, .createKey(keyName, dataFile.length(), replicationType,
replicationFactor); replicationFactor);
FileInputStream fileInputStream = new FileInputStream(dataFile); FileInputStream fileInputStream = new FileInputStream(dataFile);
IOUtils.copyBytes(fileInputStream, outputStream, IOUtils.copyBytes(fileInputStream, outputStream, (int) conf
conf.getInt(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT)); .getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT,
StorageUnit.BYTES));
outputStream.close(); outputStream.close();
fileInputStream.close(); fileInputStream.close();
return null; return null;

View File

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