|
@@ -27,8 +27,13 @@ import java.nio.charset.StandardCharsets;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Optional;
|
|
import java.util.UUID;
|
|
import java.util.UUID;
|
|
|
|
+import java.util.concurrent.Executors;
|
|
|
|
+import java.util.concurrent.ScheduledExecutorService;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
@@ -41,6 +46,11 @@ import org.apache.ratis.client.RaftClientConfigKeys;
|
|
import org.apache.ratis.conf.RaftProperties;
|
|
import org.apache.ratis.conf.RaftProperties;
|
|
import org.apache.ratis.grpc.GrpcConfigKeys;
|
|
import org.apache.ratis.grpc.GrpcConfigKeys;
|
|
import org.apache.ratis.netty.NettyConfigKeys;
|
|
import org.apache.ratis.netty.NettyConfigKeys;
|
|
|
|
+import org.apache.ratis.proto.RaftProtos.RoleInfoProto;
|
|
|
|
+import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
|
|
|
|
+import org.apache.ratis.protocol.ClientId;
|
|
|
|
+import org.apache.ratis.protocol.GroupInfoReply;
|
|
|
|
+import org.apache.ratis.protocol.GroupInfoRequest;
|
|
import org.apache.ratis.protocol.RaftGroup;
|
|
import org.apache.ratis.protocol.RaftGroup;
|
|
import org.apache.ratis.protocol.RaftGroupId;
|
|
import org.apache.ratis.protocol.RaftGroupId;
|
|
import org.apache.ratis.protocol.RaftPeer;
|
|
import org.apache.ratis.protocol.RaftPeer;
|
|
@@ -50,6 +60,7 @@ import org.apache.ratis.rpc.SupportedRpcType;
|
|
import org.apache.ratis.server.RaftServer;
|
|
import org.apache.ratis.server.RaftServer;
|
|
import org.apache.ratis.server.RaftServerConfigKeys;
|
|
import org.apache.ratis.server.RaftServerConfigKeys;
|
|
import org.apache.ratis.statemachine.impl.BaseStateMachine;
|
|
import org.apache.ratis.statemachine.impl.BaseStateMachine;
|
|
|
|
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
|
import org.apache.ratis.util.LifeCycle;
|
|
import org.apache.ratis.util.LifeCycle;
|
|
import org.apache.ratis.util.SizeInBytes;
|
|
import org.apache.ratis.util.SizeInBytes;
|
|
import org.apache.ratis.util.TimeDuration;
|
|
import org.apache.ratis.util.TimeDuration;
|
|
@@ -69,7 +80,22 @@ public final class OzoneManagerRatisServer {
|
|
private final RaftGroupId raftGroupId;
|
|
private final RaftGroupId raftGroupId;
|
|
private final RaftGroup raftGroup;
|
|
private final RaftGroup raftGroup;
|
|
private final RaftPeerId raftPeerId;
|
|
private final RaftPeerId raftPeerId;
|
|
|
|
+
|
|
private final OzoneManagerProtocol ozoneManager;
|
|
private final OzoneManagerProtocol ozoneManager;
|
|
|
|
+ private final ClientId clientId = ClientId.randomId();
|
|
|
|
+
|
|
|
|
+ private final ScheduledExecutorService scheduledRoleChecker;
|
|
|
|
+ private long roleCheckInitialDelayMs = 1000; // 1 second default
|
|
|
|
+ private long roleCheckIntervalMs;
|
|
|
|
+ private ReentrantReadWriteLock roleCheckLock = new ReentrantReadWriteLock();
|
|
|
|
+ private Optional<RaftPeerRole> cachedPeerRole = Optional.empty();
|
|
|
|
+ private Optional<RaftPeerId> cachedLeaderPeerId = Optional.empty();
|
|
|
|
+
|
|
|
|
+ private static final AtomicLong CALL_ID_COUNTER = new AtomicLong();
|
|
|
|
+
|
|
|
|
+ private static long nextCallId() {
|
|
|
|
+ return CALL_ID_COUNTER.getAndIncrement() & Long.MAX_VALUE;
|
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
/**
|
|
* Returns an OM Ratis server.
|
|
* Returns an OM Ratis server.
|
|
@@ -108,6 +134,20 @@ public final class OzoneManagerRatisServer {
|
|
.setProperties(serverProperties)
|
|
.setProperties(serverProperties)
|
|
.setStateMachine(getStateMachine(this.raftGroupId))
|
|
.setStateMachine(getStateMachine(this.raftGroupId))
|
|
.build();
|
|
.build();
|
|
|
|
+
|
|
|
|
+ // Run a scheduler to check and update the server role on the leader
|
|
|
|
+ // periodically
|
|
|
|
+ this.scheduledRoleChecker = Executors.newSingleThreadScheduledExecutor();
|
|
|
|
+ this.scheduledRoleChecker.scheduleWithFixedDelay(new Runnable() {
|
|
|
|
+ @Override
|
|
|
|
+ public void run() {
|
|
|
|
+ // Run this check only on the leader OM
|
|
|
|
+ if (cachedPeerRole.isPresent() &&
|
|
|
|
+ cachedPeerRole.get() == RaftPeerRole.LEADER) {
|
|
|
|
+ updateServerRole();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }, roleCheckInitialDelayMs, roleCheckIntervalMs, TimeUnit.MILLISECONDS);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -156,7 +196,11 @@ public final class OzoneManagerRatisServer {
|
|
* Returns OzoneManager StateMachine.
|
|
* Returns OzoneManager StateMachine.
|
|
*/
|
|
*/
|
|
private BaseStateMachine getStateMachine(RaftGroupId gid) {
|
|
private BaseStateMachine getStateMachine(RaftGroupId gid) {
|
|
- return new OzoneManagerStateMachine(ozoneManager);
|
|
|
|
|
|
+ return new OzoneManagerStateMachine(this);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public OzoneManagerProtocol getOzoneManager() {
|
|
|
|
+ return ozoneManager;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -323,6 +367,19 @@ public final class OzoneManagerRatisServer {
|
|
RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
|
|
RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
|
|
nodeFailureTimeout);
|
|
nodeFailureTimeout);
|
|
|
|
|
|
|
|
+ TimeUnit roleCheckIntervalUnit =
|
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_SERVER_ROLE_CHECK_INTERVAL_DEFAULT
|
|
|
|
+ .getUnit();
|
|
|
|
+ long roleCheckIntervalDuration = conf.getTimeDuration(
|
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_SERVER_ROLE_CHECK_INTERVAL_KEY,
|
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_SERVER_ROLE_CHECK_INTERVAL_DEFAULT
|
|
|
|
+ .getDuration(), nodeFailureTimeoutUnit);
|
|
|
|
+ this.roleCheckIntervalMs = TimeDuration.valueOf(
|
|
|
|
+ roleCheckIntervalDuration, roleCheckIntervalUnit)
|
|
|
|
+ .toLong(TimeUnit.MILLISECONDS);
|
|
|
|
+ this.roleCheckInitialDelayMs = leaderElectionMinTimeout
|
|
|
|
+ .toLong(TimeUnit.MILLISECONDS);
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* TODO: when ratis snapshots are implemented, set snapshot threshold and
|
|
* TODO: when ratis snapshots are implemented, set snapshot threshold and
|
|
* queue size.
|
|
* queue size.
|
|
@@ -331,6 +388,104 @@ public final class OzoneManagerRatisServer {
|
|
return properties;
|
|
return properties;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Check the cached leader status.
|
|
|
|
+ * @return true if cached role is Leader, false otherwise.
|
|
|
|
+ */
|
|
|
|
+ private boolean checkCachedPeerRoleIsLeader() {
|
|
|
|
+ this.roleCheckLock.readLock().lock();
|
|
|
|
+ try {
|
|
|
|
+ if (cachedPeerRole.isPresent() &&
|
|
|
|
+ cachedPeerRole.get() == RaftPeerRole.LEADER) {
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+ return false;
|
|
|
|
+ } finally {
|
|
|
|
+ this.roleCheckLock.readLock().unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Check if the current OM node is the leader node.
|
|
|
|
+ * @return true if Leader, false otherwise.
|
|
|
|
+ */
|
|
|
|
+ public boolean isLeader() {
|
|
|
|
+ if (checkCachedPeerRoleIsLeader()) {
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Get the server role from ratis server and update the cached values.
|
|
|
|
+ updateServerRole();
|
|
|
|
+
|
|
|
|
+ // After updating the server role, check and return if leader or not.
|
|
|
|
+ return checkCachedPeerRoleIsLeader();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the suggested leader peer id.
|
|
|
|
+ * @return RaftPeerId of the suggested leader node.
|
|
|
|
+ */
|
|
|
|
+ public Optional<RaftPeerId> getCachedLeaderPeerId() {
|
|
|
|
+ this.roleCheckLock.readLock().lock();
|
|
|
|
+ try {
|
|
|
|
+ return cachedLeaderPeerId;
|
|
|
|
+ } finally {
|
|
|
|
+ this.roleCheckLock.readLock().unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the gorup info (peer role and leader peer id) from Ratis server and
|
|
|
|
+ * update the OM server role.
|
|
|
|
+ */
|
|
|
|
+ public void updateServerRole() {
|
|
|
|
+ try {
|
|
|
|
+ GroupInfoReply groupInfo = getGroupInfo();
|
|
|
|
+ RoleInfoProto roleInfoProto = groupInfo.getRoleInfoProto();
|
|
|
|
+ RaftPeerRole thisNodeRole = roleInfoProto.getRole();
|
|
|
|
+
|
|
|
|
+ if (thisNodeRole.equals(RaftPeerRole.LEADER)) {
|
|
|
|
+ setServerRole(thisNodeRole, raftPeerId);
|
|
|
|
+
|
|
|
|
+ } else if (thisNodeRole.equals(RaftPeerRole.FOLLOWER)) {
|
|
|
|
+ ByteString leaderNodeId = roleInfoProto.getFollowerInfo()
|
|
|
|
+ .getLeaderInfo().getId().getId();
|
|
|
|
+ RaftPeerId leaderPeerId = RaftPeerId.valueOf(leaderNodeId);
|
|
|
|
+
|
|
|
|
+ setServerRole(thisNodeRole, leaderPeerId);
|
|
|
|
+
|
|
|
|
+ } else {
|
|
|
|
+ setServerRole(thisNodeRole, null);
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Failed to retrieve RaftPeerRole. Setting cached role to " +
|
|
|
|
+ "{} and resetting leader info.", RaftPeerRole.UNRECOGNIZED, e);
|
|
|
|
+ setServerRole(null, null);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Set the current server role and the leader peer id.
|
|
|
|
+ */
|
|
|
|
+ private void setServerRole(RaftPeerRole currentRole,
|
|
|
|
+ RaftPeerId leaderPeerId) {
|
|
|
|
+ this.roleCheckLock.writeLock().lock();
|
|
|
|
+ try {
|
|
|
|
+ this.cachedPeerRole = Optional.ofNullable(currentRole);
|
|
|
|
+ this.cachedLeaderPeerId = Optional.ofNullable(leaderPeerId);
|
|
|
|
+ } finally {
|
|
|
|
+ this.roleCheckLock.writeLock().unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private GroupInfoReply getGroupInfo() throws IOException {
|
|
|
|
+ GroupInfoRequest groupInfoRequest = new GroupInfoRequest(clientId,
|
|
|
|
+ raftPeerId, raftGroupId, nextCallId());
|
|
|
|
+ GroupInfoReply groupInfo = server.getGroupInfo(groupInfoRequest);
|
|
|
|
+ return groupInfo;
|
|
|
|
+ }
|
|
|
|
+
|
|
public int getServerPort() {
|
|
public int getServerPort() {
|
|
return port;
|
|
return port;
|
|
}
|
|
}
|