656 lines
		
	
	
		
			19 KiB
		
	
	
	
		
			C++
		
	
	
			
		
		
	
	
			656 lines
		
	
	
		
			19 KiB
		
	
	
	
		
			C++
		
	
	
| /************************************************************************
 | |
|  *
 | |
|  * Copyright (c) 2016 Alibaba.com, Inc. All Rights Reserved
 | |
|  * $Id:  paxos_server.cc,v 1.0 08/02/2016 05:50:36 PM yingqiang.zyq(yingqiang.zyq@alibaba-inc.com) $
 | |
|  *
 | |
|  ************************************************************************/
 | |
| 
 | |
| /**
 | |
|  * @file paxos_server.cc
 | |
|  * @author yingqiang.zyq(yingqiang.zyq@alibaba-inc.com)
 | |
|  * @date 08/02/2016 05:50:36 PM
 | |
|  * @version 1.0
 | |
|  * @brief 
 | |
|  *
 | |
|  **/
 | |
| 
 | |
| #include "paxos_server.h"
 | |
| #include "paxos.h"
 | |
| #include "multi_process_queue.h"
 | |
| 
 | |
| namespace alisql {
 | |
| 
 | |
| uint64_t Server::getLastLogIndex()
 | |
| {
 | |
|   return paxos->log_->getLastLogIndex();
 | |
| }
 | |
| 
 | |
| uint64_t Server::getLastCachedLogIndex()
 | |
| {
 | |
|   return paxos->log_->getLastCachedLogIndex();
 | |
| }
 | |
| 
 | |
| /* Implement for LocalServer */
 | |
| LocalServer::LocalServer (uint64_t serverId)
 | |
|   :Server(serverId)
 | |
|    ,lastSyncedIndex(1)
 | |
|    ,logType(false)
 | |
|    ,learnerConnTimeout(0)
 | |
|    ,cidx(1000)
 | |
| {}
 | |
| 
 | |
| void LocalServer::beginLeadership(void *)
 | |
| {
 | |
|   lastSyncedIndex.store(getLastLogIndex());
 | |
| }
 | |
| 
 | |
| void LocalServer::sendMsg(void *ptr)
 | |
| {
 | |
|   PaxosMsg *msg= (PaxosMsg *)ptr;
 | |
|   msg->set_serverid(serverId);
 | |
|   /* TODO put the msg into service or drop it */
 | |
| }
 | |
| 
 | |
| uint64_t LocalServer::getLastAckEpoch() const
 | |
| {
 | |
|   return paxos->getCurrentEpoch();
 | |
| }
 | |
| 
 | |
| uint64_t LocalServer::getAppliedIndex() const
 | |
| {
 | |
|   return paxos->getAppliedIndex();
 | |
| }
 | |
| 
 | |
| void LocalServer::fillInfo(void *ptr)
 | |
| {
 | |
|   Paxos::ClusterInfoType ci;
 | |
|   std::vector<Paxos::ClusterInfoType> *cis= static_cast<std::vector<Paxos::ClusterInfoType> *>(ptr);
 | |
| 
 | |
|   ci.serverId= serverId;
 | |
|   ci.ipPort= strAddr;
 | |
|   ci.matchIndex= lastSyncedIndex.load();
 | |
|   ci.nextIndex= 0;
 | |
|   ci.role= Paxos::LEADER;
 | |
|   ci.hasVoted= 1;
 | |
|   ci.forceSync= forceSync;
 | |
|   ci.electionWeight= electionWeight;
 | |
|   ci.learnerSource= 0;
 | |
|   ci.appliedIndex= getAppliedIndex();
 | |
|   ci.pipelining= false;
 | |
|   ci.useApplied= false;
 | |
| 
 | |
|   cis->push_back(std::move(ci));
 | |
| }
 | |
| 
 | |
| void LocalServer::fillFollowerMeta(void *ptr)
 | |
| {
 | |
|   return;
 | |
| }
 | |
| 
 | |
| bool LocalServer::haveVote() const
 | |
| {
 | |
|   return paxos->votedFor_ == serverId;
 | |
| }
 | |
| 
 | |
| uint64_t LocalServer::appendLog(LogEntry &entry)
 | |
| {
 | |
|   return paxos->getLog()->appendWithCheck(entry);
 | |
| }
 | |
| 
 | |
| uint64_t LocalServer::writeLog(LogEntry &entry)
 | |
| {
 | |
|   uint64_t logIndex= appendLog(entry);
 | |
|   if (logIndex > 0)
 | |
|     writeLogDoneInternal(logIndex);
 | |
|     //writeLogDone(logIndex);
 | |
|   return logIndex;
 | |
| }
 | |
| 
 | |
| uint64_t LocalServer::writeLogDoneInternal(uint64_t logIndex, bool forceSend)
 | |
| {
 | |
|   bool ret= false;
 | |
|   for (;;)
 | |
|   {
 | |
|     uint64_t old= lastSyncedIndex.load();
 | |
|     if (old > logIndex || (old < logIndex && (ret= lastSyncedIndex.compare_exchange_weak(old, logIndex))))
 | |
|       break;
 | |
|   }
 | |
|   if (ret && paxos->getConsensusAsync())
 | |
|     paxos->cond_.notify_all();
 | |
| 
 | |
|   if (forceSend) /* for large trx, send directly after sync partial */
 | |
|   {
 | |
|     easy_warn_log("Server %d : writeLogDoneInternal logIndex:%ld\n", serverId, logIndex);
 | |
|     paxos->appendLog(false);
 | |
|   }
 | |
|   return 0;
 | |
| }
 | |
| 
 | |
| uint64_t LocalServer::writeLogDone(uint64_t logIndex)
 | |
| {
 | |
|   return writeLogDoneInternal(logIndex);
 | |
| }
 | |
| 
 | |
| void LocalServer::writeCacheLogDone()
 | |
| {
 | |
|   if (paxos->getReplicateWithCacheLog())
 | |
|     paxos->appendLog(false);
 | |
| }
 | |
| 
 | |
| uint64_t AliSQLServer::writeLogDone(uint64_t logIndex)
 | |
| {
 | |
|   bool ret= false;
 | |
|   for (;;)
 | |
|   {
 | |
|     uint64_t old= lastSyncedIndex.load();
 | |
|     if (old >= logIndex || (old < logIndex && (ret= lastSyncedIndex.compare_exchange_weak(old, logIndex))))
 | |
|       break;
 | |
|   }
 | |
|   if (ret && paxos->getConsensusAsync())
 | |
|     paxos->cond_.notify_all();
 | |
|   /*
 | |
|    * In AliSQLServer mode, we write local log first, so we do not need to tryUpdateCommitIndex here.
 | |
|    * Later, we will write local log and send msg in the same time, at that time we should call tryUpdateCommitIndex here.
 | |
|    */
 | |
|   int tmp= 0;
 | |
| 
 | |
|   tmp= paxos->tryUpdateCommitIndex();
 | |
|   easy_warn_log("Server %d : writeLogDone logIndex:%ld, tryUpdateCommitIndex return:%d\n", serverId, logIndex, tmp);
 | |
| 
 | |
|   if (paxos->getReplicateWithCacheLog() == false)
 | |
|     paxos->appendLog(false);
 | |
| 
 | |
|   return logIndex;
 | |
| }
 | |
| 
 | |
| void AliSQLServer::setLastNonCommitDepIndex(uint64_t logIndex)
 | |
| {
 | |
|   /* AliSQLServer do not append log by replicateLog */
 | |
|   paxos->cdrMgr_.setLastNonCommitDepIndex(logIndex);
 | |
| }
 | |
| 
 | |
| /* Implement for RemoteServer */
 | |
| RemoteServer::RemoteServer (uint64_t serverId)
 | |
|   :Server(serverId)
 | |
|    ,sendMsgQueue(nullptr)
 | |
|    ,nextIndex(1)
 | |
|    ,matchIndex(0)
 | |
|    ,lastAckEpoch(0)
 | |
|    ,hasVote(false)
 | |
|    ,isLeader(false)
 | |
|    ,isLearner(false)
 | |
|    ,hasMatched(false)
 | |
|    ,needAddr(true)
 | |
|    ,disablePipelining(false)
 | |
|    ,lostConnect(false)
 | |
|    ,netError(true)
 | |
|    ,isStop(false)
 | |
|    ,waitForReply(0)
 | |
|    ,guardId(1)
 | |
|    ,msgId(1)
 | |
|    ,appliedIndex(0)
 | |
|    ,lastEntrySize(0)
 | |
| {
 | |
|   addr.port= 0;
 | |
|   sendMsgQueue= std::make_shared<SingleProcessQueue<SendMsgTask>>();
 | |
|   lastSendTP= now();
 | |
|   lastMergeTP= now() - std::chrono::hours(1);
 | |
| }
 | |
| 
 | |
| RemoteServer::~RemoteServer ()
 | |
| {
 | |
|   stop(nullptr);
 | |
| }
 | |
| 
 | |
| void RemoteServer::beginRequestVote(void *)
 | |
| {
 | |
|   hasVote= false;
 | |
|   if (paxos)
 | |
|   {
 | |
|     lastAckEpoch.store(paxos->getCurrentEpoch());
 | |
|   }
 | |
| }
 | |
| 
 | |
| void RemoteServer::beginLeadership(void *skipReset)
 | |
| {
 | |
|   if (!skipReset)
 | |
|   {
 | |
|     nextIndex= getLastLogIndex() + 1;
 | |
|     if (isLearner && sendByAppliedIndex && paxos != nullptr)
 | |
|       nextIndex= paxos->getAppliedIndex() + 1;
 | |
|     resetMatchIndex(0);
 | |
|   }
 | |
| 
 | |
|   if (!isLearner || paxos->option.enableLearnerHeartbeat_)
 | |
|   {
 | |
|     isLeader= true;
 | |
|     //heartbeatTimer.resetTimer(false);
 | |
|     if (heartbeatTimer != nullptr)
 | |
|       heartbeatTimer->restart();
 | |
|   }
 | |
| 
 | |
|   waitForReply= 0;
 | |
|   lostConnect.store(false);
 | |
|   sendMsgQueue->start();
 | |
|   /*
 | |
|   if (paxos)
 | |
|   {
 | |
|     lastAckEpoch.store(paxos->getCurrentEpoch());
 | |
|   }
 | |
|   */
 | |
|   isStop.store(false);
 | |
| }
 | |
| 
 | |
| void RemoteServer::stepDown(void *)
 | |
| {
 | |
|   resetMatchIndex(0);
 | |
|   nextIndex= 1;
 | |
|   isLeader= false;
 | |
|   /* Stop heartbeatTimer when we're step down. */
 | |
|   if (heartbeatTimer != nullptr)
 | |
|     heartbeatTimer->stop();
 | |
| }
 | |
| 
 | |
| void RemoteServer::stop(void *)
 | |
| {
 | |
|   bool rStop = false;
 | |
|   if (isStop.compare_exchange_weak(rStop, true))
 | |
|   {
 | |
|     stepDown(NULL);
 | |
|     if (sendMsgQueue && sendMsgQueue->stop(false))
 | |
|     {
 | |
|       sendMsgQueue.reset();
 | |
|       sendMsgQueue= nullptr;
 | |
|     }
 | |
|     disconnect(NULL);
 | |
|     if (srv != nullptr)
 | |
|       srv->getEasyNet()->delConnDataById(serverId);
 | |
|   }
 | |
| }
 | |
| 
 | |
| uint64_t RemoteServer::getConnTimeout()
 | |
| {
 | |
|   if (!isLearner || !paxos || paxos->getLocalServer()->learnerConnTimeout == 0)
 | |
|     return paxos? paxos->getHeartbeatTimeout()/4: 1000;
 | |
|   else
 | |
|     return paxos->getLocalServer()->learnerConnTimeout;
 | |
| }
 | |
| 
 | |
| void RemoteServer::connect(void *ptr)
 | |
| {
 | |
|   if (addr.port == 0)
 | |
|   {
 | |
|     uint64_t cidx;
 | |
|     if (paxos && paxos->getEnableDynamicEasyIndex())
 | |
|     {
 | |
|       // make sure cidx > 256, serverId < 1000
 | |
|       cidx = paxos->getLocalServer()->cidx + serverId;
 | |
|       paxos->getLocalServer()->cidx += 1000;
 | |
|     }
 | |
|     else
 | |
|       cidx = serverId;
 | |
|     easy_info_log("Connect server %d, cidx %llu", serverId, cidx);
 | |
|     addr= srv->createConnection(strAddr, getSharedThis(), getConnTimeout(), cidx);
 | |
|   }
 | |
| }
 | |
| 
 | |
| void RemoteServer::disconnect(void *ptr)
 | |
| {
 | |
|   if (addr.port != 0)
 | |
|   {
 | |
|     srv->disableConnnection(addr);
 | |
|     addr.port= 0;
 | |
|   }
 | |
| }
 | |
| 
 | |
| void RemoteServer::sendMsg(void *ptr)
 | |
| {
 | |
|   sendMsgFunc(false, false, ptr);
 | |
| }
 | |
| 
 | |
| void RemoteServer::sendMsgFunc(bool lockless, bool force, void *ptr)
 | |
| {
 | |
|   if (isLearner)
 | |
|   {
 | |
|     if ((learnerSource == 0 && paxos && paxos->getState() != Paxos::LEADER))
 | |
|       return;
 | |
| 
 | |
|     if (learnerSource != 0 && paxos && (paxos->getLocalServer()->serverId != learnerSource))
 | |
|       return;
 | |
|   }
 | |
|   if (!force && flowControl < Paxos::FlowControlMode::Normal)
 | |
|     return;
 | |
|   PaxosMsg *msg= (PaxosMsg *)ptr;
 | |
|   //assert(!msg->has_msgid());
 | |
|   msg->set_msgid(msgId.fetch_add(1));
 | |
|   if (lockless || msg->msgtype() != Paxos::AppendLog)
 | |
|     sendMsgFuncInternal(lockless, force, ptr, false);
 | |
|   else
 | |
|   {
 | |
|     PaxosMsg *arg= new PaxosMsg(*msg);
 | |
|     assert(msg->msgtype() == Paxos::AppendLog);
 | |
|     if (sendMsgQueue && sendMsgQueue->push(new SendMsgTask(arg, getSharedThis(), force)))
 | |
|     {
 | |
|       auto wqueue = std::weak_ptr<SingleProcessQueue<SendMsgTask>>(sendMsgQueue);
 | |
|       srv->sendAsyncEvent(&SingleProcessQueue<SendMsgTask>::mergeableProcessWeak, wqueue, RemoteServer::sendMsgFuncAsync);
 | |
|     }
 | |
|   }
 | |
| 
 | |
|   /*
 | |
|   if (paxos && paxos->getState() == Paxos::FOLLOWER)
 | |
|   {
 | |
|     paxos->updateFollowerMetaNo();
 | |
|     easy_warn_log("Server %d : updateFollowerMetaNo\n", paxos->getLocalServer()->serverId);
 | |
|   }
 | |
|   */
 | |
| }
 | |
| 
 | |
| void RemoteServer::sendMsgFuncAsync(SendMsgTask *task)
 | |
| {
 | |
|   task->server->sendMsgFuncInternal(false, task->force, (void *)task->msg, true);
 | |
| }
 | |
| 
 | |
| void RemoteServer::sendMsgFuncInternal(bool lockless, bool force, void *ptr, bool async)
 | |
| {
 | |
|   PaxosMsg *msg= (PaxosMsg *)ptr;
 | |
|   uint64_t logSize= 0;
 | |
|   bool lostConnectMode= false;
 | |
| 
 | |
|   if (isStop.load())
 | |
|     return;
 | |
|   /* Skip send msg this time, connect action will done before next send msg. */
 | |
|   if (addr.port == 0)
 | |
|   {
 | |
|     addr= srv->createConnection(strAddr, getSharedThis(), getConnTimeout(), serverId);
 | |
|     return;
 | |
|   }
 | |
| 
 | |
|   if (netError.load())
 | |
|     return;
 | |
| 
 | |
|   msg->set_clusterid(paxos->getClusterId());
 | |
| 
 | |
|   /* Fill the each server part. */
 | |
|   if (msg->msgtype() == Paxos::AppendLog)
 | |
|     msg->set_serverid(serverId);
 | |
|   else
 | |
|   {
 | |
|     if (paxos != NULL)
 | |
|       msg->set_serverid(paxos->getLocalServer()->serverId);
 | |
|     else /* For AppendLog and unit test */
 | |
|       msg->set_serverid(serverId);
 | |
|   }
 | |
|   if (isLearner)
 | |
|   {
 | |
|     msg->set_serverid(serverId);
 | |
|   }
 | |
| 
 | |
|   if (msg->msgtype() == Paxos::AppendLog)
 | |
|   {
 | |
|     bool islargeEntry= lastEntrySize >= paxos->getMaxPipeliningEntrySize();
 | |
|     auto localNextIndex= nextIndex.load();
 | |
|     auto lastLogIndex= paxos->getReplicateWithCacheLog() ? paxos->getLastCachedLogIndex() : paxos->getLastLogIndex();
 | |
|     bool isDelay= lastLogIndex > localNextIndex + paxos->getMaxDelayIndex();
 | |
|     uint64_t maxSendIndex= paxos->getMaxDelayIndex() / 10;
 | |
|     maxSendIndex= (maxSendIndex < 2) ? 2 : maxSendIndex;
 | |
|     /* Case when matchIndex is 0, this is not SendTooMuch case, at this time we're recalculate the matchIndex. */
 | |
|     bool isSendTooMuch= matchIndex.load() != 0 && ((localNextIndex - matchIndex.load()) >= maxSendIndex);
 | |
| 
 | |
|     if (!isLearner || paxos->getEnableLearnerPipelining())
 | |
|     {
 | |
|       if (!isLearner) // learner does not maintain epoch information
 | |
|       {
 | |
|       auto currentEpoch= paxos->getCurrentEpoch();
 | |
|       if (lostConnect.load())
 | |
|       {
 | |
|         lostConnectMode= true;
 | |
|       }
 | |
|       else if (currentEpoch != 0 && lastAckEpoch.load() < currentEpoch - 1)
 | |
|       {
 | |
|         easy_error_log("Detect lost connect to server %llu, currentEpoch:%llu, lastAckEpoch:%llu!", serverId, paxos->getCurrentEpoch(), lastAckEpoch.load());
 | |
|         lostConnect.store(true);
 | |
|         lostConnectMode= true;
 | |
|       }
 | |
| 
 | |
|       if (lostConnectMode && !force)
 | |
|       {
 | |
|         /* We only send "empty" heartbeat(force) msg to the lostConnectMode server! */
 | |
|         easy_warn_log("Try to send msg to server %ld, now this server is in lost connect mode, ignore.\n", serverId);
 | |
|         return;
 | |
|       }
 | |
|       }
 | |
| 
 | |
|       if (lostConnectMode && !disablePipelining)
 | |
|       {
 | |
|         easy_warn_log("Try to send msg to server %ld, server is in lost connect mode, disable pipelining.\n", serverId, lastLogIndex, localNextIndex);
 | |
|         disablePipelining= true;
 | |
|         guardId = msg->msgid() - 1;
 | |
|         if (matchIndex.load() != 0)
 | |
|           nextIndex.store(matchIndex.load() + 1);
 | |
|       }
 | |
| 
 | |
|       if (islargeEntry && !disablePipelining)
 | |
|       {
 | |
|         disablePipelining= true;
 | |
|         guardId = msg->msgid() - 1;
 | |
|         if (matchIndex.load() != 0)
 | |
|           nextIndex.store(matchIndex.load() + 1);
 | |
|         easy_warn_log("Try to send msg to server %ld, we are sending large LogEntry(lli:%llu, nextIndex:%llu), disable pipelining.\n", serverId, lastLogIndex, localNextIndex);
 | |
|       }
 | |
|       if (isDelay && !disablePipelining)
 | |
|       {
 | |
|         disablePipelining= true;
 | |
|         guardId = msg->msgid() - 1;
 | |
|         if (matchIndex.load() != 0)
 | |
|           nextIndex.store(matchIndex.load() + 1);
 | |
|         easy_warn_log("Try to send msg to server %ld, we are delay too much(lli:%llu, nextIndex:%llu), disable pipelining.\n", serverId, lastLogIndex, localNextIndex);
 | |
|       }
 | |
|       if (isSendTooMuch && !disablePipelining)
 | |
|       {
 | |
|         disablePipelining= true;
 | |
|         guardId = msg->msgid() - 1;
 | |
|         if (matchIndex.load() != 0)
 | |
|           nextIndex.store(matchIndex.load() + 1);
 | |
|         easy_warn_log("Try to send msg to server %ld, we have send too much this server, ignore this send and disable pipelining(matchIndex:%llu, nextIndex:%llu).\n", serverId, matchIndex.load(), localNextIndex);
 | |
|       }
 | |
| 
 | |
|       if ((!lostConnectMode && !islargeEntry && !isDelay && !isSendTooMuch && lastLogIndex < localNextIndex + paxos->getMinDelayIndex()) && disablePipelining)
 | |
|       {
 | |
|         disablePipelining= false;
 | |
|         easy_warn_log("Try to send msg to server %ld, enable pipelining(lli:%llu, nextIndex:%llu).\n", serverId, matchIndex.load(), localNextIndex);
 | |
|       }
 | |
|     }
 | |
| 
 | |
|     uint64_t timeout= paxos->getPipeliningTimeout() * 1000;
 | |
|     if (waitForReply == 1)
 | |
|     {
 | |
|       if (isLearner && !paxos->getEnableLearnerPipelining())
 | |
|       {
 | |
|         easy_info_log("Try to send msg to server %ld, now we are waiting for response, and this is learner skip.", serverId);
 | |
|         return;
 | |
|       }
 | |
|       if (lostConnectMode)
 | |
|       {
 | |
|         easy_warn_log("Try to send msg to server %ld, now this server is in lost connect mode, ignore.\n", serverId);
 | |
|         return;
 | |
|       }
 | |
|       uint64_t maxPacketSize= isDelay ? (paxos->getMaxPacketSize() * paxos->getLargeBatchRatio()) : paxos->getMaxPacketSize();
 | |
|       bool isTimeout= timeout != 0 && diffMS(lastSendTP) > timeout;
 | |
|       if (!isTimeout && !paxos->getLog()->getLeftSize(nextIndex, maxPacketSize))
 | |
|       {
 | |
|         easy_warn_log("Try to send msg to server %ld, now we are waiting for response, ignore.\n", serverId);
 | |
|         return;
 | |
|       }
 | |
|       if (disablePipelining)
 | |
|       {
 | |
|         easy_warn_log("Try to send msg to server %ld, now we are disable pipelining, ignore.\n", serverId);
 | |
|         return;
 | |
|       }
 | |
|       if (isTimeout)
 | |
|         easy_warn_log("Force to send msg to server %ld, because timeout.\n", serverId);
 | |
|       else
 | |
|         easy_warn_log("Force to send msg to server %ld, because the left log size is too large.\n", serverId);
 | |
| 
 | |
|       if (timeout != 0)
 | |
|         lastSendTP= now();
 | |
|     }
 | |
|     waitForReply= 1;
 | |
| 
 | |
|     Paxos::LogFillModeT mode= Paxos::NormalMode;
 | |
|     if (lostConnectMode)
 | |
|       mode= Paxos::EmptyMode;
 | |
|     else if ((isLearner && !paxos->getEnableLearnerPipelining()) || isDelay)
 | |
|       mode= Paxos::LargeBatchMode;
 | |
|     /* here we use this pointer is safe. */
 | |
|     if (flowControl >= Paxos::FlowControlMode::Slow)
 | |
|     {
 | |
|     if (async)
 | |
|       logSize= paxos->appendLogFillForEachAsync(msg, this, mode);
 | |
|     else if (!lockless)
 | |
|       logSize= paxos->appendLogFillForEach(msg, this, mode);
 | |
|     }
 | |
|     ++ (paxos->stats_.countMsgAppendLog);
 | |
| 
 | |
|     if (msg->entries_size() == 0)
 | |
|       ++ (paxos->stats_.countHeartbeat);
 | |
|     else
 | |
|       lastEntrySize = msg->entries().rbegin()->ByteSize();
 | |
|   }
 | |
|   else if (msg->msgtype() == Paxos::RequestVote)
 | |
|     ++ (paxos->stats_.countMsgRequestVote);
 | |
| 
 | |
|   /* If there are log left, we try to send the continue log entries. */
 | |
|   if (logSize >= paxos->getMaxPacketSize() && matchIndex.load() != 0 && !paxos->cdrMgr_.inRecovery)
 | |
|     paxos->appendLogToServerByPtr(getSharedThis(), true, false);
 | |
| 
 | |
|   //assert(msg->entries_size() <= 1);
 | |
|   int64_t lli = -1;
 | |
|   if (paxos) {
 | |
|     if (paxos->getReplicateWithCacheLog())
 | |
|       lli = paxos->getLastCachedLogIndex();
 | |
|     else
 | |
|       lli = paxos->getLastLogIndex();
 | |
|   }
 | |
|   easy_warn_log("Server %d : Send msg msgId(%llu) to server %ld, term:%ld, startLogIndex:%ld, entries_size:%d, log_size:%llu lli:%ld\n", paxos ? paxos->getLocalServer()->serverId : 0, msg->msgid(), serverId, msg->term(), msg->entries_size() >= 1 ? msg->entries().begin()->index() : -1, msg->entries_size(), logSize, lli);
 | |
| 
 | |
|   if (msg->entries_size() > 0)
 | |
|   {
 | |
|     assert(msg->prevlogindex() == msg->entries().begin()->index() - 1);
 | |
|     if (isLearner && !paxos->option.enableLearnerHeartbeat_)
 | |
|       heartbeatTimer->stop();
 | |
|   }
 | |
|   else if (msg->msgtype() == Paxos::AppendLog && !force)
 | |
|   {
 | |
|     easy_warn_log("Server %d : Skip send msg msgId(%llu) to server %ld because the entries_size is 0, and not force\n", paxos ? paxos->getLocalServer()->serverId : 0, msg->msgid(), serverId);
 | |
|     waitForReply= 0;
 | |
|     if (isLearner && !paxos->option.enableLearnerHeartbeat_)
 | |
|     {
 | |
|       easy_warn_log("Server %d : current server is learner but msg entries_size is 0, start heartbeat.", paxos ? paxos->getLocalServer()->serverId : 0);
 | |
|       heartbeatTimer->restart();
 | |
|     }
 | |
|     return;
 | |
|   }
 | |
| 
 | |
|   if (paxos->cdrMgr_.inRecovery)
 | |
|   {
 | |
|     /* reset term and lastlogindex */
 | |
|     msg->set_term(paxos->getTerm());
 | |
|     msg->set_lastlogindex(paxos->cdrMgr_.lastLogIndex);
 | |
|     /* in this case, prevLogTerm is not set */
 | |
|   }
 | |
| 
 | |
|   if (msg->msgtype() == Paxos::AppendLog)
 | |
|     msgCompress(msgCompressOption, *msg, logSize);
 | |
| 
 | |
|   std::string buf;
 | |
|   msg->SerializeToString(&buf);
 | |
| 
 | |
|   /*
 | |
|    * If we send RequestVote here, we don't reset the heartbeatTimer.
 | |
|    * Because we're candidate now, and resetTimer will enable heartbeatTimer !
 | |
|    * XXX here we only reset heartbeatTimer when msg type is AppendLog (not include LeaderCommand) !!
 | |
|    */
 | |
|   if (isLeader && msg->msgtype() == Paxos::AppendLog)
 | |
|     heartbeatTimer->restart();
 | |
|   srv->sendPacket(addr, buf, msg->msgid());
 | |
| }
 | |
| 
 | |
| void RemoteServer::onConnectCb()
 | |
| {
 | |
|   if (paxos && (paxos->getState() == Paxos::LEADER || isLearner))
 | |
|   {
 | |
|     /* XXX We reset nextIndex to matchIndex+1 on connected, because some resend msg may be lost in the disconnected period (libeasy's mechanism). */
 | |
|     uint64_t oldNextIndex= nextIndex;
 | |
|     resetNextIndex();
 | |
|     easy_warn_log("Server %d : update server %d 's nextIndex(old:%llu,new:%llu) when onConnect\n", paxos->getLocalServer()->serverId, serverId, oldNextIndex, nextIndex.load());
 | |
|     hasMatched= false;
 | |
| 
 | |
|     /* XXX we send append log to other servers only when we're leader, we also judge it in appendLogToServer */
 | |
|     if (isLearner)
 | |
|       paxos->appendLogToLearner(nullptr, true);
 | |
|   }
 | |
|     //Paxos::heartbeatCallback(this);
 | |
| }
 | |
| 
 | |
| void RemoteServer::resetNextIndex()
 | |
| {
 | |
|   paxos->resetNextIndexForServer(getSharedThis());
 | |
| }
 | |
| 
 | |
| void RemoteServer::fillInfo(void *ptr)
 | |
| {
 | |
|   Paxos::ClusterInfoType ci;
 | |
|   std::vector<Paxos::ClusterInfoType> *cis= static_cast<std::vector<Paxos::ClusterInfoType> *>(ptr);
 | |
| 
 | |
|   ci.serverId= serverId;
 | |
|   ci.ipPort= strAddr;
 | |
|   ci.matchIndex= matchIndex;
 | |
|   ci.nextIndex= nextIndex;
 | |
|   if (isLearner)
 | |
|     ci.role= Paxos::LEARNER;
 | |
|   else
 | |
|     ci.role= Paxos::FOLLOWER;
 | |
|   ci.hasVoted= hasVote;
 | |
|   ci.forceSync= forceSync;
 | |
|   ci.electionWeight= electionWeight;
 | |
|   if (!isLearner)
 | |
|     ci.learnerSource= 0;
 | |
|   else
 | |
|     ci.learnerSource=learnerSource;
 | |
|   ci.appliedIndex= getAppliedIndex();
 | |
|   if (isLearner && !paxos->getEnableLearnerPipelining())
 | |
|     ci.pipelining= false;
 | |
|   else
 | |
|     ci.pipelining= !disablePipelining;
 | |
|   ci.useApplied= sendByAppliedIndex;
 | |
| 
 | |
|   cis->push_back(std::move(ci));
 | |
| }
 | |
| 
 | |
| void RemoteServer::fillFollowerMeta(void *ptr)
 | |
| {
 | |
|   if (!isLearner || (learnerSource != paxos->getLocalServer()->serverId &&
 | |
|         (learnerSource == 0 || diffMS(lastMergeTP) > paxos->getMaxMergeReportTimeout() * 1000)))
 | |
|     return;
 | |
| 
 | |
|   ::google::protobuf::RepeatedPtrField< ::alisql::ClusterInfoEntry > *fms=
 | |
|     static_cast<::google::protobuf::RepeatedPtrField< ::alisql::ClusterInfoEntry > *>(ptr);
 | |
| 
 | |
|   ::alisql::ClusterInfoEntry *entry= fms->Add();
 | |
| 
 | |
|   entry->set_serverid(serverId);
 | |
|   entry->set_matchindex(matchIndex.load());
 | |
|   entry->set_nextindex(nextIndex.load());
 | |
|   entry->set_appliedindex(appliedIndex.load());
 | |
|   entry->set_learnersource(learnerSource); //for check in leader
 | |
| }
 | |
| 
 | |
| void RemoteServer::setMsgCompressOption(void *ptr)
 | |
| {
 | |
|   if (ptr == nullptr)
 | |
|     return;
 | |
|   msgCompressOption = *(MsgCompressOption *)ptr;
 | |
| }
 | |
| 
 | |
| } //namespace alisql
 |