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