HDFS-12851. Ozone: Upgrade to latest ratis build. Contributed by Mukul Kumar Singh

This commit is contained in:
Tsz-Wo Nicholas Sze 2017-11-27 13:36:16 -08:00 committed by Owen O'Malley
parent ea86945754
commit 86802a461c
6 changed files with 58 additions and 18 deletions

View File

@ -48,9 +48,14 @@ 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";
// 1 MB by default
public static final int OZONE_SCM_CHUNK_SIZE_DEFAULT = 1 * 1024 * 1024;
public static final int OZONE_SCM_CHUNK_MAX_SIZE = 1 * 1024 * 1024;
// 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_RATIS_SEGMENT_SIZE_KEY =
"ozone.scm.ratis.segment.size";
public static final int OZONE_SCM_RATIS_SEGMENT_SIZE_DEFAULT =
128 * 1024 * 1024;
public static final String OZONE_SCM_CLIENT_PORT_KEY =
"ozone.scm.client.port";

View File

@ -144,6 +144,9 @@ public final class XceiverClientRatis extends XceiverClientSpi {
@Override
public void connect() throws Exception {
LOG.debug("Connecting to pipeline:{} leader:{}",
getPipeline().getPipelineName(),
RatisHelper.toRaftPeerId(pipeline.getLeader()));
if (!client.compareAndSet(null,
RatisHelper.newRaftClient(rpcType, getPipeline()))) {
throw new IllegalStateException("Client is already connected.");

View File

@ -20,7 +20,6 @@ package org.apache.ratis;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.ratis.client.ClientFactory;
import org.apache.ratis.client.RaftClient;
import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.protocol.RaftGroup;
@ -74,7 +73,7 @@ public interface RatisHelper {
* It should be changed to a unique id for each group.
*/
RaftGroupId DUMMY_GROUP_ID =
RaftGroupId.valueOf(ByteString.copyFromUtf8("AOZONERATISGROUP"));
RaftGroupId.valueOf(ByteString.copyFromUtf8("AOzoneRatisGroup"));
RaftGroup EMPTY_GROUP = new RaftGroup(DUMMY_GROUP_ID,
Collections.emptyList());
@ -113,10 +112,9 @@ public interface RatisHelper {
RpcType rpcType, RaftPeerId leader, RaftGroup group) {
LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, leader, group);
final RaftProperties properties = new RaftProperties();
final ClientFactory factory = ClientFactory.cast(rpcType.newFactory(null));
RaftConfigKeys.Rpc.setType(properties, rpcType);
return RaftClient.newBuilder()
.setClientRpc(factory.newRaftClientRpc())
.setRaftGroup(group)
.setLeaderId(leader)
.setProperties(properties)

View File

@ -102,7 +102,7 @@ public class Dispatcher implements ContainerDispatcher {
(cmdType == Type.ListContainer) ||
(cmdType == Type.UpdateContainer) ||
(cmdType == Type.CloseContainer)) {
return containerProcessHandler(msg);
resp = containerProcessHandler(msg);
}
if ((cmdType == Type.PutKey) ||

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.ozone.container.common.transport.server
.XceiverServerSpi;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.ratis.RaftConfigKeys;
import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.grpc.GrpcConfigKeys;
@ -37,6 +38,7 @@ import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.RaftServerConfigKeys;
import org.apache.ratis.util.SizeInBytes;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -57,23 +59,43 @@ public final class XceiverServerRatis implements XceiverServerSpi {
private final RaftServer server;
private XceiverServerRatis(DatanodeID id, int port, String storageDir,
ContainerDispatcher dispatcher, RpcType rpcType) throws IOException {
ContainerDispatcher dispatcher, RpcType rpcType, int maxChunkSize,
int raftSegmentSize) throws IOException {
Objects.requireNonNull(id, "id == null");
this.port = port;
RaftProperties serverProperties = newRaftProperties(rpcType, port,
storageDir, maxChunkSize, raftSegmentSize);
this.server = RaftServer.newBuilder()
.setServerId(RatisHelper.toRaftPeerId(id))
.setGroup(RatisHelper.emptyRaftGroup())
.setProperties(newRaftProperties(rpcType, port, storageDir))
.setProperties(serverProperties)
.setStateMachine(new ContainerStateMachine(dispatcher))
.build();
}
static RaftProperties newRaftProperties(
RpcType rpc, int port, String storageDir) {
private static RaftProperties newRaftProperties(
RpcType rpc, int port, String storageDir, int scmChunkSize,
int raftSegmentSize) {
final RaftProperties properties = new RaftProperties();
RaftServerConfigKeys.Log.Appender.setBatchEnabled(properties, true);
RaftServerConfigKeys.Log.Appender.setBufferCapacity(properties,
SizeInBytes.valueOf(raftSegmentSize));
RaftServerConfigKeys.Log.setWriteBufferSize(properties,
SizeInBytes.valueOf(scmChunkSize));
RaftServerConfigKeys.Log.setPreallocatedSize(properties,
SizeInBytes.valueOf(raftSegmentSize));
RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
SizeInBytes.valueOf(raftSegmentSize));
RaftServerConfigKeys.setStorageDir(properties, new File(storageDir));
RaftConfigKeys.Rpc.setType(properties, rpc);
//TODO: change these configs to setter after RATIS-154
properties.setInt("raft.server.log.segment.cache.num.max", 2);
properties.setInt("raft.grpc.message.size.max",
scmChunkSize + raftSegmentSize);
properties.setInt("raft.server.rpc.timeout.min", 500);
properties.setInt("raft.server.rpc.timeout.max", 600);
if (rpc == SupportedRpcType.GRPC) {
GrpcConfigKeys.Server.setPort(properties, port);
} else {
@ -108,6 +130,10 @@ public final class XceiverServerRatis implements XceiverServerSpi {
OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
final int raftSegmentSize =
ozoneConf.getInt(ScmConfigKeys.OZONE_SCM_RATIS_SEGMENT_SIZE_KEY,
ScmConfigKeys.OZONE_SCM_RATIS_SEGMENT_SIZE_DEFAULT);
final int maxChunkSize = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE;
// Get an available port on current node and
// use that as the container port
@ -133,7 +159,7 @@ public final class XceiverServerRatis implements XceiverServerSpi {
}
datanodeID.setRatisPort(localPort);
return new XceiverServerRatis(datanodeID, localPort, storageDir,
dispatcher, rpc);
dispatcher, rpc, maxChunkSize, raftSegmentSize);
}
@Override

View File

@ -675,15 +675,23 @@
</property>
<property>
<name>ozone.scm.chunk.size</name>
<value>1048576</value>
<value>16777216</value>
<tag>OZONE, SCM, CONTAINER, PERFORMANCE</tag>
<description>
The chunk size for reading/writing chunk operations in bytes.
The chunk size defaults to 1MB. If the value configured is more than the
maximum size (1MB), it will be reset to the maximum size. This maps to the
network packet sizes and file write operations in the client to datanode
protocol.
The chunk size defaults to 8MB. If the value configured is more than the
maximum size (16MB), it will be reset to the maximum size. This maps to
the network packet sizes and file write operations in the client to
datanode protocol.
</description>
</property>
<property>
<name>ozone.scm.ratis.segment.size</name>
<value>134217728</value>
<tag>OZONE, SCM, CONTAINER, PERFORMANCE</tag>
<description>
The size of the raft segment used by Apache Ratis on datanodes.
</description>
</property>
<property>