github.com/unigraph-dev/dgraph@v1.1.1-0.20200923154953-8b52b426f765/worker/draft.go (about) 1 /* 2 * Copyright 2016-2018 Dgraph Labs, Inc. and Contributors 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package worker 18 19 import ( 20 "bytes" 21 "encoding/hex" 22 "fmt" 23 "sort" 24 "sync" 25 "sync/atomic" 26 "time" 27 28 humanize "github.com/dustin/go-humanize" 29 "go.etcd.io/etcd/raft" 30 "go.etcd.io/etcd/raft/raftpb" 31 32 ostats "go.opencensus.io/stats" 33 "go.opencensus.io/tag" 34 otrace "go.opencensus.io/trace" 35 36 "github.com/dgraph-io/badger" 37 bpb "github.com/dgraph-io/badger/pb" 38 "github.com/dgraph-io/badger/y" 39 dy "github.com/dgraph-io/dgo/y" 40 "github.com/dgraph-io/dgraph/conn" 41 "github.com/dgraph-io/dgraph/posting" 42 "github.com/dgraph-io/dgraph/protos/pb" 43 "github.com/dgraph-io/dgraph/raftwal" 44 "github.com/dgraph-io/dgraph/schema" 45 "github.com/dgraph-io/dgraph/types" 46 "github.com/dgraph-io/dgraph/x" 47 "github.com/pkg/errors" 48 49 "github.com/golang/glog" 50 "golang.org/x/net/context" 51 "golang.org/x/net/trace" 52 ) 53 54 type node struct { 55 *conn.Node 56 57 // Fields which are never changed after init. 58 applyCh chan []*pb.Proposal 59 rollupCh chan uint64 // Channel to run posting list rollups. 60 ctx context.Context 61 gid uint32 62 closer *y.Closer 63 64 streaming int32 // Used to avoid calculating snapshot 65 66 canCampaign bool 67 elog trace.EventLog 68 69 pendingSize int64 70 } 71 72 // Now that we apply txn updates via Raft, waiting based on Txn timestamps is 73 // sufficient. We don't need to wait for proposals to be applied. 74 75 func newNode(store *raftwal.DiskStorage, gid uint32, id uint64, myAddr string) *node { 76 glog.Infof("Node ID: %#x with GroupID: %d\n", id, gid) 77 78 rc := &pb.RaftContext{ 79 Addr: myAddr, 80 Group: gid, 81 Id: id, 82 } 83 m := conn.NewNode(rc, store) 84 85 n := &node{ 86 Node: m, 87 ctx: context.Background(), 88 gid: gid, 89 // We need a generous size for applyCh, because raft.Tick happens every 90 // 10ms. If we restrict the size here, then Raft goes into a loop trying 91 // to maintain quorum health. 92 applyCh: make(chan []*pb.Proposal, 1000), 93 rollupCh: make(chan uint64, 3), 94 elog: trace.NewEventLog("Dgraph", "ApplyCh"), 95 closer: y.NewCloser(3), // Matches CLOSER:1 96 } 97 return n 98 } 99 100 func (n *node) Ctx(key string) context.Context { 101 if pctx := n.Proposals.Get(key); pctx != nil { 102 return pctx.Ctx 103 } 104 return context.Background() 105 } 106 107 func (n *node) applyConfChange(e raftpb.Entry) { 108 var cc raftpb.ConfChange 109 if err := cc.Unmarshal(e.Data); err != nil { 110 glog.Errorf("While unmarshalling confchange: %+v", err) 111 } 112 113 if cc.Type == raftpb.ConfChangeRemoveNode { 114 n.DeletePeer(cc.NodeID) 115 } else if len(cc.Context) > 0 { 116 var rc pb.RaftContext 117 x.Check(rc.Unmarshal(cc.Context)) 118 n.Connect(rc.Id, rc.Addr) 119 } 120 121 cs := n.Raft().ApplyConfChange(cc) 122 n.SetConfState(cs) 123 n.DoneConfChange(cc.ID, nil) 124 } 125 126 var errHasPendingTxns = errors.New("Pending transactions found. Please retry operation") 127 128 // We must not wait here. Previously, we used to block until we have aborted the 129 // transactions. We're now applying all updates serially, so blocking for one 130 // operation is not an option. 131 func detectPendingTxns(attr string) error { 132 tctxs := posting.Oracle().IterateTxns(func(key []byte) bool { 133 pk, err := x.Parse(key) 134 if err != nil { 135 return false 136 } 137 return pk.Attr == attr 138 }) 139 if len(tctxs) == 0 { 140 return nil 141 } 142 go tryAbortTransactions(tctxs) 143 return errHasPendingTxns 144 } 145 146 // We don't support schema mutations across nodes in a transaction. 147 // Wait for all transactions to either abort or complete and all write transactions 148 // involving the predicate are aborted until schema mutations are done. 149 func (n *node) applyMutations(ctx context.Context, proposal *pb.Proposal) (rerr error) { 150 span := otrace.FromContext(ctx) 151 152 if proposal.Mutations.DropOp == pb.Mutations_DATA { 153 // Ensures nothing get written to disk due to commit proposals. 154 posting.Oracle().ResetTxns() 155 return posting.DeleteData() 156 } 157 158 if proposal.Mutations.DropOp == pb.Mutations_ALL { 159 // Ensures nothing get written to disk due to commit proposals. 160 posting.Oracle().ResetTxns() 161 schema.State().DeleteAll() 162 163 if err := posting.DeleteAll(); err != nil { 164 return err 165 } 166 167 if groups().groupId() == 1 { 168 initialSchema := schema.InitialSchema() 169 for _, s := range initialSchema { 170 if err := updateSchema(s); err != nil { 171 return err 172 } 173 174 if servesTablet, err := groups().ServesTablet(s.Predicate); err != nil { 175 return err 176 } else if !servesTablet { 177 return errors.Errorf("group 1 should always serve reserved predicate %s", 178 s.Predicate) 179 } 180 } 181 } 182 183 return nil 184 } 185 186 if proposal.Mutations.DropOp == pb.Mutations_TYPE { 187 return schema.State().DeleteType(proposal.Mutations.DropValue) 188 } 189 190 if proposal.Mutations.StartTs == 0 { 191 return errors.New("StartTs must be provided") 192 } 193 startTs := proposal.Mutations.StartTs 194 195 if len(proposal.Mutations.Schema) > 0 || len(proposal.Mutations.Types) > 0 { 196 span.Annotatef(nil, "Applying schema and types") 197 for _, supdate := range proposal.Mutations.Schema { 198 // We should not need to check for predicate move here. 199 if err := detectPendingTxns(supdate.Predicate); err != nil { 200 return err 201 } 202 if err := runSchemaMutation(ctx, supdate, startTs); err != nil { 203 return err 204 } 205 } 206 207 for _, tupdate := range proposal.Mutations.Types { 208 if err := runTypeMutation(ctx, tupdate); err != nil { 209 return err 210 } 211 } 212 213 return nil 214 } 215 216 // Scheduler tracks tasks at subject, predicate level, so doing 217 // schema stuff here simplies the design and we needn't worry about 218 // serializing the mutations per predicate or schema mutations 219 // We derive the schema here if it's not present 220 // Since raft committed logs are serialized, we can derive 221 // schema here without any locking 222 223 // stores a map of predicate and type of first mutation for each predicate 224 schemaMap := make(map[string]types.TypeID) 225 for _, edge := range proposal.Mutations.Edges { 226 if edge.Entity == 0 && bytes.Equal(edge.Value, []byte(x.Star)) { 227 // We should only drop the predicate if there is no pending 228 // transaction. 229 if err := detectPendingTxns(edge.Attr); err != nil { 230 span.Annotatef(nil, "Found pending transactions. Retry later.") 231 return err 232 } 233 span.Annotatef(nil, "Deleting predicate: %s", edge.Attr) 234 return posting.DeletePredicate(ctx, edge.Attr) 235 } 236 // Dont derive schema when doing deletion. 237 if edge.Op == pb.DirectedEdge_DEL { 238 continue 239 } 240 if _, ok := schemaMap[edge.Attr]; !ok { 241 schemaMap[edge.Attr] = posting.TypeID(edge) 242 } 243 } 244 245 total := len(proposal.Mutations.Edges) 246 247 // TODO: Active mutations values can go up or down but with 248 // OpenCensus stats bucket boundaries start from 0, hence 249 // recording negative and positive values skews up values. 250 ostats.Record(ctx, x.ActiveMutations.M(int64(total))) 251 defer func() { 252 ostats.Record(ctx, x.ActiveMutations.M(int64(-total))) 253 }() 254 255 for attr, storageType := range schemaMap { 256 if _, err := schema.State().TypeOf(attr); err != nil { 257 createSchema(attr, storageType) 258 } 259 } 260 261 m := proposal.Mutations 262 txn := posting.Oracle().RegisterStartTs(m.StartTs) 263 if txn.ShouldAbort() { 264 span.Annotatef(nil, "Txn %d should abort.", m.StartTs) 265 return dy.ErrConflict 266 } 267 268 // Discard the posting lists from cache to release memory at the end. 269 defer txn.Update() 270 271 sort.Slice(m.Edges, func(i, j int) bool { 272 ei := m.Edges[i] 273 ej := m.Edges[j] 274 if ei.GetAttr() != ej.GetAttr() { 275 return ei.GetAttr() < ej.GetAttr() 276 } 277 return ei.GetEntity() < ej.GetEntity() 278 }) 279 280 process := func(edges []*pb.DirectedEdge) error { 281 var retries int 282 for _, edge := range edges { 283 for { 284 err := runMutation(ctx, edge, txn) 285 if err == nil { 286 break 287 } 288 if err != posting.ErrRetry { 289 return err 290 } 291 retries++ 292 } 293 } 294 if retries > 0 { 295 span.Annotatef(nil, "retries=true num=%d", retries) 296 } 297 return nil 298 } 299 numGo, width := x.DivideAndRule(len(m.Edges)) 300 span.Annotatef(nil, "To apply: %d edges. NumGo: %d. Width: %d", len(m.Edges), numGo, width) 301 302 if numGo == 1 { 303 return process(m.Edges) 304 } 305 errCh := make(chan error, numGo) 306 for i := 0; i < numGo; i++ { 307 start := i * width 308 end := start + width 309 if end > len(m.Edges) { 310 end = len(m.Edges) 311 } 312 go func(start, end int) { 313 errCh <- process(m.Edges[start:end]) 314 }(start, end) 315 } 316 for i := 0; i < numGo; i++ { 317 if err := <-errCh; err != nil { 318 return err 319 } 320 } 321 return nil 322 } 323 324 func (n *node) applyCommitted(proposal *pb.Proposal) error { 325 ctx := n.Ctx(proposal.Key) 326 span := otrace.FromContext(ctx) 327 span.Annotatef(nil, "node.applyCommitted Node id: %d. Group id: %d. Got proposal key: %s", 328 n.Id, n.gid, proposal.Key) 329 330 if proposal.Mutations != nil { 331 // syncmarks for this shouldn't be marked done until it's committed. 332 span.Annotate(nil, "Applying mutations") 333 if err := n.applyMutations(ctx, proposal); err != nil { 334 span.Annotatef(nil, "While applying mutations: %v", err) 335 return err 336 } 337 span.Annotate(nil, "Done") 338 return nil 339 } 340 341 switch { 342 case len(proposal.Kv) > 0: 343 return populateKeyValues(ctx, proposal.Kv) 344 345 case proposal.State != nil: 346 n.elog.Printf("Applying state for key: %s", proposal.Key) 347 // This state needn't be snapshotted in this group, on restart we would fetch 348 // a state which is latest or equal to this. 349 groups().applyState(proposal.State) 350 return nil 351 352 case len(proposal.CleanPredicate) > 0: 353 n.elog.Printf("Cleaning predicate: %s", proposal.CleanPredicate) 354 return posting.DeletePredicate(ctx, proposal.CleanPredicate) 355 356 case proposal.Delta != nil: 357 n.elog.Printf("Applying Oracle Delta for key: %s", proposal.Key) 358 return n.commitOrAbort(proposal.Key, proposal.Delta) 359 360 case proposal.Snapshot != nil: 361 existing, err := n.Store.Snapshot() 362 if err != nil { 363 return err 364 } 365 snap := proposal.Snapshot 366 if existing.Metadata.Index >= snap.Index { 367 log := fmt.Sprintf("Skipping snapshot at %d, because found one at %d", 368 snap.Index, existing.Metadata.Index) 369 n.elog.Printf(log) 370 glog.Info(log) 371 return nil 372 } 373 n.elog.Printf("Creating snapshot: %+v", snap) 374 glog.Infof("Creating snapshot at index: %d. ReadTs: %d.\n", snap.Index, snap.ReadTs) 375 376 data, err := snap.Marshal() 377 x.Check(err) 378 for { 379 // We should never let CreateSnapshot have an error. 380 err := n.Store.CreateSnapshot(snap.Index, n.ConfState(), data) 381 if err == nil { 382 break 383 } 384 glog.Warningf("Error while calling CreateSnapshot: %v. Retrying...", err) 385 } 386 // Roll up all posting lists as a best-effort operation. 387 n.rollupCh <- snap.ReadTs 388 return nil 389 } 390 x.Fatalf("Unknown proposal: %+v", proposal) 391 return nil 392 } 393 394 func (n *node) processRollups() { 395 defer n.closer.Done() // CLOSER:1 396 tick := time.NewTicker(5 * time.Minute) // Rolling up once every 5 minutes seems alright. 397 defer tick.Stop() 398 399 var readTs, last uint64 400 for { 401 select { 402 case <-n.closer.HasBeenClosed(): 403 return 404 case readTs = <-n.rollupCh: 405 case <-tick.C: 406 if readTs <= last { 407 break // Break out of the select case. 408 } 409 if err := n.rollupLists(readTs); err != nil { 410 // If we encounter error here, we don't need to do anything about 411 // it. Just let the user know. 412 glog.Errorf("Error while rolling up lists at %d: %v\n", readTs, err) 413 } else { 414 last = readTs // Update last only if we succeeded. 415 glog.Infof("List rollup at Ts %d: OK.\n", readTs) 416 } 417 } 418 } 419 } 420 421 func (n *node) processApplyCh() { 422 defer n.closer.Done() // CLOSER:1 423 424 type P struct { 425 err error 426 size int 427 seen time.Time 428 } 429 previous := make(map[string]*P) 430 431 // This function must be run serially. 432 handle := func(proposals []*pb.Proposal) { 433 var totalSize int64 434 for _, proposal := range proposals { 435 // We use the size as a double check to ensure that we're 436 // working with the same proposal as before. 437 psz := proposal.Size() 438 totalSize += int64(psz) 439 440 var perr error 441 p, ok := previous[proposal.Key] 442 if ok && p.err == nil && p.size == psz { 443 n.elog.Printf("Proposal with key: %s already applied. Skipping index: %d.\n", 444 proposal.Key, proposal.Index) 445 previous[proposal.Key].seen = time.Now() // Update the ts. 446 // Don't break here. We still need to call the Done below. 447 448 } else { 449 start := time.Now() 450 perr = n.applyCommitted(proposal) 451 if len(proposal.Key) > 0 { 452 p := &P{err: perr, size: psz, seen: time.Now()} 453 previous[proposal.Key] = p 454 } 455 if perr != nil { 456 glog.Errorf("Applying proposal. Error: %v. Proposal: %q.", perr, proposal) 457 } 458 n.elog.Printf("Applied proposal with key: %s, index: %d. Err: %v", 459 proposal.Key, proposal.Index, perr) 460 461 var tags []tag.Mutator 462 switch { 463 case proposal.Mutations != nil: 464 tags = append(tags, tag.Upsert(x.KeyMethod, "apply.Mutations")) 465 case proposal.Delta != nil: 466 tags = append(tags, tag.Upsert(x.KeyMethod, "apply.Delta")) 467 } 468 ms := x.SinceMs(start) 469 _ = ostats.RecordWithTags(context.Background(), tags, x.LatencyMs.M(ms)) 470 } 471 472 n.Proposals.Done(proposal.Key, perr) 473 n.Applied.Done(proposal.Index) 474 ostats.Record(context.Background(), x.RaftAppliedIndex.M(int64(n.Applied.DoneUntil()))) 475 } 476 if sz := atomic.AddInt64(&n.pendingSize, -totalSize); sz < 0 { 477 glog.Warningf("Pending size should remain above zero: %d", sz) 478 } 479 } 480 481 maxAge := 10 * time.Minute 482 tick := time.NewTicker(maxAge / 2) 483 defer tick.Stop() 484 485 for { 486 select { 487 case entries, ok := <-n.applyCh: 488 if !ok { 489 return 490 } 491 handle(entries) 492 case <-tick.C: 493 // We use this ticker to clear out previous map. 494 now := time.Now() 495 for key, p := range previous { 496 if now.Sub(p.seen) > maxAge { 497 delete(previous, key) 498 } 499 } 500 n.elog.Printf("Size of previous map: %d", len(previous)) 501 } 502 } 503 } 504 505 func (n *node) commitOrAbort(pkey string, delta *pb.OracleDelta) error { 506 // First let's commit all mutations to disk. 507 writer := posting.NewTxnWriter(pstore) 508 toDisk := func(start, commit uint64) { 509 txn := posting.Oracle().GetTxn(start) 510 if txn == nil { 511 return 512 } 513 txn.Update() 514 err := x.RetryUntilSuccess(x.WorkerConfig.MaxRetries, 10*time.Millisecond, func() error { 515 return txn.CommitToDisk(writer, commit) 516 }) 517 518 if err != nil { 519 glog.Errorf("Error while applying txn status to disk (%d -> %d): %v", 520 start, commit, err) 521 } 522 } 523 524 for _, status := range delta.Txns { 525 toDisk(status.StartTs, status.CommitTs) 526 } 527 if err := writer.Flush(); err != nil { 528 return errors.Wrapf(err, "while flushing to disk") 529 } 530 531 g := groups() 532 atomic.StoreUint64(&g.deltaChecksum, delta.GroupChecksums[g.groupId()]) 533 534 // Now advance Oracle(), so we can service waiting reads. 535 posting.Oracle().ProcessDelta(delta) 536 return nil 537 } 538 539 func (n *node) leaderBlocking() (*conn.Pool, error) { 540 pool := groups().Leader(groups().groupId()) 541 if pool == nil { 542 // Functions like retrieveSnapshot and joinPeers are blocking at initial start and 543 // leader election for a group might not have happened when it is called. If we can't 544 // find a leader, get latest state from Zero. 545 if err := UpdateMembershipState(context.Background()); err != nil { 546 return nil, errors.Errorf("Error while trying to update membership state: %+v", err) 547 } 548 return nil, errors.Errorf("Unable to reach leader in group %d", n.gid) 549 } 550 return pool, nil 551 } 552 553 func (n *node) Snapshot() (*pb.Snapshot, error) { 554 if n == nil || n.Store == nil { 555 return nil, conn.ErrNoNode 556 } 557 snap, err := n.Store.Snapshot() 558 if err != nil { 559 return nil, err 560 } 561 res := &pb.Snapshot{} 562 if err := res.Unmarshal(snap.Data); err != nil { 563 return nil, err 564 } 565 return res, nil 566 } 567 568 func (n *node) retrieveSnapshot(snap pb.Snapshot) error { 569 // In some edge cases, the Zero leader might not have been able to update 570 // the status of Alpha leader. So, instead of blocking forever on waiting 571 // for Zero to send us the updates info about the leader, we can just use 572 // the Snapshot RaftContext, which contains the address of the leader. 573 var pool *conn.Pool 574 addr := snap.Context.GetAddr() 575 glog.V(2).Infof("Snapshot.RaftContext.Addr: %q", addr) 576 if len(addr) > 0 { 577 p, err := conn.GetPools().Get(addr) 578 if err != nil { 579 glog.V(2).Infof("conn.Get(%q) Error: %v", addr, err) 580 } else { 581 pool = p 582 glog.V(2).Infof("Leader connection picked from RaftContext") 583 } 584 } 585 if pool == nil { 586 glog.V(2).Infof("No leader conn from RaftContext. Using membership state.") 587 p, err := n.leaderBlocking() 588 if err != nil { 589 return err 590 } 591 pool = p 592 } 593 594 // Need to clear pl's stored in memory for the case when retrieving snapshot with 595 // index greater than this node's last index 596 // Should invalidate/remove pl's to this group only ideally 597 // 598 // We can safely evict posting lists from memory. Because, all the updates corresponding to txn 599 // commits up until then have already been written to pstore. And the way we take snapshots, we 600 // keep all the pre-writes for a pending transaction, so they will come back to memory, as Raft 601 // logs are replayed. 602 if _, err := n.populateSnapshot(snap, pool); err != nil { 603 return errors.Wrapf(err, "cannot retrieve snapshot from peer") 604 } 605 // Populate shard stores the streamed data directly into db, so we need to refresh 606 // schema for current group id 607 if err := schema.LoadFromDb(); err != nil { 608 return errors.Wrapf(err, "while initializing schema") 609 } 610 groups().triggerMembershipSync() 611 return nil 612 } 613 614 func (n *node) proposeSnapshot(discardN int) error { 615 snap, err := n.calculateSnapshot(0, discardN) 616 if err != nil { 617 return err 618 } 619 if snap == nil { 620 return nil 621 } 622 proposal := &pb.Proposal{ 623 Snapshot: snap, 624 } 625 n.elog.Printf("Proposing snapshot: %+v\n", snap) 626 data, err := proposal.Marshal() 627 x.Check(err) 628 return n.Raft().Propose(n.ctx, data) 629 } 630 631 const maxPendingSize int64 = 64 << 20 // in bytes. 632 633 func (n *node) rampMeter() { 634 start := time.Now() 635 defer func() { 636 if dur := time.Since(start); dur > time.Second { 637 glog.Infof("Blocked pushing to applyCh for %v", dur.Round(time.Millisecond)) 638 } 639 }() 640 for { 641 if atomic.LoadInt64(&n.pendingSize) <= maxPendingSize { 642 return 643 } 644 time.Sleep(3 * time.Millisecond) 645 } 646 } 647 648 func (n *node) updateRaftProgress() error { 649 // Both leader and followers can independently update their Raft progress. We don't store 650 // this in Raft WAL. Instead, this is used to just skip over log records that this Alpha 651 // has already applied, to speed up things on a restart. 652 // 653 // Let's check what we already have. And only update if the new snap.Index is ahead of the last 654 // stored applied. 655 applied, err := n.Store.Checkpoint() 656 if err != nil { 657 return err 658 } 659 660 snap, err := n.calculateSnapshot(applied, 3) // 3 is a randomly chosen small number. 661 if err != nil || snap == nil || snap.Index <= applied { 662 return err 663 } 664 665 if err := n.Store.UpdateCheckpoint(snap); err != nil { 666 return err 667 } 668 glog.V(2).Infof("[%#x] Set Raft progress to index: %d.", n.Id, snap.Index) 669 return nil 670 } 671 672 func (n *node) checkpointAndClose(done chan struct{}) { 673 slowTicker := time.NewTicker(time.Minute) 674 defer slowTicker.Stop() 675 676 for { 677 select { 678 case <-slowTicker.C: 679 // Do these operations asynchronously away from the main Run loop to allow heartbeats to 680 // be sent on time. Otherwise, followers would just keep running elections. 681 682 n.elog.Printf("Size of applyCh: %d", len(n.applyCh)) 683 if err := n.updateRaftProgress(); err != nil { 684 glog.Errorf("While updating Raft progress: %v", err) 685 } 686 687 if n.AmLeader() { 688 var calculate bool 689 if chk, err := n.Store.Checkpoint(); err == nil { 690 if first, err := n.Store.FirstIndex(); err == nil { 691 // Save some cycles by only calculating snapshot if the checkpoint has gone 692 // quite a bit further than the first index. 693 calculate = chk >= first+uint64(x.WorkerConfig.SnapshotAfter) 694 } 695 } 696 // We keep track of the applied index in the p directory. Even if we don't take 697 // snapshot for a while and let the Raft logs grow and restart, we would not have to 698 // run all the log entries, because we can tell Raft.Config to set Applied to that 699 // index. 700 // This applied index tracking also covers the case when we have a big index 701 // rebuild. The rebuild would be tracked just like others and would not need to be 702 // replayed after a restart, because the Applied config would let us skip right 703 // through it. 704 // We use disk based storage for Raft. So, we're not too concerned about 705 // snapshotting. We just need to do enough, so that we don't have a huge backlog of 706 // entries to process on a restart. 707 if calculate { 708 if err := n.proposeSnapshot(x.WorkerConfig.SnapshotAfter); err != nil { 709 glog.Errorf("While calculating and proposing snapshot: %v", err) 710 } 711 } 712 go n.abortOldTransactions() 713 } 714 715 case <-n.closer.HasBeenClosed(): 716 glog.Infof("Stopping node.Run") 717 if peerId, has := groups().MyPeer(); has && n.AmLeader() { 718 n.Raft().TransferLeadership(n.ctx, x.WorkerConfig.RaftId, peerId) 719 time.Sleep(time.Second) // Let transfer happen. 720 } 721 n.Raft().Stop() 722 close(done) 723 return 724 } 725 } 726 } 727 728 func (n *node) Run() { 729 defer n.closer.Done() // CLOSER:1 730 731 firstRun := true 732 var leader bool 733 // See also our configuration of HeartbeatTick and ElectionTick. 734 // Before we used to have 20ms ticks, but they would overload the Raft tick channel, causing 735 // "tick missed to fire" logs. Etcd uses 100ms and they haven't seen those issues. 736 // Additionally, using 100ms for ticks does not cause proposals to slow down, because they get 737 // sent out asap and don't rely on ticks. So, setting this to 100ms instead of 20ms is a NOOP. 738 ticker := time.NewTicker(100 * time.Millisecond) 739 defer ticker.Stop() 740 741 done := make(chan struct{}) 742 go n.checkpointAndClose(done) 743 go n.ReportRaftComms() 744 745 applied, err := n.Store.Checkpoint() 746 if err != nil { 747 glog.Errorf("While trying to find raft progress: %v", err) 748 } else { 749 glog.Infof("Found Raft progress: %d", applied) 750 } 751 752 var timer x.Timer 753 for { 754 select { 755 case <-done: 756 // We use done channel here instead of closer.HasBeenClosed so that we can transfer 757 // leadership in a goroutine. The push to n.applyCh happens in this loop, so the close 758 // should happen here too. Otherwise, race condition between push and close happens. 759 close(n.applyCh) 760 glog.Infoln("Raft node done.") 761 return 762 763 // Slow ticker can't be placed here because figuring out checkpoints and snapshots takes 764 // time and if the leader does not send heartbeats out during this time, the followers 765 // start an election process. And that election process would just continue to happen 766 // indefinitely because checkpoints and snapshots are being calculated indefinitely. 767 case <-ticker.C: 768 n.Raft().Tick() 769 770 case rd := <-n.Raft().Ready(): 771 timer.Start() 772 _, span := otrace.StartSpan(n.ctx, "Alpha.RunLoop", 773 otrace.WithSampler(otrace.ProbabilitySampler(0.001))) 774 775 if rd.SoftState != nil { 776 groups().triggerMembershipSync() 777 leader = rd.RaftState == raft.StateLeader 778 } 779 if leader { 780 // Leader can send messages in parallel with writing to disk. 781 for _, msg := range rd.Messages { 782 // NOTE: We can do some optimizations here to drop messages. 783 n.Send(msg) 784 } 785 } 786 if span != nil { 787 span.Annotate(nil, "Handled ReadStates and SoftState.") 788 } 789 790 // We move the retrieval of snapshot before we store the rd.Snapshot, so that in case 791 // this node fails to get the snapshot, the Raft state would reflect that by not having 792 // the snapshot on a future probe. This is different from the recommended order in Raft 793 // docs where they assume that the Snapshot contains the full data, so even on a crash 794 // between n.SaveToStorage and n.retrieveSnapshot, that Snapshot can be applied by the 795 // node on a restart. In our case, we don't store the full data in snapshot, only the 796 // metadata. So, we should only store the snapshot received in Raft, iff we actually 797 // were able to update the state. 798 if !raft.IsEmptySnap(rd.Snapshot) { 799 // We don't send snapshots to other nodes. But, if we get one, that means 800 // either the leader is trying to bring us up to state; or this is the 801 // snapshot that I created. Only the former case should be handled. 802 var snap pb.Snapshot 803 x.Check(snap.Unmarshal(rd.Snapshot.Data)) 804 rc := snap.GetContext() 805 x.AssertTrue(rc.GetGroup() == n.gid) 806 if rc.Id != n.Id { 807 // We are getting a new snapshot from leader. We need to wait for the applyCh to 808 // finish applying the updates, otherwise, we'll end up overwriting the data 809 // from the new snapshot that we retrieved. 810 maxIndex := n.Applied.LastIndex() 811 glog.Infof("Waiting for applyCh to become empty by reaching %d before"+ 812 " retrieving snapshot\n", maxIndex) 813 if err := n.Applied.WaitForMark(context.Background(), maxIndex); err != nil { 814 glog.Errorf("Error waiting for mark for index %d: %+v", maxIndex, err) 815 } 816 817 if currSnap, err := n.Snapshot(); err != nil { 818 // Retrieve entire snapshot from leader if node does not have 819 // a current snapshot. 820 glog.Errorf("Could not retrieve previous snapshot. Setting SinceTs to 0.") 821 snap.SinceTs = 0 822 } else { 823 snap.SinceTs = currSnap.ReadTs 824 } 825 826 // It's ok to block ticks while retrieving snapshot, since it's a follower. 827 glog.Infof("---> SNAPSHOT: %+v. Group %d from node id %#x\n", 828 snap, n.gid, rc.Id) 829 830 for { 831 err := n.retrieveSnapshot(snap) 832 if err == nil { 833 glog.Infoln("---> Retrieve snapshot: OK.") 834 break 835 } 836 glog.Errorf("While retrieving snapshot, error: %v. Retrying...", err) 837 time.Sleep(100 * time.Millisecond) // Wait for a bit. 838 } 839 glog.Infof("---> SNAPSHOT: %+v. Group %d. DONE.\n", snap, n.gid) 840 } else { 841 glog.Infof("---> SNAPSHOT: %+v. Group %d from node id %#x [SELF]. Ignoring.\n", 842 snap, n.gid, rc.Id) 843 } 844 if span != nil { 845 span.Annotate(nil, "Applied or retrieved snapshot.") 846 } 847 } 848 849 // Store the hardstate and entries. Note that these are not CommittedEntries. 850 n.SaveToStorage(rd.HardState, rd.Entries, rd.Snapshot) 851 timer.Record("disk") 852 if rd.MustSync { 853 if err := n.Store.Sync(); err != nil { 854 glog.Errorf("Error while calling Store.Sync: %+v", err) 855 } 856 timer.Record("sync") 857 } 858 if span != nil { 859 span.Annotatef(nil, "Saved %d entries. Snapshot, HardState empty? (%v, %v)", 860 len(rd.Entries), 861 raft.IsEmptySnap(rd.Snapshot), 862 raft.IsEmptyHardState(rd.HardState)) 863 } 864 865 // Now schedule or apply committed entries. 866 var proposals []*pb.Proposal 867 for _, entry := range rd.CommittedEntries { 868 // Need applied watermarks for schema mutation also for read linearazibility 869 // Applied watermarks needs to be emitted as soon as possible sequentially. 870 // If we emit Mark{4, false} and Mark{4, true} before emitting Mark{3, false} 871 // then doneUntil would be set as 4 as soon as Mark{4,true} is done and before 872 // Mark{3, false} is emitted. So it's safer to emit watermarks as soon as 873 // possible sequentially 874 n.Applied.Begin(entry.Index) 875 876 if entry.Type == raftpb.EntryConfChange { 877 n.applyConfChange(entry) 878 // Not present in proposal map. 879 n.Applied.Done(entry.Index) 880 groups().triggerMembershipSync() 881 882 } else if len(entry.Data) == 0 { 883 n.elog.Printf("Found empty data at index: %d", entry.Index) 884 n.Applied.Done(entry.Index) 885 886 } else if entry.Index < applied { 887 n.elog.Printf("Skipping over already applied entry: %d", entry.Index) 888 n.Applied.Done(entry.Index) 889 890 } else { 891 proposal := &pb.Proposal{} 892 if err := proposal.Unmarshal(entry.Data); err != nil { 893 x.Fatalf("Unable to unmarshal proposal: %v %q\n", err, entry.Data) 894 } 895 if pctx := n.Proposals.Get(proposal.Key); pctx != nil { 896 atomic.AddUint32(&pctx.Found, 1) 897 if span := otrace.FromContext(pctx.Ctx); span != nil { 898 span.Annotate(nil, "Proposal found in CommittedEntries") 899 } 900 } 901 proposal.Index = entry.Index 902 proposals = append(proposals, proposal) 903 } 904 } 905 // Send the whole lot to applyCh in one go, instead of sending proposals one by one. 906 if len(proposals) > 0 { 907 // Apply the meter this before adding size to pending size so some crazy big 908 // proposal can be pushed to applyCh. If this do this after adding its size to 909 // pending size, we could block forever in rampMeter. 910 n.rampMeter() 911 var pendingSize int64 912 for _, p := range proposals { 913 pendingSize += int64(p.Size()) 914 } 915 if sz := atomic.AddInt64(&n.pendingSize, pendingSize); sz > 2*maxPendingSize { 916 glog.Warningf("Inflight proposal size: %d. There would be some throttling.", sz) 917 } 918 n.applyCh <- proposals 919 } 920 921 if span != nil { 922 span.Annotatef(nil, "Handled %d committed entries.", len(rd.CommittedEntries)) 923 } 924 925 if !leader { 926 // Followers should send messages later. 927 for _, msg := range rd.Messages { 928 // NOTE: We can do some optimizations here to drop messages. 929 n.Send(msg) 930 } 931 } 932 if span != nil { 933 span.Annotate(nil, "Followed queued messages.") 934 } 935 timer.Record("proposals") 936 937 n.Raft().Advance() 938 timer.Record("advance") 939 940 if firstRun && n.canCampaign { 941 go func() { 942 if err := n.Raft().Campaign(n.ctx); err != nil { 943 glog.Errorf("Error starting campaign for node %v: %+v", n.gid, err) 944 } 945 }() 946 firstRun = false 947 } 948 if span != nil { 949 span.Annotate(nil, "Advanced Raft. Done.") 950 span.End() 951 if err := ostats.RecordWithTags(context.Background(), 952 []tag.Mutator{tag.Upsert(x.KeyMethod, "alpha.RunLoop")}, 953 x.LatencyMs.M(float64(timer.Total())/1e6)); err != nil { 954 glog.Errorf("Error recording stats: %+v", err) 955 } 956 } 957 if timer.Total() > 200*time.Millisecond { 958 glog.Warningf( 959 "Raft.Ready took too long to process: %s"+ 960 " Num entries: %d. MustSync: %v", 961 timer.String(), len(rd.Entries), rd.MustSync) 962 } 963 } 964 } 965 } 966 967 func listWrap(kv *bpb.KV) *bpb.KVList { 968 return &bpb.KVList{Kv: []*bpb.KV{kv}} 969 } 970 971 // rollupLists would consolidate all the deltas that constitute one posting 972 // list, and write back a complete posting list. 973 func (n *node) rollupLists(readTs uint64) error { 974 writer := posting.NewTxnWriter(pstore) 975 976 // We're doing rollups. We should use this opportunity to calculate the tablet sizes. 977 amLeader := n.AmLeader() 978 m := new(sync.Map) 979 980 addTo := func(key []byte, delta int64) { 981 if !amLeader { 982 // Only leader needs to calculate the tablet sizes. 983 return 984 } 985 pk, err := x.Parse(key) 986 if err != nil { 987 glog.Errorf("Error while parsing key %s: %v", hex.Dump(key), err) 988 return 989 } 990 val, ok := m.Load(pk.Attr) 991 if !ok { 992 sz := new(int64) 993 val, _ = m.LoadOrStore(pk.Attr, sz) 994 } 995 size := val.(*int64) 996 atomic.AddInt64(size, delta) 997 } 998 999 stream := pstore.NewStreamAt(readTs) 1000 stream.LogPrefix = "Rolling up" 1001 stream.ChooseKey = func(item *badger.Item) bool { 1002 switch item.UserMeta() { 1003 case posting.BitSchemaPosting, posting.BitCompletePosting, posting.BitEmptyPosting: 1004 addTo(item.Key(), item.EstimatedSize()) 1005 return false 1006 case x.ByteUnused: 1007 return false 1008 default: 1009 return true 1010 } 1011 } 1012 var numKeys uint64 1013 stream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { 1014 l, err := posting.ReadPostingList(key, itr) 1015 if err != nil { 1016 return nil, err 1017 } 1018 atomic.AddUint64(&numKeys, 1) 1019 kvs, err := l.Rollup() 1020 1021 // If there are multiple keys, the posting list was split into multiple 1022 // parts. The key of the first part is the right key to use for tablet 1023 // size calculations. 1024 for _, kv := range kvs { 1025 addTo(kvs[0].Key, int64(kv.Size())) 1026 } 1027 1028 return &bpb.KVList{Kv: kvs}, err 1029 } 1030 stream.Send = func(list *bpb.KVList) error { 1031 return writer.Write(list) 1032 } 1033 if err := stream.Orchestrate(context.Background()); err != nil { 1034 return err 1035 } 1036 if err := writer.Flush(); err != nil { 1037 return err 1038 } 1039 // For all the keys, let's see if they're in the LRU cache. If so, we can roll them up. 1040 glog.Infof("Rolled up %d keys. Done", atomic.LoadUint64(&numKeys)) 1041 1042 // We can now discard all invalid versions of keys below this ts. 1043 pstore.SetDiscardTs(readTs) 1044 1045 if amLeader { 1046 // Only leader sends the tablet size updates to Zero. No one else does. 1047 // doSendMembership is also being concurrently called from another goroutine. 1048 go func() { 1049 tablets := make(map[string]*pb.Tablet) 1050 var total int64 1051 m.Range(func(key, val interface{}) bool { 1052 pred := key.(string) 1053 size := atomic.LoadInt64(val.(*int64)) 1054 tablets[pred] = &pb.Tablet{ 1055 GroupId: n.gid, 1056 Predicate: pred, 1057 Space: size, 1058 } 1059 total += size 1060 return true 1061 }) 1062 // Update Zero with the tablet sizes. If Zero sees a tablet which does not belong to 1063 // this group, it would send instruction to delete that tablet. There's an edge case 1064 // here if the followers are still running Rollup, and happen to read a key before and 1065 // write after the tablet deletion, causing that tablet key to resurface. Then, only the 1066 // follower would have that key, not the leader. 1067 // However, if the follower then becomes the leader, we'd be able to get rid of that 1068 // key then. Alternatively, we could look into cancelling the Rollup if we see a 1069 // predicate deletion. 1070 if err := groups().doSendMembership(tablets); err != nil { 1071 glog.Warningf("While sending membership to Zero. Error: %v", err) 1072 } else { 1073 glog.V(2).Infof("Sent tablet size update to Zero. Total size: %s", 1074 humanize.Bytes(uint64(total))) 1075 } 1076 }() 1077 } 1078 return nil 1079 } 1080 1081 var errNoConnection = errors.New("No connection exists") 1082 1083 func (n *node) blockingAbort(req *pb.TxnTimestamps) error { 1084 pl := groups().Leader(0) 1085 if pl == nil { 1086 return errNoConnection 1087 } 1088 zc := pb.NewZeroClient(pl.Get()) 1089 ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) 1090 defer cancel() 1091 1092 delta, err := zc.TryAbort(ctx, req) 1093 glog.Infof("TryAbort %d txns with start ts. Error: %v\n", len(req.Ts), err) 1094 if err != nil || len(delta.Txns) == 0 { 1095 return err 1096 } 1097 1098 // Let's propose the txn updates received from Zero. This is important because there are edge 1099 // cases where a txn status might have been missed by the group. 1100 aborted := &pb.OracleDelta{} 1101 for _, txn := range delta.Txns { 1102 // Only pick the aborts. DO NOT propose the commits. They must come in the right order via 1103 // oracle delta stream, otherwise, we'll end up losing some committed txns. 1104 if txn.CommitTs == 0 { 1105 aborted.Txns = append(aborted.Txns, txn) 1106 } 1107 } 1108 if len(aborted.Txns) == 0 { 1109 glog.Infoln("TryAbort: No aborts found. Quitting.") 1110 return nil 1111 } 1112 1113 // We choose not to store the MaxAssigned, because it would cause our Oracle to move ahead 1114 // artificially. The Oracle delta stream moves that ahead in the right order, and we shouldn't 1115 // muck with that order here. 1116 glog.Infof("TryAbort selectively proposing only aborted txns: %+v\n", aborted) 1117 proposal := &pb.Proposal{Delta: aborted} 1118 return n.proposeAndWait(n.ctx, proposal) 1119 } 1120 1121 // abortOldTransactions would find txns which have done pre-writes, but have been pending for a 1122 // while. The time that is used is based on the last pre-write seen, so if a txn is doing a 1123 // pre-write multiple times, we'll pick the timestamp of the last pre-write. Thus, this function 1124 // would only act on the txns which have not been active in the last N minutes, and send them for 1125 // abort. Note that only the leader runs this function. 1126 func (n *node) abortOldTransactions() { 1127 // Aborts if not already committed. 1128 starts := posting.Oracle().TxnOlderThan(x.WorkerConfig.AbortOlderThan) 1129 if len(starts) == 0 { 1130 return 1131 } 1132 glog.Infof("Found %d old transactions. Acting to abort them.\n", len(starts)) 1133 req := &pb.TxnTimestamps{Ts: starts} 1134 err := n.blockingAbort(req) 1135 glog.Infof("Done abortOldTransactions for %d txns. Error: %+v\n", len(req.Ts), err) 1136 } 1137 1138 // calculateSnapshot would calculate a snapshot index, considering these factors: 1139 // - We only start discarding once we have at least discardN entries. 1140 // - We are not overshooting the max applied entry. That is, we're not removing 1141 // Raft entries before they get applied. 1142 // - We are considering the minimum start ts that has yet to be committed or 1143 // aborted. This way, we still keep all the mutations corresponding to this 1144 // start ts in the Raft logs. This is important, because we don't persist 1145 // pre-writes to disk in pstore. 1146 // - In simple terms, this means we MUST keep all pending transactions in the Raft logs. 1147 // - Find the maximum commit timestamp that we have seen. 1148 // That would tell us about the maximum timestamp used to do any commits. This 1149 // ts is what we can use for future reads of this snapshot. 1150 // - Finally, this function would propose this snapshot index, so the entire 1151 // group can apply it to their Raft stores. 1152 // 1153 // Txn0 | S0 | | | C0 | | | 1154 // Txn1 | | S1 | | | | C1 | 1155 // Txn2 | | | S2 | C2 | | | 1156 // Txn3 | | | | | S3 | | 1157 // Txn4 | | | | | | | S4 1158 // Index | i1 | i2 | i3 | i4 | i5 | i6 | i7 1159 // 1160 // At i7, min pending start ts = S3, therefore snapshotIdx = i5 - 1 = i4. 1161 // At i7, max commit ts = C1, therefore readTs = C1. 1162 // 1163 // This function also takes a startIdx, which can be used an optimization to skip over Raft entries. 1164 // This is useful when we already have a previous snapshot checkpoint (all txns have concluded up 1165 // until that last checkpoint) that we can use as a new start point for the snapshot calculation. 1166 func (n *node) calculateSnapshot(startIdx uint64, discardN int) (*pb.Snapshot, error) { 1167 _, span := otrace.StartSpan(n.ctx, "Calculate.Snapshot", 1168 otrace.WithSampler(otrace.AlwaysSample())) 1169 defer span.End() 1170 1171 // We do not need to block snapshot calculation because of a pending stream. Badger would have 1172 // pending iterators which would ensure that the data above their read ts would not be 1173 // discarded. Secondly, if a new snapshot does get calculated and applied, the follower can just 1174 // ask for the new snapshot. Blocking snapshot calculation has caused us issues when a follower 1175 // somehow kept streaming forever. Then, the leader didn't calculate snapshot, instead it 1176 // kept appending to Raft logs forever causing group wide issues. 1177 1178 first, err := n.Store.FirstIndex() 1179 if err != nil { 1180 span.Annotatef(nil, "Error: %v", err) 1181 return nil, err 1182 } 1183 span.Annotatef(nil, "First index: %d", first) 1184 if startIdx > first { 1185 // If we're starting from a higher index, set first to that. 1186 first = startIdx 1187 span.Annotatef(nil, "Setting first to: %d", startIdx) 1188 } 1189 1190 rsnap, err := n.Store.Snapshot() 1191 if err != nil { 1192 return nil, err 1193 } 1194 var snap pb.Snapshot 1195 if len(rsnap.Data) > 0 { 1196 if err := snap.Unmarshal(rsnap.Data); err != nil { 1197 return nil, err 1198 } 1199 } 1200 span.Annotatef(nil, "Last snapshot: %+v", snap) 1201 1202 last := n.Applied.DoneUntil() 1203 if int(last-first) < discardN { 1204 span.Annotate(nil, "Skipping due to insufficient entries") 1205 return nil, nil 1206 } 1207 span.Annotatef(nil, "Found Raft entries: %d", last-first) 1208 1209 if num := posting.Oracle().NumPendingTxns(); num > 0 { 1210 glog.V(2).Infof("Num pending txns: %d", num) 1211 } 1212 1213 // We can't rely upon the Raft entries to determine the minPendingStart, 1214 // because there are many cases during mutations where we don't commit or 1215 // abort the transaction. This might happen due to an early error thrown. 1216 // Only the mutations which make it to Zero for a commit/abort decision have 1217 // corresponding Delta entries. So, instead of replicating all that logic 1218 // here, we just use the MinPendingStartTs tracked by the Oracle, and look 1219 // for that in the logs. 1220 // 1221 // So, we iterate over logs. If we hit MinPendingStartTs, that generates our 1222 // snapshotIdx. In any case, we continue picking up txn updates, to generate 1223 // a maxCommitTs, which would become the readTs for the snapshot. 1224 minPendingStart := posting.Oracle().MinPendingStartTs() 1225 maxCommitTs := snap.ReadTs 1226 var snapshotIdx uint64 1227 1228 // Trying to retrieve all entries at once might cause out-of-memory issues in 1229 // cases where the raft log is too big to fit into memory. Instead of retrieving 1230 // all entries at once, retrieve it in batches of 64MB. 1231 var lastEntry raftpb.Entry 1232 for batchFirst := first; batchFirst <= last; { 1233 entries, err := n.Store.Entries(batchFirst, last+1, 64<<20) 1234 if err != nil { 1235 span.Annotatef(nil, "Error: %v", err) 1236 return nil, err 1237 } 1238 1239 // Exit early from the loop if no entries were found. 1240 if len(entries) == 0 { 1241 break 1242 } 1243 1244 // Store the last entry (as it might be needed outside the loop) and set the 1245 // start of the new batch at the entry following it. Also set foundEntries to 1246 // true to indicate to the code outside the loop that entries were retrieved. 1247 lastEntry = entries[len(entries)-1] 1248 batchFirst = lastEntry.Index + 1 1249 1250 for _, entry := range entries { 1251 if entry.Type != raftpb.EntryNormal { 1252 continue 1253 } 1254 var proposal pb.Proposal 1255 if err := proposal.Unmarshal(entry.Data); err != nil { 1256 span.Annotatef(nil, "Error: %v", err) 1257 return nil, err 1258 } 1259 if proposal.Mutations != nil { 1260 start := proposal.Mutations.StartTs 1261 if start >= minPendingStart && snapshotIdx == 0 { 1262 snapshotIdx = entry.Index - 1 1263 } 1264 } 1265 if proposal.Delta != nil { 1266 for _, txn := range proposal.Delta.GetTxns() { 1267 maxCommitTs = x.Max(maxCommitTs, txn.CommitTs) 1268 } 1269 } 1270 } 1271 } 1272 1273 if maxCommitTs == 0 { 1274 span.Annotate(nil, "maxCommitTs is zero") 1275 return nil, nil 1276 } 1277 if snapshotIdx <= 0 { 1278 // It is possible that there are no pending transactions. In that case, 1279 // snapshotIdx would be zero. 1280 snapshotIdx = lastEntry.Index 1281 span.Annotatef(nil, "snapshotIdx is zero. Using last entry's index: %d", snapshotIdx) 1282 } 1283 1284 numDiscarding := snapshotIdx - first + 1 1285 span.Annotatef(nil, 1286 "Got snapshotIdx: %d. MaxCommitTs: %d. Discarding: %d. MinPendingStartTs: %d", 1287 snapshotIdx, maxCommitTs, numDiscarding, minPendingStart) 1288 1289 if int(numDiscarding) < discardN { 1290 span.Annotate(nil, "Skipping snapshot because insufficient discard entries") 1291 glog.Infof("Skipping snapshot at index: %d. Insufficient discard entries: %d."+ 1292 " MinPendingStartTs: %d\n", snapshotIdx, numDiscarding, minPendingStart) 1293 return nil, nil 1294 } 1295 1296 result := &pb.Snapshot{ 1297 Context: n.RaftContext, 1298 Index: snapshotIdx, 1299 ReadTs: maxCommitTs, 1300 } 1301 span.Annotatef(nil, "Got snapshot: %+v", result) 1302 return result, nil 1303 } 1304 1305 func (n *node) joinPeers() error { 1306 pl, err := n.leaderBlocking() 1307 if err != nil { 1308 return err 1309 } 1310 1311 gconn := pl.Get() 1312 c := pb.NewRaftClient(gconn) 1313 glog.Infof("Calling JoinCluster via leader: %s", pl.Addr) 1314 if _, err := c.JoinCluster(n.ctx, n.RaftContext); err != nil { 1315 return errors.Wrapf(err, "error while joining cluster") 1316 } 1317 glog.Infof("Done with JoinCluster call\n") 1318 return nil 1319 } 1320 1321 // Checks if its a peer from the leader of the group. 1322 func (n *node) isMember() (bool, error) { 1323 pl, err := n.leaderBlocking() 1324 if err != nil { 1325 return false, err 1326 } 1327 1328 gconn := pl.Get() 1329 c := pb.NewRaftClient(gconn) 1330 glog.Infof("Calling IsPeer") 1331 pr, err := c.IsPeer(n.ctx, n.RaftContext) 1332 if err != nil { 1333 return false, errors.Wrapf(err, "error while joining cluster") 1334 } 1335 glog.Infof("Done with IsPeer call\n") 1336 return pr.Status, nil 1337 } 1338 1339 func (n *node) retryUntilSuccess(fn func() error, pause time.Duration) { 1340 var err error 1341 for { 1342 if err = fn(); err == nil { 1343 break 1344 } 1345 glog.Errorf("Error while calling fn: %v. Retrying...\n", err) 1346 time.Sleep(pause) 1347 } 1348 } 1349 1350 // InitAndStartNode gets called after having at least one membership sync with the cluster. 1351 func (n *node) InitAndStartNode() { 1352 _, restart, err := n.PastLife() 1353 x.Check(err) 1354 1355 if _, hasPeer := groups().MyPeer(); !restart && hasPeer { 1356 // The node has other peers, it might have crashed after joining the cluster and before 1357 // writing a snapshot. Check from leader, if it is part of the cluster. Consider this a 1358 // restart if it is part of the cluster, else start a new node. 1359 for { 1360 if restart, err = n.isMember(); err == nil { 1361 break 1362 } 1363 glog.Errorf("Error while calling hasPeer: %v. Retrying...\n", err) 1364 time.Sleep(time.Second) 1365 } 1366 } 1367 1368 if restart { 1369 glog.Infof("Restarting node for group: %d\n", n.gid) 1370 sp, err := n.Store.Snapshot() 1371 x.Checkf(err, "Unable to get existing snapshot") 1372 if !raft.IsEmptySnap(sp) { 1373 // It is important that we pick up the conf state here. 1374 // Otherwise, we'll lose the store conf state, and it would get 1375 // overwritten with an empty state when a new snapshot is taken. 1376 // This causes a node to just hang on restart, because it finds a 1377 // zero-member Raft group. 1378 n.SetConfState(&sp.Metadata.ConfState) 1379 1380 members := groups().members(n.gid) 1381 for _, id := range sp.Metadata.ConfState.Nodes { 1382 m, ok := members[id] 1383 if ok { 1384 n.Connect(id, m.Addr) 1385 } 1386 } 1387 } 1388 n.SetRaft(raft.RestartNode(n.Cfg)) 1389 glog.V(2).Infoln("Restart node complete") 1390 1391 } else { 1392 glog.Infof("New Node for group: %d\n", n.gid) 1393 if _, hasPeer := groups().MyPeer(); hasPeer { 1394 // Get snapshot before joining peers as it can take time to retrieve it and we dont 1395 // want the quorum to be inactive when it happens. 1396 // Update: This is an optimization, which adds complexity because it requires us to 1397 // understand the Raft state of the node. Let's instead have the node retrieve the 1398 // snapshot as needed after joining the group, instead of us forcing one upfront. 1399 glog.Infoln("Trying to join peers.") 1400 n.retryUntilSuccess(n.joinPeers, time.Second) 1401 n.SetRaft(raft.StartNode(n.Cfg, nil)) 1402 } else { 1403 peers := []raft.Peer{{ID: n.Id}} 1404 n.SetRaft(raft.StartNode(n.Cfg, peers)) 1405 // Trigger election, so this node can become the leader of this single-node cluster. 1406 n.canCampaign = true 1407 } 1408 } 1409 go n.processRollups() 1410 go n.processApplyCh() 1411 go n.BatchAndSendMessages() 1412 go n.Run() 1413 } 1414 1415 func (n *node) AmLeader() bool { 1416 if n.Raft() == nil { 1417 return false 1418 } 1419 r := n.Raft() 1420 return r.Status().Lead == r.Status().ID 1421 }