syncReplication.c 11.5 KB
Newer Older
M
Minghao Li 已提交
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15
/*
 * Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
 *
 * This program is free software: you can use, redistribute, and/or modify
 * it under the terms of the GNU Affero General Public License, version 3
 * or later ("AGPL"), as published by the Free Software Foundation.
 *
 * This program is distributed in the hope that it will be useful, but WITHOUT
 * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
 * FITNESS FOR A PARTICULAR PURPOSE.
 *
 * You should have received a copy of the GNU Affero General Public License
 * along with this program. If not, see <http://www.gnu.org/licenses/>.
 */

M
Minghao Li 已提交
16
#include "syncReplication.h"
M
Minghao Li 已提交
17
#include "syncIndexMgr.h"
M
Minghao Li 已提交
18
#include "syncMessage.h"
M
Minghao Li 已提交
19
#include "syncRaftCfg.h"
M
Minghao Li 已提交
20
#include "syncRaftEntry.h"
M
Minghao Li 已提交
21
#include "syncRaftLog.h"
M
Minghao Li 已提交
22
#include "syncRaftStore.h"
23
#include "syncSnapshot.h"
M
Minghao Li 已提交
24
#include "syncUtil.h"
M
Minghao Li 已提交
25

M
Minghao Li 已提交
26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49
// TLA+ Spec
// AppendEntries(i, j) ==
//    /\ i /= j
//    /\ state[i] = Leader
//    /\ LET prevLogIndex == nextIndex[i][j] - 1
//           prevLogTerm == IF prevLogIndex > 0 THEN
//                              log[i][prevLogIndex].term
//                          ELSE
//                              0
//           \* Send up to 1 entry, constrained by the end of the log.
//           lastEntry == Min({Len(log[i]), nextIndex[i][j]})
//           entries == SubSeq(log[i], nextIndex[i][j], lastEntry)
//       IN Send([mtype          |-> AppendEntriesRequest,
//                mterm          |-> currentTerm[i],
//                mprevLogIndex  |-> prevLogIndex,
//                mprevLogTerm   |-> prevLogTerm,
//                mentries       |-> entries,
//                \* mlog is used as a history variable for the proof.
//                \* It would not exist in a real implementation.
//                mlog           |-> log[i],
//                mcommitIndex   |-> Min({commitIndex[i], lastEntry}),
//                msource        |-> i,
//                mdest          |-> j])
//    /\ UNCHANGED <<serverVars, candidateVars, leaderVars, logVars>>
M
Minghao Li 已提交
50
//
M
Minghao Li 已提交
51
int32_t syncNodeAppendEntriesPeers(SSyncNode* pSyncNode) {
M
Minghao Li 已提交
52
  ASSERT(pSyncNode->state == TAOS_SYNC_STATE_LEADER);
M
Minghao Li 已提交
53

M
Minghao Li 已提交
54 55 56 57
  syncIndexMgrLog2("==syncNodeAppendEntriesPeers== pNextIndex", pSyncNode->pNextIndex);
  syncIndexMgrLog2("==syncNodeAppendEntriesPeers== pMatchIndex", pSyncNode->pMatchIndex);
  logStoreSimpleLog2("==syncNodeAppendEntriesPeers==", pSyncNode->pLogStore);

M
Minghao Li 已提交
58
  int32_t ret = 0;
M
Minghao Li 已提交
59
  for (int i = 0; i < pSyncNode->peersNum; ++i) {
M
Minghao Li 已提交
60
    SRaftId* pDestId = &(pSyncNode->peersId[i]);
M
Minghao Li 已提交
61

M
Minghao Li 已提交
62 63
    // set prevLogIndex
    SyncIndex nextIndex = syncIndexMgrGetIndex(pSyncNode->pNextIndex, pDestId);
M
Minghao Li 已提交
64

M
Minghao Li 已提交
65
    SyncIndex preLogIndex = nextIndex - 1;
M
Minghao Li 已提交
66

M
Minghao Li 已提交
67
    // set preLogTerm
M
Minghao Li 已提交
68 69
    SyncTerm preLogTerm = 0;
    if (preLogIndex >= SYNC_INDEX_BEGIN) {
M
Minghao Li 已提交
70
      SSyncRaftEntry* pPreEntry = pSyncNode->pLogStore->getEntry(pSyncNode->pLogStore, preLogIndex);
M
Minghao Li 已提交
71
      ASSERT(pPreEntry != NULL);
M
Minghao Li 已提交
72

M
Minghao Li 已提交
73
      preLogTerm = pPreEntry->term;
M
Minghao Li 已提交
74
      syncEntryDestory(pPreEntry);
M
Minghao Li 已提交
75
    }
M
Minghao Li 已提交
76

M
Minghao Li 已提交
77 78 79 80
    // batch optimized
    // SyncIndex lastIndex = syncUtilMinIndex(pSyncNode->pLogStore->getLastIndex(pSyncNode->pLogStore), nextIndex);

    SyncAppendEntries* pMsg = NULL;
81
    SSyncRaftEntry*    pEntry = pSyncNode->pLogStore->getEntry(pSyncNode->pLogStore, nextIndex);
M
Minghao Li 已提交
82
    if (pEntry != NULL) {
M
Minghao Li 已提交
83
      pMsg = syncAppendEntriesBuild(pEntry->bytes, pSyncNode->vgId);
M
Minghao Li 已提交
84
      ASSERT(pMsg != NULL);
M
Minghao Li 已提交
85

M
Minghao Li 已提交
86 87 88
      // add pEntry into msg
      uint32_t len;
      char*    serialized = syncEntrySerialize(pEntry, &len);
M
Minghao Li 已提交
89
      ASSERT(len == pEntry->bytes);
M
Minghao Li 已提交
90 91
      memcpy(pMsg->data, serialized, len);

wafwerar's avatar
wafwerar 已提交
92
      taosMemoryFree(serialized);
M
Minghao Li 已提交
93 94 95 96
      syncEntryDestory(pEntry);

    } else {
      // maybe overflow, send empty record
M
Minghao Li 已提交
97
      pMsg = syncAppendEntriesBuild(0, pSyncNode->vgId);
M
Minghao Li 已提交
98
      ASSERT(pMsg != NULL);
M
Minghao Li 已提交
99
    }
M
Minghao Li 已提交
100

M
Minghao Li 已提交
101
    ASSERT(pMsg != NULL);
M
Minghao Li 已提交
102 103
    pMsg->srcId = pSyncNode->myRaftId;
    pMsg->destId = *pDestId;
M
Minghao Li 已提交
104
    pMsg->term = pSyncNode->pRaftStore->currentTerm;
M
Minghao Li 已提交
105 106 107 108
    pMsg->prevLogIndex = preLogIndex;
    pMsg->prevLogTerm = preLogTerm;
    pMsg->commitIndex = pSyncNode->commitIndex;

M
Minghao Li 已提交
109 110
    syncAppendEntriesLog2("==syncNodeAppendEntriesPeers==", pMsg);

M
Minghao Li 已提交
111
    // send AppendEntries
M
Minghao Li 已提交
112
    syncNodeAppendEntries(pSyncNode, pDestId, pMsg);
M
Minghao Li 已提交
113
    syncAppendEntriesDestroy(pMsg);
M
Minghao Li 已提交
114 115
  }

M
Minghao Li 已提交
116 117 118
  return ret;
}

