github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/kv/kvserver/closedts/storage/storage.go (about) 1 // Copyright 2018 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package storage 12 13 import ( 14 "bytes" 15 "fmt" 16 "sort" 17 "unsafe" 18 19 "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" 20 "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" 21 "github.com/cockroachdb/cockroach/pkg/roachpb" 22 "github.com/cockroachdb/cockroach/pkg/util/syncutil" 23 ) 24 25 // SingleStorage stores and manages closed timestamp updates originating from a 26 // single source (i.e. node). A SingleStorage internally maintains multiple 27 // buckets for historical closed timestamp information. The reason for this is 28 // twofold: 29 // 30 // 1. The most recent closed timestamp update is also the hardest to prove a 31 // read for, since it comes with larger minimum lease applied indexes. In 32 // situations in which followers are lagging behind with their command 33 // application, this could lead to a runaway scenario, in which a closed 34 // timestamp update can never be used until it is replaced by a new one, which 35 // in turn also will never be used, etc. Instead, a SingleStorage keeps some 36 // amount of history and upstream systems can try to prove a follower read using 37 // an older closed timestamp instead. 38 // 39 // 2. Follower reads can be used to implement recovery of a consistent 40 // cluster-wide snapshot after catastrophic loss of quorum. To do this, the 41 // mechanism must locate at least one replica of every range in the cluster, and 42 // for each range find the largest possible timestamp at which follower reads 43 // are possible among the surviving replicas. Of all these per-range timestamps, 44 // the smallest can be used to read from all ranges, resulting in a consistent 45 // snapshot. This makes it crucial that every replica can serve at least some 46 // follower reads, even when regularly outpaced by the closed timestamp 47 // frontier. Emitted MLAIs may never even be proposed to Raft in the event of 48 // an ill-timed crash, and so historic information is invaluable. 49 // 50 // TODO(tschottdorf): revisit whether this shouldn't be a concrete impl instead, 51 // with only the buckets abstracted out. 52 type SingleStorage interface { 53 fmt.Stringer 54 // VisitAscending walks through the buckets of the storage in ascending 55 // closed timestamp order, until the closure returns true (or all buckets 56 // have been visited). 57 VisitAscending(func(ctpb.Entry) (done bool)) 58 // VisitDescending walks through the buckets of the storage in descending 59 // closed timestamp order, until the closure returns true (or all buckets 60 // have been visited). 61 VisitDescending(func(ctpb.Entry) (done bool)) 62 // Add adds a new Entry to this storage. The entry is added to the most 63 // recent bucket and remaining buckets are rotated as indicated by their age 64 // relative to the newly added Entry. 65 Add(ctpb.Entry) 66 // Clear removes all Entries from this storage. 67 Clear() 68 } 69 70 type entry struct { 71 SingleStorage 72 } 73 74 // MultiStorage implements the closedts.Storage interface. 75 type MultiStorage struct { 76 // constructor creates a SingleStorage whenever one is initialized for a new 77 // NodeID. 78 constructor func() SingleStorage 79 // TODO(tschottdorf): clean up storages that haven't been used for extended 80 // periods of time. 81 m syncutil.IntMap 82 } 83 84 var _ closedts.Storage = (*MultiStorage)(nil) 85 86 // NewMultiStorage sets up a MultiStorage which uses the given factory method 87 // for setting up the SingleStorage used for each individual NodeID for which 88 // operations are received. 89 func NewMultiStorage(constructor func() SingleStorage) *MultiStorage { 90 return &MultiStorage{constructor: constructor} 91 } 92 93 func (ms *MultiStorage) getOrCreate(nodeID roachpb.NodeID) SingleStorage { 94 key := int64(nodeID) 95 p, found := ms.m.Load(key) 96 if found { 97 // Fast path that avoids calling f(). 98 return (*entry)(p).SingleStorage 99 } 100 101 ss := ms.constructor() 102 p, _ = ms.m.LoadOrStore(key, unsafe.Pointer(&entry{ss})) 103 return (*entry)(p).SingleStorage 104 } 105 106 // VisitAscending implements closedts.Storage. 107 func (ms *MultiStorage) VisitAscending(nodeID roachpb.NodeID, f func(ctpb.Entry) (done bool)) { 108 ss := ms.getOrCreate(nodeID) 109 ss.VisitAscending(f) 110 } 111 112 // VisitDescending implements closedts.Storage. 113 func (ms *MultiStorage) VisitDescending(nodeID roachpb.NodeID, f func(ctpb.Entry) (done bool)) { 114 ss := ms.getOrCreate(nodeID) 115 ss.VisitDescending(f) 116 } 117 118 // Add implements closedts.Storage. 119 func (ms *MultiStorage) Add(nodeID roachpb.NodeID, entry ctpb.Entry) { 120 ss := ms.getOrCreate(nodeID) 121 ss.Add(entry) 122 } 123 124 // Clear implements closedts.Storage. 125 func (ms *MultiStorage) Clear() { 126 ms.m.Range(func(_ int64, p unsafe.Pointer) bool { 127 (*entry)(p).SingleStorage.Clear() 128 return true // continue 129 }) 130 } 131 132 // String prints a tabular rundown of the contents of the MultiStorage. 133 func (ms *MultiStorage) String() string { 134 return ms.StringForNodes() 135 } 136 137 // StringForNodes is like String, but restricted to the supplied NodeIDs. 138 // If none are specified, is equivalent to String(). 139 func (ms *MultiStorage) StringForNodes(nodes ...roachpb.NodeID) string { 140 type tuple struct { 141 roachpb.NodeID 142 SingleStorage 143 } 144 145 var shouldPrint map[roachpb.NodeID]struct{} 146 if len(nodes) > 0 { 147 shouldPrint = make(map[roachpb.NodeID]struct{}, len(nodes)) 148 for _, nodeID := range nodes { 149 shouldPrint[nodeID] = struct{}{} 150 } 151 } 152 153 var sl []tuple 154 ms.m.Range(func(k int64, p unsafe.Pointer) bool { 155 nodeID := roachpb.NodeID(k) 156 if _, ok := shouldPrint[nodeID]; ok || len(shouldPrint) == 0 { 157 sl = append(sl, tuple{nodeID, (*entry)(p).SingleStorage}) 158 } 159 return true // want more 160 }) 161 sort.Slice(sl, func(i, j int) bool { 162 return sl[i].NodeID < sl[j].NodeID 163 }) 164 var buf bytes.Buffer 165 for i := range sl { 166 buf.WriteString(fmt.Sprintf("***** n%d *****\n", sl[i].NodeID)) 167 buf.WriteString(sl[i].SingleStorage.String()) 168 } 169 return buf.String() 170 }