github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/cluster/database.go (about) 1 // Copyright (c) 2016 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package cluster 22 23 import ( 24 "errors" 25 "fmt" 26 "sync" 27 "time" 28 29 "github.com/m3db/m3/src/cluster/shard" 30 "github.com/m3db/m3/src/dbnode/sharding" 31 "github.com/m3db/m3/src/dbnode/storage" 32 "github.com/m3db/m3/src/dbnode/topology" 33 "github.com/uber-go/tally" 34 "go.uber.org/zap" 35 ) 36 37 var ( 38 // newStorageDatabase is the injected constructor to construct a database, 39 // useful for replacing which database constructor is called in tests 40 newStorageDatabase = storage.NewDatabase 41 42 errAlreadyWatchingTopology = errors.New("cluster database is already watching topology") 43 errNotWatchingTopology = errors.New("cluster database is not watching topology") 44 ) 45 46 type newStorageDatabaseFn func( 47 shardSet sharding.ShardSet, 48 opts storage.Options, 49 ) (storage.Database, error) 50 51 type databaseMetrics struct { 52 initializing tally.Gauge 53 leaving tally.Gauge 54 available tally.Gauge 55 shardsClusterTotal tally.Gauge 56 shardsClusterReplicas tally.Gauge 57 } 58 59 func newDatabaseMetrics(scope tally.Scope) databaseMetrics { 60 shardsScope := scope.SubScope("shards") 61 shardsClusterScope := scope.SubScope("shards-cluster") 62 return databaseMetrics{ 63 initializing: shardsScope.Gauge("initializing"), 64 leaving: shardsScope.Gauge("leaving"), 65 available: shardsScope.Gauge("available"), 66 shardsClusterTotal: shardsClusterScope.Gauge("total"), 67 shardsClusterReplicas: shardsClusterScope.Gauge("replicas"), 68 } 69 } 70 71 type clusterDB struct { 72 storage.Database 73 74 opts storage.Options 75 log *zap.Logger 76 metrics databaseMetrics 77 hostID string 78 topo topology.Topology 79 watch topology.MapWatch 80 81 watchMutex sync.Mutex 82 watching bool 83 doneCh chan struct{} 84 closedCh chan struct{} 85 86 initializing map[uint32]shard.Shard 87 bootstrapCount map[uint32]int 88 } 89 90 // NewDatabase creates a new clustered time series database 91 func NewDatabase( 92 hostID string, 93 topo topology.Topology, 94 topoWatch topology.MapWatch, 95 opts storage.Options, 96 ) (Database, error) { 97 instrumentOpts := opts.InstrumentOptions() 98 log := instrumentOpts.Logger() 99 m := newDatabaseMetrics(instrumentOpts.MetricsScope().SubScope("cluster")) 100 101 log.Info("cluster database initializing topology") 102 103 // Wait for the topology to be available 104 log.Info("cluster database resolving topology") 105 <-topoWatch.C() 106 log.Info("cluster database resolved topology") 107 108 d := &clusterDB{ 109 opts: opts, 110 log: log, 111 metrics: m, 112 hostID: hostID, 113 topo: topo, 114 watch: topoWatch, 115 initializing: make(map[uint32]shard.Shard), 116 bootstrapCount: make(map[uint32]int), 117 } 118 119 shardSet := d.hostOrEmptyShardSet(topoWatch.Get()) 120 db, err := newStorageDatabase(shardSet, opts) 121 if err != nil { 122 return nil, err 123 } 124 125 d.Database = db 126 return d, nil 127 } 128 129 func (d *clusterDB) Topology() topology.Topology { 130 return d.topo 131 } 132 133 func (d *clusterDB) TopologyMap() (topology.Map, error) { 134 return d.topo.Get(), nil 135 } 136 137 func (d *clusterDB) Open() error { 138 select { 139 case <-d.watch.C(): 140 shardSet := d.hostOrEmptyShardSet(d.watch.Get()) 141 d.Database.AssignShardSet(shardSet) 142 default: 143 // No updates to the topology since cluster DB created 144 } 145 if err := d.Database.Open(); err != nil { 146 return err 147 } 148 return d.startActiveTopologyWatch() 149 } 150 151 func (d *clusterDB) Close() error { 152 if err := d.Database.Close(); err != nil { 153 return err 154 } 155 return d.stopActiveTopologyWatch() 156 } 157 158 // IsBootstrappedAndDurable determines whether the database is bootstrapped 159 // and durable, meaning that it could recover all data in memory using only 160 // the local disk. 161 // 162 // The logic here is a little tricky because there are two levels of 163 // IsBootstrappedAndDurable(): 164 // 165 // The first level is this method which exists on the clustered database. It is 166 // used by our health check endpoints and tooling in general to determine when 167 // it is safe to continue a deploy or performing topology changes. In that case, 168 // we only need to determine two things: 169 // 170 // 1. Is the node bootstrapped? 171 // 2. Are all of its shards available? 172 // 173 // If so, then the node has finished bootstrapping and will be able to recover 174 // all of its data (assuming the default bootstrapper configuration of 175 // [filesystem, commitlog, peers]) if it goes down and its safe to continue 176 // operations. The reason this is true is because a node will ONLY mark its shards 177 // as available once it reaches a point where it is durable for the new shards it 178 // has received, and M3DB is architected in such a way (again, assuming the default 179 // bootstrapping configuration) that once a node reaches the AVAILABLE state it will 180 // always remain durable for that shard. 181 // 182 // The implications of only checking those two conditions means that when we're 183 // deploying a cluster, we only have to wait for the node to finish bootstrapping 184 // because all the shards will already be in the AVAILABLE state. When performing 185 // topology changes (like adding nodes) we'll have to wait until the node finishes 186 // bootstrapping AND that it marks its newly acquired shards as available. This is 187 // also desired because it means that the operations won't proceed until this node 188 // is capable of restoring all of the data it is responsible for from its own disk 189 // without relying on its peers. 190 // 191 // The second level of IsBootstrappedAndDurable exists on the storage database (see 192 // the comment on that method for a high-level overview of the conditions it checks 193 // for) and we only use that method when we're trying to determine if it is safe to 194 // mark newly acquired shards as AVAILABLE. That method is responsible for determining 195 // that all the shards it has been assigned are durable. The storage database method 196 // is very precautious so we want to avoid checking it if we don't have to (I.E when our 197 // shards are already in the AVAILABLE state) because it would significantly slow down 198 // our deployments and topology changes operations as every step would require the nodes 199 // to wait for a complete snapshot to take place before proceeding, when in fact that is 200 // often not required for correctness. 201 func (d *clusterDB) IsBootstrappedAndDurable() bool { 202 if !d.Database.IsBootstrapped() { 203 return false 204 } 205 206 _, ok := d.topo.(topology.DynamicTopology) 207 if !ok { 208 // If the topology is not dynamic, then the only thing we care 209 // about is whether the node is bootstrapped or not because the 210 // concept of durability as it relates to shard state doesn't 211 // make sense if we're using a static topology. 212 // 213 // In other words, we don't need to check the shards states because 214 // they don't change, and we don't need to check if the storage 215 // database IsBootstrappedAndDurable() because that is only important 216 // when we're trying to figure out if the storage database has become 217 // durable since we made a topology change which is not possible with 218 // a static topology. 219 return true 220 } 221 222 entry, ok := d.watch.Get().LookupHostShardSet(d.hostID) 223 if !ok { 224 // If we're bootstrapped, but not in the placement, then we 225 // are durable because we don't have any data we need to store 226 // anyways. 227 return true 228 } 229 230 for _, s := range entry.ShardSet().All() { 231 switch s.State() { 232 case shard.Leaving: 233 continue 234 case shard.Available: 235 continue 236 } 237 238 return false 239 } 240 241 // If all of the shards we own are either LEAVING or AVAILABLE then we know 242 // we are durable because we will only mark shards as AVAILABLE once we become 243 // durable for them, and then once a shard has reached the AVAILABLE state we 244 // are responsible for always remaining in a durable state. 245 return true 246 } 247 248 func (d *clusterDB) startActiveTopologyWatch() error { 249 d.watchMutex.Lock() 250 defer d.watchMutex.Unlock() 251 252 if d.watching { 253 return errAlreadyWatchingTopology 254 } 255 256 d.watching = true 257 258 d.doneCh = make(chan struct{}, 1) 259 d.closedCh = make(chan struct{}, 1) 260 261 go d.activeTopologyWatch() 262 263 return nil 264 } 265 266 func (d *clusterDB) stopActiveTopologyWatch() error { 267 d.watchMutex.Lock() 268 defer d.watchMutex.Unlock() 269 270 if !d.watching { 271 return errNotWatchingTopology 272 } 273 274 d.watching = false 275 276 close(d.doneCh) 277 <-d.closedCh 278 279 return nil 280 } 281 282 func (d *clusterDB) activeTopologyWatch() { 283 reportClosingCh := make(chan struct{}, 1) 284 reportClosedCh := make(chan struct{}, 1) 285 go func() { 286 ticker := time.NewTicker(time.Second) 287 for { 288 select { 289 case <-ticker.C: 290 d.analyzeAndReportShardStates() 291 case <-reportClosingCh: 292 ticker.Stop() 293 close(reportClosedCh) 294 return 295 } 296 } 297 }() 298 299 defer func() { 300 // Issue closing signal to report channel 301 close(reportClosingCh) 302 // Wait for report channel to close 303 <-reportClosedCh 304 // Signal all closed 305 close(d.closedCh) 306 }() 307 308 for { 309 select { 310 case <-d.doneCh: 311 return 312 case _, ok := <-d.watch.C(): 313 // NB(prateek): cluster/Database shares the topology with client/Session, so we 314 // explicitly check if the watch channel has been closed 315 if !ok { 316 return 317 } 318 d.log.Info("received update from kv topology watch") 319 shardSet := d.hostOrEmptyShardSet(d.watch.Get()) 320 d.Database.AssignShardSet(shardSet) 321 } 322 } 323 } 324 325 func (d *clusterDB) analyzeAndReportShardStates() { 326 placement := d.watch.Get() 327 entry, ok := placement.LookupHostShardSet(d.hostID) 328 if !ok { 329 return 330 } 331 332 reportStats := func() { 333 var ( 334 initializing int64 335 leaving int64 336 available int64 337 ) 338 for _, s := range entry.ShardSet().All() { 339 switch s.State() { 340 case shard.Initializing: 341 initializing++ 342 case shard.Leaving: 343 leaving++ 344 case shard.Available: 345 available++ 346 } 347 } 348 d.metrics.initializing.Update(float64(initializing)) 349 d.metrics.leaving.Update(float64(leaving)) 350 d.metrics.available.Update(float64(available)) 351 shardsClusterTotal := len(placement.ShardSet().All()) 352 d.metrics.shardsClusterTotal.Update(float64(shardsClusterTotal)) 353 d.metrics.shardsClusterReplicas.Update(float64(placement.Replicas())) 354 } 355 356 defer reportStats() 357 358 // Manage the reusable vars 359 d.resetReusable() 360 defer d.resetReusable() 361 362 for _, s := range entry.ShardSet().All() { 363 if s.State() == shard.Initializing { 364 d.initializing[s.ID()] = s 365 } 366 } 367 368 if len(d.initializing) == 0 { 369 // No initializing shards 370 return 371 } 372 373 // To mark any initializing shards as available we need a 374 // dynamic topology, check if we have one and if not we will report 375 // that shards are initialzing and that we do not have a dynamic 376 // topology to mark them as available. 377 topo, ok := d.topo.(topology.DynamicTopology) 378 if !ok { 379 err := fmt.Errorf("topology constructed is not a dynamic topology") 380 d.log.Error("cluster db cannot mark shard available", zap.Error(err)) 381 return 382 } 383 384 // Call IsBootstrappedAndDurable on storage database, not cluster. 385 if !d.Database.IsBootstrappedAndDurable() { 386 return 387 } 388 389 // Count if initializing shards have bootstrapped in all namespaces. This 390 // check is redundant with the database check above, but we do it for 391 // posterity just to make sure everything is in the correct state. 392 namespaces := d.Database.Namespaces() 393 for _, n := range namespaces { 394 for _, s := range n.Shards() { 395 if _, ok := d.initializing[s.ID()]; !ok { 396 continue 397 } 398 if !s.IsBootstrapped() { 399 continue 400 } 401 d.bootstrapCount[s.ID()]++ 402 } 403 } 404 405 var markAvailable []uint32 406 for id := range d.initializing { 407 count := d.bootstrapCount[id] 408 if count != len(namespaces) { 409 // This could temporarily occur due to the race condition, e.g. database was bootstrapped and durable 410 // at the time we checked but then new shards were assigned which are still not bootstrapped. 411 d.log.Debug("database indicated that it was bootstrapped and durable, "+ 412 "but number of bootstrapped shards did not match number of namespaces", 413 zap.Uint32("shard", id), 414 zap.Int("count", count), 415 zap.Int("numNamespaces", len(namespaces))) 416 continue 417 } 418 419 // Mark this shard as available 420 if markAvailable == nil { 421 // Defer allocation until needed, alloc as much as could be required. 422 markAvailable = make([]uint32, 0, len(d.initializing)) 423 } 424 markAvailable = append(markAvailable, id) 425 } 426 427 if len(markAvailable) == 0 { 428 return 429 } 430 431 if err := topo.MarkShardsAvailable(d.hostID, markAvailable...); err != nil { 432 d.log.Error("cluster db failed marking shards available", 433 zap.Uint32s("shards", markAvailable), zap.Error(err)) 434 return 435 } 436 437 d.log.Info("cluster db successfully marked shards as available", 438 zap.Uint32s("shards", markAvailable)) 439 } 440 441 func (d *clusterDB) resetReusable() { 442 d.resetInitializing() 443 d.resetBootstrapCount() 444 } 445 446 func (d *clusterDB) resetInitializing() { 447 for id := range d.initializing { 448 delete(d.initializing, id) 449 } 450 } 451 452 func (d *clusterDB) resetBootstrapCount() { 453 for id := range d.bootstrapCount { 454 delete(d.bootstrapCount, id) 455 } 456 } 457 458 // hostOrEmptyShardSet returns a shard set for the given host ID from a 459 // topology map and if none exists then an empty shard set. If successfully 460 // found the shard set for the host the second parameter returns true, 461 // otherwise false. 462 func (d *clusterDB) hostOrEmptyShardSet(m topology.Map) sharding.ShardSet { 463 if hostShardSet, ok := m.LookupHostShardSet(d.hostID); ok { 464 return hostShardSet.ShardSet() 465 } 466 d.log.Warn("topology has no shard set for host ID", zap.String("hostID", d.hostID)) 467 return sharding.NewEmptyShardSet(m.ShardSet().HashFn()) 468 }