119 120 121 122 123 124 125 126 127 128 129 130 131
int32_t syncNodeAppendEntriesPeersSnapshot2(SSyncNode* pSyncNode) {
  if (pSyncNode->state != TAOS_SYNC_STATE_LEADER) {
    return -1;
  }

  int32_t ret = 0;
  for (int i = 0; i < pSyncNode->peersNum; ++i) {
    SRaftId* pDestId = &(pSyncNode->peersId[i]);

    // next index
    SyncIndex nextIndex = syncIndexMgrGetIndex(pSyncNode->pNextIndex, pDestId);

    // pre index, pre term
132 133
    SyncIndex preLogIndex = syncNodeGetPreIndex(pSyncNode, nextIndex);
    SyncTerm  preLogTerm = syncNodeGetPreTerm(pSyncNode, nextIndex);
134 135 136 137 138 139 140 141 142 143
    if (preLogTerm == SYNC_TERM_INVALID) {
      SyncIndex newNextIndex = syncNodeGetLastIndex(pSyncNode) + 1;
      syncIndexMgrSetIndex(pSyncNode->pNextIndex, pDestId, newNextIndex);
      syncIndexMgrSetIndex(pSyncNode->pMatchIndex, pDestId, SYNC_INDEX_INVALID);
      sError("vgId:%d sync get pre term error, nextIndex:%ld, update next-index:%ld, match-index:%d, raftid:%ld",
             pSyncNode->vgId, nextIndex, newNextIndex, SYNC_INDEX_INVALID, pDestId->addr);

      return -1;
    }

M
Minghao Li 已提交
144
    // entry pointer array
145 146
    SSyncRaftEntry* entryPArr[SYNC_MAX_BATCH_SIZE];
    memset(entryPArr, 0, sizeof(entryPArr));
147

M
Minghao Li 已提交
148
    // get entry batch
149 150
    int32_t   getCount = 0;
    SyncIndex getEntryIndex = nextIndex;
151
    for (int32_t i = 0; i < pSyncNode->pRaftCfg->batchSize; ++i) {
M
Minghao Li 已提交
152
      SSyncRaftEntry* pEntry = NULL;
153
      int32_t         code = pSyncNode->pLogStore->syncLogGetEntry(pSyncNode->pLogStore, getEntryIndex, &pEntry);
154 155
      if (code == 0) {
        ASSERT(pEntry != NULL);
156
        entryPArr[i] = pEntry;
157
        getCount++;
M
Minghao Li 已提交
158 159
        getEntryIndex++;

160 161 162 163 164
      } else {
        break;
      }
    }

165 166
    // event log
    do {
M
Minghao Li 已提交
167 168 169 170 171
      char     logBuf[128];
      char     host[64];
      uint16_t port;
      syncUtilU642Addr(pDestId->addr, host, sizeof(host), &port);
      snprintf(logBuf, sizeof(logBuf), "build batch:%d for %s:%d", getCount, host, port);
172 173 174
      syncNodeEventLog(pSyncNode, logBuf);
    } while (0);

M
Minghao Li 已提交
175
    // build msg
176
    SyncAppendEntriesBatch* pMsg = syncAppendEntriesBatchBuild(entryPArr, getCount, pSyncNode->vgId);
177 178
    ASSERT(pMsg != NULL);

M
Minghao Li 已提交
179
    // free entries
180
    for (int32_t i = 0; i < pSyncNode->pRaftCfg->batchSize; ++i) {
181 182 183 184 185 186 187
      SSyncRaftEntry* pEntry = entryPArr[i];
      if (pEntry != NULL) {
        syncEntryDestory(pEntry);
        entryPArr[i] = NULL;
      }
    }

188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205
    // prepare msg
    pMsg->srcId = pSyncNode->myRaftId;
    pMsg->destId = *pDestId;
    pMsg->term = pSyncNode->pRaftStore->currentTerm;
    pMsg->prevLogIndex = preLogIndex;
    pMsg->prevLogTerm = preLogTerm;
    pMsg->commitIndex = pSyncNode->commitIndex;
    pMsg->privateTerm = 0;
    pMsg->dataCount = getCount;

    // send msg
    syncNodeAppendEntriesBatch(pSyncNode, pDestId, pMsg);
    syncAppendEntriesBatchDestroy(pMsg);
  }

  return 0;
}

