forked from couchbase/indexing
-
Notifications
You must be signed in to change notification settings - Fork 0
/
storage_manager.go
1004 lines (823 loc) · 27.8 KB
/
storage_manager.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
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright (c) 2014 Couchbase, Inc.
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file
// except in compliance with the License. You may obtain a copy of the License at
// http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed under the
// License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
// either express or implied. See the License for the specific language governing permissions
// and limitations under the License.
package indexer
import (
"bytes"
"encoding/gob"
"errors"
"github.com/couchbase/indexing/secondary/common"
"github.com/couchbase/indexing/secondary/fdb"
"github.com/couchbase/indexing/secondary/logging"
"sync"
"time"
)
var (
ErrIndexRollback = errors.New("Indexer rollback")
ErrIndexRollbackOrBootstrap = errors.New("Indexer rollback or warmup")
)
//StorageManager manages the snapshots for the indexes and responsible for storing
//indexer metadata in a config database
const INST_MAP_KEY_NAME = "IndexInstMap"
type StorageManager interface {
}
type storageMgr struct {
supvCmdch MsgChannel //supervisor sends commands on this channel
supvRespch MsgChannel //channel to send any async message to supervisor
snapshotNotifych chan IndexSnapshot
indexInstMap common.IndexInstMap
indexPartnMap IndexPartnMap
// Latest readable index snapshot for each index instance
indexSnapMap map[common.IndexInstId]IndexSnapshot
// List of waiters waiting for a snapshot to be created with expected
// atleast-timestamp
waitersMap map[common.IndexInstId][]*snapshotWaiter
dbfile *forestdb.File
meta *forestdb.KVStore // handle for index meta
config common.Config
stats IndexerStatsHolder
muSnap sync.Mutex //lock to protect snapMap and waitersMap
}
type IndexSnapMap map[common.IndexInstId]IndexSnapshot
type snapshotWaiter struct {
wch chan interface{}
ts *common.TsVbuuid
cons common.Consistency
idxInstId common.IndexInstId
expired time.Time
}
func newSnapshotWaiter(idxId common.IndexInstId, ts *common.TsVbuuid,
cons common.Consistency,
ch chan interface{}, expired time.Time) *snapshotWaiter {
return &snapshotWaiter{
ts: ts,
cons: cons,
wch: ch,
idxInstId: idxId,
expired: expired,
}
}
func (w *snapshotWaiter) Notify(is IndexSnapshot) {
w.wch <- is
}
func (w *snapshotWaiter) Error(err error) {
w.wch <- err
}
//NewStorageManager returns an instance of storageMgr or err message
//It listens on supvCmdch for command and every command is followed
//by a synchronous response of the supvCmdch.
//Any async response to supervisor is sent to supvRespch.
//If supvCmdch get closed, storageMgr will shut itself down.
func NewStorageManager(supvCmdch MsgChannel, supvRespch MsgChannel,
indexPartnMap IndexPartnMap, config common.Config, snapshotNotifych chan IndexSnapshot) (
StorageManager, Message) {
//Init the storageMgr struct
s := &storageMgr{
supvCmdch: supvCmdch,
supvRespch: supvRespch,
snapshotNotifych: snapshotNotifych,
indexSnapMap: make(map[common.IndexInstId]IndexSnapshot),
waitersMap: make(map[common.IndexInstId][]*snapshotWaiter),
config: config,
}
//if manager is not enabled, create meta file
if config["enableManager"].Bool() == false {
fdbconfig := forestdb.DefaultConfig()
kvconfig := forestdb.DefaultKVStoreConfig()
var err error
if s.dbfile, err = forestdb.Open("meta", fdbconfig); err != nil {
return nil, &MsgError{err: Error{cause: err}}
}
// Make use of default kvstore provided by forestdb
if s.meta, err = s.dbfile.OpenKVStore("default", kvconfig); err != nil {
return nil, &MsgError{err: Error{cause: err}}
}
}
s.updateIndexSnapMap(indexPartnMap, common.ALL_STREAMS, "")
//start Storage Manager loop which listens to commands from its supervisor
go s.run()
return s, &MsgSuccess{}
}
//run starts the storage manager loop which listens to messages
//from its supervisor(indexer)
func (s *storageMgr) run() {
//main Storage Manager loop
loop:
for {
select {
case cmd, ok := <-s.supvCmdch:
if ok {
if cmd.GetMsgType() == STORAGE_MGR_SHUTDOWN {
logging.Infof("StorageManager::run Shutting Down")
close(s.snapshotNotifych)
s.supvCmdch <- &MsgSuccess{}
break loop
}
s.handleSupvervisorCommands(cmd)
} else {
//supervisor channel closed. exit
break loop
}
}
}
}
func (s *storageMgr) handleSupvervisorCommands(cmd Message) {
switch cmd.GetMsgType() {
case MUT_MGR_FLUSH_DONE:
s.handleCreateSnapshot(cmd)
case INDEXER_ROLLBACK:
s.handleRollback(cmd)
case UPDATE_INDEX_INSTANCE_MAP:
s.handleUpdateIndexInstMap(cmd)
case UPDATE_INDEX_PARTITION_MAP:
s.handleUpdateIndexPartnMap(cmd)
case STORAGE_INDEX_SNAP_REQUEST:
s.handleGetIndexSnapshot(cmd)
case STORAGE_INDEX_STORAGE_STATS:
s.handleGetIndexStorageStats(cmd)
case STORAGE_INDEX_COMPACT:
s.handleIndexCompaction(cmd)
case STORAGE_STATS:
s.handleStats(cmd)
}
}
//handleCreateSnapshot will create the necessary snapshots
//after flush has completed
func (s *storageMgr) handleCreateSnapshot(cmd Message) {
s.supvCmdch <- &MsgSuccess{}
logging.Tracef("StorageMgr::handleCreateSnapshot %v", cmd)
msgFlushDone := cmd.(*MsgMutMgrFlushDone)
bucket := msgFlushDone.GetBucket()
tsVbuuid := msgFlushDone.GetTS()
streamId := msgFlushDone.GetStreamId()
flushWasAborted := msgFlushDone.GetAborted()
numVbuckets := s.config["numVbuckets"].Int()
snapType := tsVbuuid.GetSnapType()
tsVbuuid.Crc64 = common.HashVbuuid(tsVbuuid.Vbuuids)
if snapType == common.NO_SNAP {
logging.Infof("StorageMgr::handleCreateSnapshot Skip Snapshot For %v "+
"%v SnapType %v", streamId, bucket, snapType)
s.supvRespch <- &MsgMutMgrFlushDone{mType: STORAGE_SNAP_DONE,
streamId: streamId,
bucket: bucket,
ts: tsVbuuid,
aborted: flushWasAborted}
return
}
s.muSnap.Lock()
defer s.muSnap.Unlock()
//pass copy of maps to worker
indexSnapMap := copyIndexSnapMap(s.indexSnapMap)
indexInstMap := common.CopyIndexInstMap(s.indexInstMap)
indexPartnMap := CopyIndexPartnMap(s.indexPartnMap)
stats := s.stats.Get()
go s.createSnapshotWorker(streamId, bucket, tsVbuuid, indexSnapMap,
numVbuckets, indexInstMap, indexPartnMap, stats, flushWasAborted)
}
func (s *storageMgr) createSnapshotWorker(streamId common.StreamId, bucket string,
tsVbuuid *common.TsVbuuid, indexSnapMap IndexSnapMap, numVbuckets int,
indexInstMap common.IndexInstMap, indexPartnMap IndexPartnMap, stats *IndexerStats,
flushWasAborted bool) {
defer destroyIndexSnapMap(indexSnapMap)
var needsCommit bool
var forceCommit bool
snapType := tsVbuuid.GetSnapType()
if snapType == common.DISK_SNAP {
needsCommit = true
} else if snapType == common.FORCE_COMMIT {
forceCommit = true
}
var wg sync.WaitGroup
//for every index managed by this indexer
for idxInstId, partnMap := range indexPartnMap {
// Create snapshots for all indexes in parallel
wg.Add(1)
go func(idxInstId common.IndexInstId, partnMap PartitionInstMap) {
defer wg.Done()
idxInst := indexInstMap[idxInstId]
idxStats := stats.indexes[idxInst.InstId]
lastIndexSnap := indexSnapMap[idxInstId]
//if index belongs to the flushed bucket and stream
if idxInst.Defn.Bucket == bucket &&
idxInst.Stream == streamId &&
idxInst.State != common.INDEX_STATE_DELETED {
// List of snapshots for reading current timestamp
var isSnapCreated bool = true
partnSnaps := make(map[common.PartitionId]PartitionSnapshot)
//for all partitions managed by this indexer
for partnId, partnInst := range partnMap {
var lastPartnSnap PartitionSnapshot
if lastIndexSnap != nil {
lastPartnSnap = lastIndexSnap.Partitions()[partnId]
}
sc := partnInst.Sc
sliceSnaps := make(map[SliceId]SliceSnapshot)
//create snapshot for all the slices
for _, slice := range sc.GetAllSlices() {
if flushWasAborted {
slice.IsDirty()
return
}
var latestSnapshot Snapshot
if lastIndexSnap.Partitions() != nil {
lastSliceSnap := lastPartnSnap.Slices()[slice.Id()]
latestSnapshot = lastSliceSnap.Snapshot()
}
//if flush timestamp is greater than last
//snapshot timestamp, create a new snapshot
snapTs := NewTimestamp(numVbuckets)
if latestSnapshot != nil {
snapTsVbuuid := latestSnapshot.Timestamp()
snapTs = getSeqTsFromTsVbuuid(snapTsVbuuid)
}
ts := getSeqTsFromTsVbuuid(tsVbuuid)
//if the flush TS is greater than the last snapshot TS
//and slice has some changes. Skip only in-memory snapshot
//in case of unchanged data.
if latestSnapshot == nil || (ts.GreaterThan(snapTs) &&
(slice.IsDirty() || needsCommit)) || forceCommit {
newTsVbuuid := tsVbuuid.Copy()
var err error
var info SnapshotInfo
var newSnapshot Snapshot
logging.Tracef("StorageMgr::handleCreateSnapshot Creating New Snapshot "+
"Index: %v PartitionId: %v SliceId: %v Commit: %v Force: %v", idxInstId, partnId, slice.Id(), needsCommit, forceCommit)
if forceCommit {
needsCommit = forceCommit
}
snapCreateStart := time.Now()
if info, err = slice.NewSnapshot(newTsVbuuid, needsCommit); err != nil {
logging.Errorf("handleCreateSnapshot::handleCreateSnapshot Error "+
"Creating new snapshot Slice Index: %v Slice: %v. Skipped. Error %v", idxInstId,
slice.Id(), err)
isSnapCreated = false
common.CrashOnError(err)
continue
}
snapCreateDur := time.Since(snapCreateStart)
idxStats := stats.indexes[idxInstId]
idxStats.numSnapshots.Add(1)
if needsCommit {
idxStats.numCommits.Add(1)
}
snapOpenStart := time.Now()
if newSnapshot, err = slice.OpenSnapshot(info); err != nil {
logging.Errorf("StorageMgr::handleCreateSnapshot Error Creating Snapshot "+
"for Index: %v Slice: %v. Skipped. Error %v", idxInstId,
slice.Id(), err)
isSnapCreated = false
common.CrashOnError(err)
continue
}
snapOpenDur := time.Since(snapOpenStart)
logging.Infof("StorageMgr::handleCreateSnapshot Added New Snapshot Index: %v "+
"PartitionId: %v SliceId: %v Crc64: %v (%v) SnapCreateDur %v SnapOpenDur %v", idxInstId, partnId, slice.Id(), tsVbuuid.Crc64, info, snapCreateDur, snapOpenDur)
ss := &sliceSnapshot{
id: slice.Id(),
snap: newSnapshot,
}
sliceSnaps[slice.Id()] = ss
} else {
// Increment reference
latestSnapshot.Open()
ss := &sliceSnapshot{
id: slice.Id(),
snap: latestSnapshot,
}
sliceSnaps[slice.Id()] = ss
logging.Warnf("StorageMgr::handleCreateSnapshot Skipped Creating New Snapshot for Index %v "+
"PartitionId %v SliceId %v. No New Mutations. IsDirty %v", idxInstId, partnId, slice.Id(), slice.IsDirty())
logging.Debugf("StorageMgr::handleCreateSnapshot SnapTs %v FlushTs %v", snapTs, ts)
continue
}
}
ps := &partitionSnapshot{
id: partnId,
slices: sliceSnaps,
}
partnSnaps[partnId] = ps
}
is := &indexSnapshot{
instId: idxInstId,
ts: tsVbuuid.Copy(),
partns: partnSnaps,
}
if isSnapCreated {
s.updateSnapMapAndNotify(is, idxStats)
} else {
DestroyIndexSnapshot(is)
}
s.updateSnapIntervalStat(idxStats)
}
}(idxInstId, partnMap)
}
wg.Wait()
s.supvRespch <- &MsgMutMgrFlushDone{mType: STORAGE_SNAP_DONE,
streamId: streamId,
bucket: bucket,
ts: tsVbuuid,
aborted: flushWasAborted}
}
func (s *storageMgr) updateSnapIntervalStat(idxStats *IndexStats) {
// Compute avgTsInterval
last := idxStats.lastTsTime.Value()
curr := int64(time.Now().UnixNano())
avg := idxStats.avgTsInterval.Value()
avg = common.ComputeAvg(avg, last, curr)
if avg != 0 {
idxStats.avgTsInterval.Set(avg)
idxStats.sinceLastSnapshot.Set(curr - last)
}
idxStats.lastTsTime.Set(curr)
// Compute avgTsItemsCount
last = idxStats.lastNumFlushQueued.Value()
curr = idxStats.numDocsFlushQueued.Value()
avg = idxStats.avgTsItemsCount.Value()
avg = common.ComputeAvg(avg, last, curr)
idxStats.avgTsItemsCount.Set(avg)
idxStats.lastNumFlushQueued.Set(curr)
}
// Update index-snapshot map whenever a snapshot is created for an index
func (s *storageMgr) updateSnapMapAndNotify(is IndexSnapshot, idxStats *IndexStats) {
s.muSnap.Lock()
defer s.muSnap.Unlock()
DestroyIndexSnapshot(s.indexSnapMap[is.IndexInstId()])
s.indexSnapMap[is.IndexInstId()] = is
// notify a new snapshot through channel
// the channel receiver needs to destroy snapshot when done
s.notifySnapshotCreation(is)
var numReplies int64
t := time.Now()
// Also notify any waiters for snapshots creation
var newWaiters []*snapshotWaiter
waiters := s.waitersMap[is.IndexInstId()]
for _, w := range waiters {
// Clean up expired requests from queue
if !w.expired.IsZero() && t.After(w.expired) {
w.Error(common.ErrScanTimedOut)
idxStats.numSnapshotWaiters.Add(-1)
continue
}
if isSnapshotConsistent(is, w.cons, w.ts) {
w.Notify(CloneIndexSnapshot(is))
numReplies++
idxStats.numSnapshotWaiters.Add(-1)
continue
}
newWaiters = append(newWaiters, w)
}
s.waitersMap[is.IndexInstId()] = newWaiters
idxStats.numLastSnapshotReply.Set(numReplies)
}
//handleRollback will rollback to given timestamp
func (sm *storageMgr) handleRollback(cmd Message) {
streamId := cmd.(*MsgRollback).GetStreamId()
rollbackTs := cmd.(*MsgRollback).GetRollbackTs()
bucket := cmd.(*MsgRollback).GetBucket()
logging.Infof("StorageMgr::handleRollback rollbackTs is %v", rollbackTs)
var respTs *common.TsVbuuid
//for every index managed by this indexer
for idxInstId, partnMap := range sm.indexPartnMap {
idxInst := sm.indexInstMap[idxInstId]
//if this bucket in stream needs to be rolled back
if idxInst.Defn.Bucket == bucket &&
idxInst.Stream == streamId &&
idxInst.State != common.INDEX_STATE_DELETED {
//for all partitions managed by this indexer
for partnId, partnInst := range partnMap {
sc := partnInst.Sc
//rollback all slices
for _, slice := range sc.GetAllSlices() {
infos, err := slice.GetSnapshots()
// TODO: Proper error handling if possible
if err != nil {
panic("Unable read snapinfo -" + err.Error())
}
s := NewSnapshotInfoContainer(infos)
snapInfo := s.GetOlderThanTS(rollbackTs)
if snapInfo != nil {
err := slice.Rollback(snapInfo)
if err == nil {
logging.Infof("StorageMgr::handleRollback Rollback Index: %v "+
"PartitionId: %v SliceId: %v To Snapshot %v ", idxInstId, partnId,
slice.Id(), snapInfo)
respTs = snapInfo.Timestamp()
} else {
//send error response back
//TODO handle the case where some of the slices fail to rollback
sm.supvCmdch <- &MsgError{err: Error{code: ERROR_STORAGE_MGR_ROLLBACK_FAIL,
severity: FATAL,
category: STORAGE_MGR,
cause: err}}
return
}
} else {
//if there is no snapshot available, rollback to zero
err := slice.RollbackToZero()
if err == nil {
logging.Infof("StorageMgr::handleRollback Rollback Index: %v "+
"PartitionId: %v SliceId: %v To Zero ", idxInstId, partnId,
slice.Id())
//once rollback to zero has happened, set response ts to nil
//to represent the initial state of storage
respTs = nil
} else {
//send error response back
//TODO handle the case where some of the slices fail to rollback
sm.supvCmdch <- &MsgError{err: Error{code: ERROR_STORAGE_MGR_ROLLBACK_FAIL,
severity: FATAL,
category: STORAGE_MGR,
cause: err}}
return
}
}
}
}
}
}
go func() {
sm.muSnap.Lock()
defer sm.muSnap.Unlock()
// Notify all scan waiters for indexes in this bucket
// and stream with error
stats := sm.stats.Get()
for idxInstId, waiters := range sm.waitersMap {
idxInst := sm.indexInstMap[idxInstId]
idxStats := stats.indexes[idxInst.InstId]
if idxInst.Defn.Bucket == bucket &&
idxInst.Stream == streamId {
for _, w := range waiters {
w.Error(ErrIndexRollback)
if idxStats != nil {
idxStats.numSnapshotWaiters.Add(-1)
}
}
delete(sm.waitersMap, idxInstId)
}
}
}()
sm.updateIndexSnapMap(sm.indexPartnMap, streamId, bucket)
stats := sm.stats.Get()
if bStats, ok := stats.buckets[bucket]; ok {
bStats.numRollbacks.Add(1)
}
sm.supvCmdch <- &MsgRollback{streamId: streamId,
bucket: bucket,
rollbackTs: respTs}
}
func (s *storageMgr) addNilSnapshot(idxInstId common.IndexInstId, bucket string) {
if _, ok := s.indexSnapMap[idxInstId]; !ok {
ts := common.NewTsVbuuid(bucket, s.config["numVbuckets"].Int())
snap := &indexSnapshot{
instId: idxInstId,
ts: ts, // nil snapshot should have ZERO Crc64 :)
epoch: true,
}
s.indexSnapMap[idxInstId] = snap
s.notifySnapshotCreation(snap)
}
}
func (s *storageMgr) notifySnapshotDeletion(instId common.IndexInstId) {
defer func() {
if r := recover(); r != nil {
logging.Errorf("storageMgr::notifySnapshot %v", r)
}
}()
snap := &indexSnapshot{
instId: instId,
ts: nil, // signal deletion with nil timestamp
}
s.snapshotNotifych <- snap
}
func (s *storageMgr) notifySnapshotCreation(is IndexSnapshot) {
defer func() {
if r := recover(); r != nil {
logging.Errorf("storageMgr::notifySnapshot %v", r)
}
}()
s.snapshotNotifych <- CloneIndexSnapshot(is)
}
func (s *storageMgr) handleUpdateIndexInstMap(cmd Message) {
logging.Tracef("StorageMgr::handleUpdateIndexInstMap %v", cmd)
req := cmd.(*MsgUpdateInstMap)
indexInstMap := req.GetIndexInstMap()
s.stats.Set(req.GetStatsObject())
s.indexInstMap = common.CopyIndexInstMap(indexInstMap)
s.muSnap.Lock()
defer s.muSnap.Unlock()
// Remove all snapshot waiters for indexes that do not exist anymore
for id, ws := range s.waitersMap {
if inst, ok := s.indexInstMap[id]; !ok ||
inst.State == common.INDEX_STATE_DELETED {
for _, w := range ws {
w.Error(common.ErrIndexNotFound)
}
delete(s.waitersMap, id)
}
}
// Cleanup all invalid index's snapshots
for idxInstId, is := range s.indexSnapMap {
if inst, ok := s.indexInstMap[idxInstId]; !ok ||
inst.State == common.INDEX_STATE_DELETED {
DestroyIndexSnapshot(is)
delete(s.indexSnapMap, idxInstId)
s.notifySnapshotDeletion(idxInstId)
}
}
// Add 0 items index snapshots for newly added indexes
for idxInstId, inst := range s.indexInstMap {
s.addNilSnapshot(idxInstId, inst.Defn.Bucket)
}
//if manager is not enable, store the updated InstMap in
//meta file
if s.config["enableManager"].Bool() == false {
instMap := common.CopyIndexInstMap(s.indexInstMap)
for id, inst := range instMap {
inst.Pc = nil
instMap[id] = inst
}
//store indexInstMap in metadata store
var instBytes bytes.Buffer
var err error
enc := gob.NewEncoder(&instBytes)
err = enc.Encode(instMap)
if err != nil {
logging.Errorf("StorageMgr::handleUpdateIndexInstMap \n\t Error Marshalling "+
"IndexInstMap %v. Err %v", instMap, err)
}
if err = s.meta.SetKV([]byte(INST_MAP_KEY_NAME), instBytes.Bytes()); err != nil {
logging.Errorf("StorageMgr::handleUpdateIndexInstMap \n\tError "+
"Storing IndexInstMap %v", err)
}
s.dbfile.Commit(forestdb.COMMIT_MANUAL_WAL_FLUSH)
}
s.supvCmdch <- &MsgSuccess{}
}
func (s *storageMgr) handleUpdateIndexPartnMap(cmd Message) {
logging.Tracef("StorageMgr::handleUpdateIndexPartnMap %v", cmd)
indexPartnMap := cmd.(*MsgUpdatePartnMap).GetIndexPartnMap()
s.indexPartnMap = CopyIndexPartnMap(indexPartnMap)
s.supvCmdch <- &MsgSuccess{}
}
// Process req for providing an index snapshot for index scan.
// The request contains atleast-timestamp and the storage
// manager will reply with a index snapshot soon after a
// snapshot meeting requested criteria is available.
// The requester will block wait until the response is
// available.
func (s *storageMgr) handleGetIndexSnapshot(cmd Message) {
s.supvCmdch <- &MsgSuccess{}
req := cmd.(*MsgIndexSnapRequest)
inst, found := s.indexInstMap[req.GetIndexId()]
if !found || inst.State == common.INDEX_STATE_DELETED {
req.respch <- common.ErrIndexNotFound
return
}
stats := s.stats.Get()
idxStats := stats.indexes[req.GetIndexId()]
s.muSnap.Lock()
defer s.muSnap.Unlock()
// Return snapshot immediately if a matching snapshot exists already
// Otherwise add into waiters list so that next snapshot creation event
// can notify the requester when a snapshot with matching timestamp
// is available.
is := s.indexSnapMap[req.GetIndexId()]
if is != nil && isSnapshotConsistent(is, req.GetConsistency(), req.GetTS()) {
req.respch <- CloneIndexSnapshot(is)
return
}
if idxStats != nil {
idxStats.numSnapshotWaiters.Add(1)
}
w := newSnapshotWaiter(
req.GetIndexId(), req.GetTS(), req.GetConsistency(),
req.GetReplyChannel(), req.GetExpiredTime())
if ws, ok := s.waitersMap[req.GetIndexId()]; ok {
s.waitersMap[req.idxInstId] = append(ws, w)
} else {
s.waitersMap[req.idxInstId] = []*snapshotWaiter{w}
}
}
func (s *storageMgr) handleGetIndexStorageStats(cmd Message) {
s.supvCmdch <- &MsgSuccess{}
req := cmd.(*MsgIndexStorageStats)
replych := req.GetReplyChannel()
stats := s.getIndexStorageStats()
replych <- stats
}
func (s *storageMgr) handleStats(cmd Message) {
s.supvCmdch <- &MsgSuccess{}
req := cmd.(*MsgStatsRequest)
replych := req.GetReplyChannel()
storageStats := s.getIndexStorageStats()
stats := s.stats.Get()
for _, st := range storageStats {
inst := s.indexInstMap[st.InstId]
if inst.State == common.INDEX_STATE_DELETED {
continue
}
idxStats := stats.indexes[st.InstId]
// TODO(sarath): Investigate the reason for inconsistent stats map
// This nil check is a workaround to avoid indexer crashes for now.
if idxStats != nil {
idxStats.diskSize.Set(st.Stats.DiskSize)
idxStats.dataSize.Set(st.Stats.DataSize)
if common.GetStorageMode() != common.MOI {
idxStats.fragPercent.Set(int64(st.GetFragmentation()))
}
idxStats.getBytes.Set(st.Stats.GetBytes)
idxStats.insertBytes.Set(st.Stats.InsertBytes)
idxStats.deleteBytes.Set(st.Stats.DeleteBytes)
// compute mutation rate
now := time.Now().UnixNano()
elapsed := float64(now-idxStats.lastMutateGatherTime.Value()) / float64(time.Second)
if elapsed > 0 {
numDocsIndexed := idxStats.numDocsIndexed.Value()
mutationRate := float64(numDocsIndexed-idxStats.lastNumDocsIndexed.Value()) / elapsed
idxStats.avgMutationRate.Set(int64((mutationRate + float64(idxStats.avgMutationRate.Value())) / 2))
idxStats.lastNumDocsIndexed.Set(numDocsIndexed)
numItemsFlushed := idxStats.numItemsFlushed.Value()
drainRate := float64(numItemsFlushed-idxStats.lastNumItemsFlushed.Value()) / elapsed
idxStats.avgDrainRate.Set(int64((drainRate + float64(idxStats.avgDrainRate.Value())) / 2))
idxStats.lastNumItemsFlushed.Set(numItemsFlushed)
idxStats.lastMutateGatherTime.Set(now)
}
}
}
replych <- true
}
func (s *storageMgr) getIndexStorageStats() []IndexStorageStats {
var stats []IndexStorageStats
var err error
var sts StorageStatistics
for idxInstId, partnMap := range s.indexPartnMap {
inst, ok := s.indexInstMap[idxInstId]
//skip deleted indexes
if !ok || inst.State == common.INDEX_STATE_DELETED {
continue
}
var internalData []string
var dataSz, diskSz, extraSnapDataSize int64
var getBytes, insertBytes, deleteBytes int64
var nslices int64
var needUpgrade = false
loop:
for _, partnInst := range partnMap {
slices := partnInst.Sc.GetAllSlices()
nslices += int64(len(slices))
for _, slice := range slices {
sts, err = slice.Statistics()
if err != nil {
break loop
}
dataSz += sts.DataSize
diskSz += sts.DiskSize
getBytes += sts.GetBytes
insertBytes += sts.InsertBytes
deleteBytes += sts.DeleteBytes
extraSnapDataSize += sts.ExtraSnapDataSize
internalData = append(internalData, sts.InternalData...)
needUpgrade = needUpgrade || sts.NeedUpgrade
}
}
if err == nil {
stat := IndexStorageStats{
InstId: idxInstId,
Name: inst.Defn.Name,
Bucket: inst.Defn.Bucket,
Stats: StorageStatistics{
DataSize: dataSz,
DiskSize: diskSz,
GetBytes: getBytes,
InsertBytes: insertBytes,
DeleteBytes: deleteBytes,
ExtraSnapDataSize: extraSnapDataSize,
NeedUpgrade: needUpgrade,
InternalData: internalData,
},
}
stats = append(stats, stat)
}
}
return stats
}
func (s *storageMgr) handleIndexCompaction(cmd Message) {
s.supvCmdch <- &MsgSuccess{}
req := cmd.(*MsgIndexCompact)
errch := req.GetErrorChannel()
abortTime := req.GetAbortTime()
var slices []Slice
inst, ok := s.indexInstMap[req.GetInstId()]
stats := s.stats.Get()
if !ok || inst.State == common.INDEX_STATE_DELETED {
errch <- common.ErrIndexNotFound
return
}
partnMap, _ := s.indexPartnMap[req.GetInstId()]
idxStats := stats.indexes[req.GetInstId()]
idxStats.numCompactions.Add(1)
// Increment rc for slices
for _, partnInst := range partnMap {
for _, slice := range partnInst.Sc.GetAllSlices() {
slice.IncrRef()
slices = append(slices, slice)
}
}
// Perform file compaction without blocking storage manager main loop
go func() {
for _, slice := range slices {
err := slice.Compact(abortTime)
slice.DecrRef()
if err != nil {
errch <- err
return
}
}
errch <- nil
}()
}
// Update index-snapshot map using index partition map
// This function should be called only during initialization
// of storage manager and during rollback.
// FIXME: Current implementation makes major assumption that
// single slice is supported.
func (s *storageMgr) updateIndexSnapMap(indexPartnMap IndexPartnMap,
streamId common.StreamId, bucket string) {
s.muSnap.Lock()
defer s.muSnap.Unlock()
var tsVbuuid *common.TsVbuuid
for idxInstId, partnMap := range indexPartnMap {
//if bucket and stream have been provided
if bucket != "" && streamId != common.ALL_STREAMS {
idxInst := s.indexInstMap[idxInstId]
//skip the index if either bucket or stream don't match
if idxInst.Defn.Bucket != bucket || idxInst.Stream != streamId {
continue
}
//skip deleted indexes
if idxInst.State == common.INDEX_STATE_DELETED {
continue
}
}
//there is only one partition for now
partnInst := partnMap[0]
sc := partnInst.Sc
//there is only one slice for now
slice := sc.GetSliceById(0)
infos, err := slice.GetSnapshots()
// TODO: Proper error handling if possible
if err != nil {
panic("Unable to read snapinfo -" + err.Error())
}
DestroyIndexSnapshot(s.indexSnapMap[idxInstId])
delete(s.indexSnapMap, idxInstId)
s.notifySnapshotDeletion(idxInstId)
snapInfoContainer := NewSnapshotInfoContainer(infos)
latestSnapshotInfo := snapInfoContainer.GetLatest()
if latestSnapshotInfo != nil {
logging.Infof("StorageMgr::updateIndexSnapMap IndexInst:%v Attempting to open snapshot (%v)",
idxInstId, latestSnapshotInfo)
latestSnapshot, err := slice.OpenSnapshot(latestSnapshotInfo)
if err != nil {
panic("Unable to open snapshot -" + err.Error())
}
ss := &sliceSnapshot{
id: SliceId(0),
snap: latestSnapshot,
}
tsVbuuid = latestSnapshotInfo.Timestamp()
sid := SliceId(0)
pid := common.PartitionId(0)
ps := &partitionSnapshot{
id: pid,
slices: map[SliceId]SliceSnapshot{sid: ss},
}
is := &indexSnapshot{
instId: idxInstId,
ts: tsVbuuid,
partns: map[common.PartitionId]PartitionSnapshot{pid: ps},
}
s.indexSnapMap[idxInstId] = is
s.notifySnapshotCreation(is)
} else {
s.addNilSnapshot(idxInstId, bucket)
}
}
}
func copyIndexSnapMap(inMap IndexSnapMap) IndexSnapMap {
outMap := make(IndexSnapMap)
for k, v := range inMap {
outMap[k] = CloneIndexSnapshot(v)
}
return outMap
}
func destroyIndexSnapMap(ism IndexSnapMap) {
for _, v := range ism {