提交 461b140d 编写于 作者: J JackieTien97

Revert "[IOTDB-6116] Disassociate the IoTConsensus retry logic from the forkjoinPool (#10872)"

This reverts commit b4455404.
上级 0cd30225
...@@ -22,8 +22,6 @@ package org.apache.iotdb.consensus.iot; ...@@ -22,8 +22,6 @@ package org.apache.iotdb.consensus.iot;
import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.IClientManager;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.concurrent.ThreadName;
import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.exception.StartupException; import org.apache.iotdb.commons.exception.StartupException;
import org.apache.iotdb.commons.service.RegisterManager; import org.apache.iotdb.commons.service.RegisterManager;
...@@ -66,8 +64,6 @@ import java.util.ArrayList; ...@@ -66,8 +64,6 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
public class IoTConsensus implements IConsensus { public class IoTConsensus implements IConsensus {
...@@ -85,7 +81,6 @@ public class IoTConsensus implements IConsensus { ...@@ -85,7 +81,6 @@ public class IoTConsensus implements IConsensus {
private final IoTConsensusConfig config; private final IoTConsensusConfig config;
private final IClientManager<TEndPoint, AsyncIoTConsensusServiceClient> clientManager; private final IClientManager<TEndPoint, AsyncIoTConsensusServiceClient> clientManager;
private final IClientManager<TEndPoint, SyncIoTConsensusServiceClient> syncClientManager; private final IClientManager<TEndPoint, SyncIoTConsensusServiceClient> syncClientManager;
private final ScheduledExecutorService retryService;
public IoTConsensus(ConsensusConfig config, Registry registry) { public IoTConsensus(ConsensusConfig config, Registry registry) {
this.thisNode = config.getThisNodeEndPoint(); this.thisNode = config.getThisNodeEndPoint();
...@@ -102,9 +97,6 @@ public class IoTConsensus implements IConsensus { ...@@ -102,9 +97,6 @@ public class IoTConsensus implements IConsensus {
new IClientManager.Factory<TEndPoint, SyncIoTConsensusServiceClient>() new IClientManager.Factory<TEndPoint, SyncIoTConsensusServiceClient>()
.createClientManager( .createClientManager(
new SyncIoTConsensusServiceClientPoolFactory(config.getIoTConsensusConfig())); new SyncIoTConsensusServiceClientPoolFactory(config.getIoTConsensusConfig()));
this.retryService =
IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
ThreadName.LOG_DISPATCHER_RETRY_EXECUTOR.getName());
// init IoTConsensus memory manager // init IoTConsensus memory manager
IoTConsensusMemoryManager.getInstance() IoTConsensusMemoryManager.getInstance()
.init( .init(
...@@ -141,7 +133,6 @@ public class IoTConsensus implements IConsensus { ...@@ -141,7 +133,6 @@ public class IoTConsensus implements IConsensus {
new Peer(consensusGroupId, thisNodeId, thisNode), new Peer(consensusGroupId, thisNodeId, thisNode),
new ArrayList<>(), new ArrayList<>(),
registry.apply(consensusGroupId), registry.apply(consensusGroupId),
retryService,
clientManager, clientManager,
syncClientManager, syncClientManager,
config); config);
...@@ -158,13 +149,6 @@ public class IoTConsensus implements IConsensus { ...@@ -158,13 +149,6 @@ public class IoTConsensus implements IConsensus {
clientManager.close(); clientManager.close();
syncClientManager.close(); syncClientManager.close();
registerManager.deregisterAll(); registerManager.deregisterAll();
retryService.shutdown();
try {
retryService.awaitTermination(5, TimeUnit.SECONDS);
} catch (InterruptedException e) {
logger.warn("{}: interrupted when shutting down add Executor with exception {}", this, e);
Thread.currentThread().interrupt();
}
} }
@Override @Override
...@@ -230,7 +214,6 @@ public class IoTConsensus implements IConsensus { ...@@ -230,7 +214,6 @@ public class IoTConsensus implements IConsensus {
new Peer(groupId, thisNodeId, thisNode), new Peer(groupId, thisNodeId, thisNode),
peers, peers,
registry.apply(groupId), registry.apply(groupId),
retryService,
clientManager, clientManager,
syncClientManager, syncClientManager,
config); config);
......
...@@ -80,7 +80,6 @@ import java.util.LinkedList; ...@@ -80,7 +80,6 @@ import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.PriorityQueue; import java.util.PriorityQueue;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
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.atomic.AtomicLong;
import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Condition;
...@@ -113,14 +112,12 @@ public class IoTConsensusServerImpl { ...@@ -113,14 +112,12 @@ public class IoTConsensusServerImpl {
private final IClientManager<TEndPoint, SyncIoTConsensusServiceClient> syncClientManager; private final IClientManager<TEndPoint, SyncIoTConsensusServiceClient> syncClientManager;
private final IoTConsensusServerMetrics ioTConsensusServerMetrics; private final IoTConsensusServerMetrics ioTConsensusServerMetrics;
private final String consensusGroupId; private final String consensusGroupId;
private final ScheduledExecutorService retryService;
public IoTConsensusServerImpl( public IoTConsensusServerImpl(
String storageDir, String storageDir,
Peer thisNode, Peer thisNode,
List<Peer> configuration, List<Peer> configuration,
IStateMachine stateMachine, IStateMachine stateMachine,
ScheduledExecutorService retryService,
IClientManager<TEndPoint, AsyncIoTConsensusServiceClient> clientManager, IClientManager<TEndPoint, AsyncIoTConsensusServiceClient> clientManager,
IClientManager<TEndPoint, SyncIoTConsensusServiceClient> syncClientManager, IClientManager<TEndPoint, SyncIoTConsensusServiceClient> syncClientManager,
IoTConsensusConfig config) { IoTConsensusConfig config) {
...@@ -136,7 +133,6 @@ public class IoTConsensusServerImpl { ...@@ -136,7 +133,6 @@ public class IoTConsensusServerImpl {
} else { } else {
persistConfiguration(); persistConfiguration();
} }
this.retryService = retryService;
this.config = config; this.config = config;
this.consensusGroupId = thisNode.getGroupId().toString(); this.consensusGroupId = thisNode.getGroupId().toString();
consensusReqReader = (ConsensusReqReader) stateMachine.read(new GetConsensusReqReaderPlan()); consensusReqReader = (ConsensusReqReader) stateMachine.read(new GetConsensusReqReaderPlan());
...@@ -736,10 +732,6 @@ public class IoTConsensusServerImpl { ...@@ -736,10 +732,6 @@ public class IoTConsensusServerImpl {
return searchIndex; return searchIndex;
} }
public ScheduledExecutorService getRetryService() {
return retryService;
}
public boolean isReadOnly() { public boolean isReadOnly() {
return stateMachine.isReadOnly(); return stateMachine.isReadOnly();
} }
......
...@@ -29,7 +29,7 @@ import org.apache.thrift.async.AsyncMethodCallback; ...@@ -29,7 +29,7 @@ import org.apache.thrift.async.AsyncMethodCallback;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.concurrent.TimeUnit; import java.util.concurrent.CompletableFuture;
public class DispatchLogHandler implements AsyncMethodCallback<TSyncLogEntriesRes> { public class DispatchLogHandler implements AsyncMethodCallback<TSyncLogEntriesRes> {
...@@ -88,29 +88,31 @@ public class DispatchLogHandler implements AsyncMethodCallback<TSyncLogEntriesRe ...@@ -88,29 +88,31 @@ public class DispatchLogHandler implements AsyncMethodCallback<TSyncLogEntriesRe
} }
private void sleepCorrespondingTimeAndRetryAsynchronous() { private void sleepCorrespondingTimeAndRetryAsynchronous() {
long sleepTime = // TODO handle forever retry
Math.min( CompletableFuture.runAsync(
(long) () -> {
(thread.getConfig().getReplication().getBasicRetryWaitTimeMs() try {
* Math.pow(2, retryCount)), long defaultSleepTime =
thread.getConfig().getReplication().getMaxRetryWaitTimeMs()); (long)
thread (thread.getConfig().getReplication().getBasicRetryWaitTimeMs()
.getImpl() * Math.pow(2, retryCount));
.getRetryService() Thread.sleep(
.schedule( Math.min(
() -> { defaultSleepTime, thread.getConfig().getReplication().getMaxRetryWaitTimeMs()));
if (thread.isStopped()) { } catch (InterruptedException e) {
logger.debug( Thread.currentThread().interrupt();
"LogDispatcherThread {} has been stopped, " logger.warn("Unexpected interruption during retry pending batch");
+ "we will not retrying this Batch {} after {} times", }
thread.getPeer(), if (thread.isStopped()) {
batch, logger.debug(
retryCount); "LogDispatcherThread {} has been stopped, "
} else { + "we will not retrying this Batch {} after {} times",
thread.sendBatchAsync(batch, this); thread.getPeer(),
} batch,
}, retryCount);
sleepTime, } else {
TimeUnit.MILLISECONDS); thread.sendBatchAsync(batch, this);
}
});
} }
} }
...@@ -299,10 +299,6 @@ public class LogDispatcher { ...@@ -299,10 +299,6 @@ public class LogDispatcher {
return stopped; return stopped;
} }
public IoTConsensusServerImpl getImpl() {
return impl;
}
@Override @Override
public void run() { public void run() {
logger.info("{}: Dispatcher for {} starts", impl.getThisNode(), peer); logger.info("{}: Dispatcher for {} starts", impl.getThisNode(), peer);
......
...@@ -93,6 +93,7 @@ import java.util.Collections; ...@@ -93,6 +93,7 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
...@@ -125,6 +126,7 @@ class RatisConsensus implements IConsensus { ...@@ -125,6 +126,7 @@ class RatisConsensus implements IConsensus {
/** TODO make it configurable */ /** TODO make it configurable */
private static final int DEFAULT_WAIT_LEADER_READY_TIMEOUT = (int) TimeUnit.SECONDS.toMillis(20); private static final int DEFAULT_WAIT_LEADER_READY_TIMEOUT = (int) TimeUnit.SECONDS.toMillis(20);
private final ExecutorService addExecutor;
private final ScheduledExecutorService diskGuardian; private final ScheduledExecutorService diskGuardian;
private final long triggerSnapshotThreshold; private final long triggerSnapshotThreshold;
...@@ -154,6 +156,7 @@ class RatisConsensus implements IConsensus { ...@@ -154,6 +156,7 @@ class RatisConsensus implements IConsensus {
this.ratisMetricSet = new RatisMetricSet(); this.ratisMetricSet = new RatisMetricSet();
this.triggerSnapshotThreshold = this.config.getImpl().getTriggerSnapshotFileSize(); this.triggerSnapshotThreshold = this.config.getImpl().getTriggerSnapshotFileSize();
addExecutor = IoTDBThreadPoolFactory.newCachedThreadPool(ThreadName.RATIS_ADD.getName());
diskGuardian = diskGuardian =
IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor( IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
ThreadName.RATIS_BG_DISK_GUARDIAN.getName()); ThreadName.RATIS_BG_DISK_GUARDIAN.getName());
...@@ -186,8 +189,10 @@ class RatisConsensus implements IConsensus { ...@@ -186,8 +189,10 @@ class RatisConsensus implements IConsensus {
@Override @Override
public void stop() throws IOException { public void stop() throws IOException {
addExecutor.shutdown();
diskGuardian.shutdown(); diskGuardian.shutdown();
try { try {
addExecutor.awaitTermination(5, TimeUnit.SECONDS);
diskGuardian.awaitTermination(5, TimeUnit.SECONDS); diskGuardian.awaitTermination(5, TimeUnit.SECONDS);
} catch (InterruptedException e) { } catch (InterruptedException e) {
logger.warn("{}: interrupted when shutting down add Executor with exception {}", this, e); logger.warn("{}: interrupted when shutting down add Executor with exception {}", this, e);
...@@ -195,8 +200,8 @@ class RatisConsensus implements IConsensus { ...@@ -195,8 +200,8 @@ class RatisConsensus implements IConsensus {
} finally { } finally {
clientManager.close(); clientManager.close();
server.close(); server.close();
MetricService.getInstance().removeMetricSet(this.ratisMetricSet);
} }
MetricService.getInstance().removeMetricSet(this.ratisMetricSet);
} }
private boolean shouldRetry(RaftClientReply reply) { private boolean shouldRetry(RaftClientReply reply) {
......
...@@ -1070,7 +1070,7 @@ public class IoTDBConfig { ...@@ -1070,7 +1070,7 @@ public class IoTDBConfig {
// IoTConsensus Config // IoTConsensus Config
private int maxLogEntriesNumPerBatch = 1024; private int maxLogEntriesNumPerBatch = 1024;
private int maxSizePerBatch = 16 * 1024 * 1024; private int maxSizePerBatch = 16 * 1024 * 1024;
private int maxPendingBatchesNum = 5; private int maxPendingBatchesNum = 12;
private double maxMemoryRatioForQueue = 0.6; private double maxMemoryRatioForQueue = 0.6;
/** Pipe related */ /** Pipe related */
......
...@@ -103,13 +103,13 @@ public enum ThreadName { ...@@ -103,13 +103,13 @@ public enum ThreadName {
IOT_CONSENSUS_RPC_PROCESSOR("IoTConsensusRPC-Processor"), IOT_CONSENSUS_RPC_PROCESSOR("IoTConsensusRPC-Processor"),
ASYNC_DATANODE_IOT_CONSENSUS_CLIENT_POOL("AsyncDataNodeIoTConsensusServiceClientPool"), ASYNC_DATANODE_IOT_CONSENSUS_CLIENT_POOL("AsyncDataNodeIoTConsensusServiceClientPool"),
LOG_DISPATCHER("LogDispatcher"), LOG_DISPATCHER("LogDispatcher"),
LOG_DISPATCHER_RETRY_EXECUTOR("LogDispatcherRetryExecutor"),
// -------------------------- Ratis -------------------------- // -------------------------- Ratis --------------------------
// NOTICE: The thread name of ratis cannot be edited here! // NOTICE: The thread name of ratis cannot be edited here!
// We list the thread name here just for distinguishing what module the thread belongs to. // We list the thread name here just for distinguishing what module the thread belongs to.
RAFT_SERVER_PROXY_EXECUTOR("\\d+-impl-thread"), RAFT_SERVER_PROXY_EXECUTOR("\\d+-impl-thread"),
RAFT_SERVER_EXECUTOR("\\d+-server-thread"), RAFT_SERVER_EXECUTOR("\\d+-server-thread"),
RAFT_SERVER_CLIENT_EXECUTOR("\\d+-client-thread"), RAFT_SERVER_CLIENT_EXECUTOR("\\d+-client-thread"),
RATIS_ADD("Ratis-Add"),
SEGMENT_RAFT_WORKER("SegmentedRaftLogWorker"), SEGMENT_RAFT_WORKER("SegmentedRaftLogWorker"),
STATE_MACHINE_UPDATER("StateMachineUpdater"), STATE_MACHINE_UPDATER("StateMachineUpdater"),
FOLLOWER_STATE("FollowerState"), FOLLOWER_STATE("FollowerState"),
...@@ -236,8 +236,7 @@ public enum ThreadName { ...@@ -236,8 +236,7 @@ public enum ThreadName {
IOT_CONSENSUS_RPC_SERVICE, IOT_CONSENSUS_RPC_SERVICE,
IOT_CONSENSUS_RPC_PROCESSOR, IOT_CONSENSUS_RPC_PROCESSOR,
ASYNC_DATANODE_IOT_CONSENSUS_CLIENT_POOL, ASYNC_DATANODE_IOT_CONSENSUS_CLIENT_POOL,
LOG_DISPATCHER, LOG_DISPATCHER));
LOG_DISPATCHER_RETRY_EXECUTOR));
private static final Set<ThreadName> ratisThreadNames = private static final Set<ThreadName> ratisThreadNames =
new HashSet<>( new HashSet<>(
...@@ -245,6 +244,7 @@ public enum ThreadName { ...@@ -245,6 +244,7 @@ public enum ThreadName {
RAFT_SERVER_PROXY_EXECUTOR, RAFT_SERVER_PROXY_EXECUTOR,
RAFT_SERVER_EXECUTOR, RAFT_SERVER_EXECUTOR,
RAFT_SERVER_CLIENT_EXECUTOR, RAFT_SERVER_CLIENT_EXECUTOR,
RATIS_ADD,
SEGMENT_RAFT_WORKER, SEGMENT_RAFT_WORKER,
STATE_MACHINE_UPDATER, STATE_MACHINE_UPDATER,
FOLLOWER_STATE, FOLLOWER_STATE,
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册