206
int32_t syncNodeAppendEntriesPeersSnapshot(SSyncNode* pSyncNode) {
207
  ASSERT(pSyncNode->state == TAOS_SYNC_STATE_LEADER);
208

209 210 211
  syncIndexMgrLog2("begin append entries peers pNextIndex:", pSyncNode->pNextIndex);
  syncIndexMgrLog2("begin append entries peers pMatchIndex:", pSyncNode->pMatchIndex);
  logStoreSimpleLog2("begin append entries peers LogStore:", pSyncNode->pLogStore);
212 213 214 215 216

  int32_t ret = 0;
  for (int i = 0; i < pSyncNode->peersNum; ++i) {
    SRaftId* pDestId = &(pSyncNode->peersId[i]);

217
    // next index
218
    SyncIndex nextIndex = syncIndexMgrGetIndex(pSyncNode->pNextIndex, pDestId);
219 220 221 222

    // pre index, pre term
    SyncIndex preLogIndex = syncNodeGetPreIndex(pSyncNode, nextIndex);
    SyncTerm  preLogTerm = syncNodeGetPreTerm(pSyncNode, nextIndex);
223 224 225 226 227 228 229 230 231
    if (preLogTerm == SYNC_TERM_INVALID) {
      SyncIndex newNextIndex = syncNodeGetLastIndex(pSyncNode) + 1;
      syncIndexMgrSetIndex(pSyncNode->pNextIndex, pDestId, newNextIndex);
      syncIndexMgrSetIndex(pSyncNode->pMatchIndex, pDestId, SYNC_INDEX_INVALID);
      sError("vgId:%d sync get pre term error, nextIndex:%ld, update next-index:%ld, match-index:%d, raftid:%ld",
             pSyncNode->vgId, nextIndex, newNextIndex, SYNC_INDEX_INVALID, pDestId->addr);

      return -1;
    }
232

233 234
    // prepare entry
    SyncAppendEntries* pMsg = NULL;
235

236 237
    SSyncRaftEntry* pEntry;
    int32_t         code = pSyncNode->pLogStore->syncLogGetEntry(pSyncNode->pLogStore, nextIndex, &pEntry);
M
Minghao Li 已提交
238

239 240 241
    if (code == 0) {
      ASSERT(pEntry != NULL);

242 243
      pMsg = syncAppendEntriesBuild(pEntry->bytes, pSyncNode->vgId);
      ASSERT(pMsg != NULL);
244

245 246 247
      // add pEntry into msg
      uint32_t len;
      char*    serialized = syncEntrySerialize(pEntry, &len);
M
Minghao Li 已提交
248
      ASSERT(len == pEntry->bytes);
249
      memcpy(pMsg->data, serialized, len);
M
Minghao Li 已提交
250

251 252
      taosMemoryFree(serialized);
      syncEntryDestory(pEntry);
253 254

    } else {
255 256 257 258 259 260 261 262 263
      if (terrno == TSDB_CODE_WAL_LOG_NOT_EXIST) {
        // no entry in log
        pMsg = syncAppendEntriesBuild(0, pSyncNode->vgId);
        ASSERT(pMsg != NULL);

      } else {
        syncNodeLog3("", pSyncNode);
        ASSERT(0);
      }
264
    }
265 266 267 268 269 270 271 272 273

    // prepare msg
    ASSERT(pMsg != NULL);
    pMsg->srcId = pSyncNode->myRaftId;
    pMsg->destId = *pDestId;
    pMsg->term = pSyncNode->pRaftStore->currentTerm;
    pMsg->prevLogIndex = preLogIndex;
    pMsg->prevLogTerm = preLogTerm;
    pMsg->commitIndex = pSyncNode->commitIndex;
M
Minghao Li 已提交
274 275
    pMsg->privateTerm = 0;
    // pMsg->privateTerm = syncIndexMgrGetTerm(pSyncNode->pNextIndex, pDestId);
276 277 278 279

    // send msg
    syncNodeAppendEntries(pSyncNode, pDestId, pMsg);
    syncAppendEntriesDestroy(pMsg);
280 281 282 283
  }

  return ret;
}
M
Minghao Li 已提交
284

