github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/bootstrap.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 storage 22 23 import ( 24 "errors" 25 "fmt" 26 "sync" 27 "time" 28 29 "github.com/m3db/m3/src/dbnode/storage/bootstrap" 30 "github.com/m3db/m3/src/x/clock" 31 "github.com/m3db/m3/src/x/context" 32 xerrors "github.com/m3db/m3/src/x/errors" 33 xtime "github.com/m3db/m3/src/x/time" 34 ) 35 36 var ( 37 // errNamespaceIsBootstrapping raised when trying to bootstrap a namespace that's being bootstrapped. 38 errNamespaceIsBootstrapping = errors.New("namespace is bootstrapping") 39 40 // errNamespaceNotBootstrapped raised when trying to flush/snapshot data for a namespace that's not yet bootstrapped. 41 errNamespaceNotBootstrapped = errors.New("namespace is not yet bootstrapped") 42 43 // errShardIsBootstrapping raised when trying to bootstrap a shard that's being bootstrapped. 44 errShardIsBootstrapping = errors.New("shard is bootstrapping") 45 46 // errShardNotBootstrappedToFlush raised when trying to flush data for a shard that's not yet bootstrapped. 47 errShardNotBootstrappedToFlush = errors.New("shard is not yet bootstrapped to flush") 48 49 // errShardNotBootstrappedToSnapshot raised when trying to snapshot data for a shard that's not yet bootstrapped. 50 errShardNotBootstrappedToSnapshot = errors.New("shard is not yet bootstrapped to snapshot") 51 52 // errShardNotBootstrappedToRead raised when trying to read data for a shard that's not yet bootstrapped. 53 errShardNotBootstrappedToRead = errors.New("shard is not yet bootstrapped to read") 54 55 // errIndexNotBootstrappedToRead raised when trying to read the index before being bootstrapped. 56 errIndexNotBootstrappedToRead = errors.New("index is not yet bootstrapped to read") 57 58 // errBootstrapEnqueued raised when trying to bootstrap and bootstrap becomes enqueued. 59 errBootstrapEnqueued = errors.New("database bootstrapping enqueued bootstrap") 60 ) 61 62 const ( 63 bootstrapRetryInterval = 10 * time.Second 64 ) 65 66 type bootstrapFn func() error 67 68 type bootstrapNamespace struct { 69 namespace databaseNamespace 70 shards []databaseShard 71 } 72 73 type bootstrapManager struct { 74 sync.RWMutex 75 76 database database 77 mediator databaseMediator 78 bootstrapFn bootstrapFn 79 processProvider bootstrap.ProcessProvider 80 state BootstrapState 81 hasPending bool 82 pendingOnCompleteFns []BootstrapCompleteFn 83 sleepFn sleepFn 84 nowFn clock.NowFn 85 lastBootstrapCompletionTime xtime.UnixNano 86 instrumentation *bootstrapInstrumentation 87 } 88 89 func newBootstrapManager( 90 database database, 91 mediator databaseMediator, 92 opts Options, 93 ) databaseBootstrapManager { 94 m := &bootstrapManager{ 95 database: database, 96 mediator: mediator, 97 processProvider: opts.BootstrapProcessProvider(), 98 sleepFn: time.Sleep, 99 nowFn: opts.ClockOptions().NowFn(), 100 instrumentation: newBootstrapInstrumentation(opts), 101 } 102 m.bootstrapFn = m.bootstrap 103 return m 104 } 105 106 func (m *bootstrapManager) IsBootstrapped() bool { 107 m.RLock() 108 state := m.state 109 m.RUnlock() 110 return state == Bootstrapped 111 } 112 113 func (m *bootstrapManager) LastBootstrapCompletionTime() (xtime.UnixNano, bool) { 114 m.RLock() 115 bsTime := m.lastBootstrapCompletionTime 116 m.RUnlock() 117 return bsTime, bsTime > 0 118 } 119 120 func (m *bootstrapManager) BootstrapEnqueue( 121 opts BootstrapEnqueueOptions, 122 ) { 123 go func() { 124 result, err := m.startBootstrap(opts.OnCompleteFn) 125 if err != nil && !result.AlreadyBootstrapping { 126 m.instrumentation.emitAndLogInvariantViolation(err, "error bootstrapping") 127 } 128 }() 129 } 130 131 func (m *bootstrapManager) Bootstrap() (BootstrapResult, error) { 132 return m.startBootstrap(nil) 133 } 134 135 func (m *bootstrapManager) startBootstrap( 136 onCompleteFn BootstrapCompleteFn, 137 ) (BootstrapResult, error) { 138 m.Lock() 139 if onCompleteFn != nil { 140 // Append completion fn if specified. 141 m.pendingOnCompleteFns = append(m.pendingOnCompleteFns, onCompleteFn) 142 } 143 switch m.state { 144 case Bootstrapping: 145 // NB(r): Already bootstrapping, now a consequent bootstrap 146 // request comes in - we queue this up to bootstrap again 147 // once the current bootstrap has completed. 148 // This is an edge case that can occur if during either an 149 // initial bootstrap or a resharding bootstrap if a new 150 // reshard occurs and we need to bootstrap more shards. 151 m.hasPending = true 152 m.Unlock() 153 result := BootstrapResult{AlreadyBootstrapping: true} 154 return result, errBootstrapEnqueued 155 default: 156 m.state = Bootstrapping 157 } 158 m.Unlock() 159 // NB(xichen): disable filesystem manager before we bootstrap to minimize 160 // the impact of file operations on bootstrapping performance 161 m.instrumentation.log.Info("disable fileOps and wait") 162 m.mediator.DisableFileOpsAndWait() 163 defer m.mediator.EnableFileOps() 164 m.instrumentation.log.Info("fileOps disabled") 165 166 var result BootstrapResult 167 // Keep performing bootstraps until none pending and no error returned. 168 for i := 0; true; i++ { 169 // NB(r): Decouple implementation of bootstrap so can override in tests. 170 bootstrapErr := m.bootstrapFn() 171 if bootstrapErr != nil { 172 result.ErrorsBootstrap = append(result.ErrorsBootstrap, bootstrapErr) 173 } 174 175 m.Lock() 176 currPending := m.hasPending 177 if currPending { 178 // New bootstrap calls should now enqueue another pending bootstrap 179 m.hasPending = false 180 } 181 m.Unlock() 182 183 if currPending { 184 // NB(r): Requires another bootstrap. 185 continue 186 } 187 188 if bootstrapErr != nil { 189 // NB(r): Last bootstrap failed, since this could be due to transient 190 // failure we retry the bootstrap again. This is to avoid operators 191 // needing to manually intervene for cases where failures are transient. 192 m.instrumentation.bootstrapFailed(i+1, bootstrapErr) 193 m.sleepFn(bootstrapRetryInterval) 194 continue 195 } 196 197 // No pending bootstraps and last finished successfully. 198 break 199 } 200 201 // NB(xichen): in order for bootstrapped data to be flushed to disk, a tick 202 // needs to happen to drain the in-memory buffers and a consequent flush will 203 // flush all the data onto disk. However, this has shown to be too intensive 204 // to do immediately after bootstrap due to bootstrapping nodes simultaneously 205 // attempting to tick through their series and flushing data, adding significant 206 // load to the cluster. It turns out to be better to let ticking happen naturally 207 // on its own course so that the load of ticking and flushing is more spread out 208 // across the cluster. 209 m.Lock() 210 m.lastBootstrapCompletionTime = xtime.ToUnixNano(m.nowFn()) 211 m.state = Bootstrapped 212 // NB(r): Clear out the pending completion functions and execute them if 213 // needed. 214 pendingOnCompleteFns := m.pendingOnCompleteFns 215 m.pendingOnCompleteFns = nil 216 m.Unlock() 217 218 if len(pendingOnCompleteFns) > 0 { 219 // Execute any on complete functions that were queued. 220 for _, fn := range pendingOnCompleteFns { 221 fn(result) 222 } 223 } 224 225 return result, nil 226 } 227 228 func (m *bootstrapManager) Report() { 229 m.instrumentation.setIsBootstrapped(m.IsBootstrapped()) 230 m.instrumentation.setIsBootstrappedAndDurable(m.database.IsBootstrappedAndDurable()) 231 } 232 233 func (m *bootstrapManager) bootstrap() error { 234 ctx := context.NewBackground() 235 defer ctx.Close() 236 237 // NB(r): construct new instance of the bootstrap process to avoid 238 // state being kept around by bootstrappers. 239 process, err := m.processProvider.Provide() 240 if err != nil { 241 return err 242 } 243 244 namespaces, err := m.database.OwnedNamespaces() 245 if err != nil { 246 return err 247 } 248 249 instrCtx := m.instrumentation.bootstrapPreparing() 250 251 accmulators := make([]bootstrap.NamespaceDataAccumulator, 0, len(namespaces)) 252 defer func() { 253 // Close all accumulators at bootstrap completion, only error 254 // it returns is if already closed, so this is a code bug if ever 255 // an error returned. 256 for _, accumulator := range accmulators { 257 if err := accumulator.Close(); err != nil { 258 instrCtx.emitAndLogInvariantViolation(err, "could not close bootstrap data accumulator") 259 } 260 } 261 }() 262 263 var ( 264 bootstrapNamespaces = make([]bootstrapNamespace, len(namespaces)) 265 prepareWg sync.WaitGroup 266 prepareLock sync.Mutex 267 prepareMultiErr xerrors.MultiError 268 ) 269 for i, namespace := range namespaces { 270 i, namespace := i, namespace 271 prepareWg.Add(1) 272 go func() { 273 shards, err := namespace.PrepareBootstrap(ctx) 274 275 prepareLock.Lock() 276 defer func() { 277 prepareLock.Unlock() 278 prepareWg.Done() 279 }() 280 281 if err != nil { 282 prepareMultiErr = prepareMultiErr.Add(err) 283 return 284 } 285 286 bootstrapNamespaces[i] = bootstrapNamespace{ 287 namespace: namespace, 288 shards: shards, 289 } 290 }() 291 } 292 293 prepareWg.Wait() 294 295 if err := prepareMultiErr.FinalError(); err != nil { 296 m.instrumentation.bootstrapPrepareFailed(err) 297 return err 298 } 299 300 var uniqueShards map[uint32]struct{} 301 targets := make([]bootstrap.ProcessNamespace, 0, len(namespaces)) 302 for _, ns := range bootstrapNamespaces { 303 bootstrapShards := make([]uint32, 0, len(ns.shards)) 304 if uniqueShards == nil { 305 uniqueShards = make(map[uint32]struct{}, len(ns.shards)) 306 } 307 308 for _, shard := range ns.shards { 309 if shard.IsBootstrapped() { 310 continue 311 } 312 313 uniqueShards[shard.ID()] = struct{}{} 314 bootstrapShards = append(bootstrapShards, shard.ID()) 315 } 316 317 // Add hooks so that each bootstrapper when it interacts 318 // with the namespace and shards during data accumulation 319 // gets an up to date view of all the file volumes that 320 // actually exist on disk (since bootstrappers can write 321 // new blocks to disk). 322 hooks := bootstrap.NewNamespaceHooks(bootstrap.NamespaceHooksOptions{ 323 BootstrapSourceEnd: newBootstrapSourceEndHook(ns.shards), 324 }) 325 326 accumulator := NewDatabaseNamespaceDataAccumulator(ns.namespace) 327 accmulators = append(accmulators, accumulator) 328 329 targets = append(targets, bootstrap.ProcessNamespace{ 330 Metadata: ns.namespace.Metadata(), 331 Shards: bootstrapShards, 332 Hooks: hooks, 333 DataAccumulator: accumulator, 334 }) 335 } 336 337 instrCtx.bootstrapStarted(len(uniqueShards)) 338 // Run the bootstrap. 339 bootstrapResult, err := process.Run(ctx, instrCtx.start, targets) 340 if err != nil { 341 instrCtx.bootstrapFailed(err) 342 return err 343 } 344 345 instrCtx.bootstrapSucceeded() 346 347 instrCtx.bootstrapNamespacesStarted() 348 // Use a multi-error here because we want to at least bootstrap 349 // as many of the namespaces as possible. 350 multiErr := xerrors.NewMultiError() 351 for _, namespace := range namespaces { 352 id := namespace.ID() 353 result, ok := bootstrapResult.Results.Get(id) 354 if !ok { 355 err := fmt.Errorf("missing namespace from bootstrap result: %v", 356 id.String()) 357 instrCtx.emitAndLogInvariantViolation(err, "bootstrap failed") 358 return err 359 } 360 361 if err := namespace.Bootstrap(ctx, result); err != nil { 362 instrCtx.bootstrapNamespaceFailed(err, id) 363 multiErr = multiErr.Add(err) 364 } 365 } 366 367 if err := multiErr.FinalError(); err != nil { 368 instrCtx.bootstrapNamespacesFailed(err) 369 return err 370 } 371 372 instrCtx.bootstrapNamespacesSucceeded() 373 return nil 374 }