M
Minghao Li 已提交
285 286
int32_t syncNodeReplicate(SSyncNode* pSyncNode) {
  // start replicate
M
Minghao Li 已提交
287 288
  int32_t ret = 0;

M
Minghao Li 已提交
289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304
  switch (pSyncNode->pRaftCfg->snapshotStrategy) {
    case SYNC_STRATEGY_NO_SNAPSHOT:
      ret = syncNodeAppendEntriesPeers(pSyncNode);
      break;

    case SYNC_STRATEGY_STANDARD_SNAPSHOT:
      ret = syncNodeAppendEntriesPeersSnapshot(pSyncNode);
      break;

    case SYNC_STRATEGY_WAL_FIRST:
      ret = syncNodeAppendEntriesPeersSnapshot2(pSyncNode);
      break;

    default:
      ret = syncNodeAppendEntriesPeers(pSyncNode);
      break;
M
Minghao Li 已提交
305
  }
M
Minghao Li 已提交
306

M
Minghao Li 已提交
307 308
  return ret;
}
M
Minghao Li 已提交
309 310 311 312

int32_t syncNodeAppendEntries(SSyncNode* pSyncNode, const SRaftId* destRaftId, const SyncAppendEntries* pMsg) {
  int32_t ret = 0;

313 314 315 316
  do {
    char     host[128];
    uint16_t port;
    syncUtilU642Addr(destRaftId->addr, host, sizeof(host), &port);
M
Minghao Li 已提交
317
    sDebug(
M
Minghao Li 已提交
318 319
        "vgId:%d, send sync-append-entries to %s:%d, {term:%lu, pre-index:%ld, pre-term:%lu, pterm:%lu, commit:%ld, "
        "datalen:%d}",
M
Minghao Li 已提交
320 321
        pSyncNode->vgId, host, port, pMsg->term, pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->privateTerm,
        pMsg->commitIndex, pMsg->dataLen);
322 323
  } while (0);

M
Minghao Li 已提交
324 325 326 327
  SRpcMsg rpcMsg;
  syncAppendEntries2RpcMsg(pMsg, &rpcMsg);
  syncNodeSendMsgById(destRaftId, pSyncNode, &rpcMsg);
  return ret;
328 329 330 331 332 333 334 335 336 337 338
}

int32_t syncNodeAppendEntriesBatch(SSyncNode* pSyncNode, const SRaftId* destRaftId,
                                   const SyncAppendEntriesBatch* pMsg) {
  do {
    char     host[128];
    uint16_t port;
    syncUtilU642Addr(destRaftId->addr, host, sizeof(host), &port);
    sDebug(
        "vgId:%d, send sync-append-entries-batch to %s:%d, {term:%lu, pre-index:%ld, pre-term:%lu, pterm:%lu, "
        "commit:%ld, "
339
        "datalen:%d, datacount:%d}",
340 341 342 343 344 345 346 347
        pSyncNode->vgId, host, port, pMsg->term, pMsg->prevLogIndex, pMsg->prevLogTerm, pMsg->privateTerm,
        pMsg->commitIndex, pMsg->dataLen, pMsg->dataCount);
  } while (0);

  SRpcMsg rpcMsg;
  syncAppendEntriesBatch2RpcMsg(pMsg, &rpcMsg);
  syncNodeSendMsgById(destRaftId, pSyncNode, &rpcMsg);
  return 0;
M
Minghao Li 已提交
348
}