-
Notifications
You must be signed in to change notification settings - Fork 451
/
server.go
1260 lines (1113 loc) · 40.8 KB
/
server.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) 2017 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 server
import (
"errors"
"fmt"
"io"
"math"
"net/http"
"os"
"os/signal"
"runtime"
"runtime/debug"
"syscall"
"time"
clusterclient "github.com/m3db/m3/src/cluster/client"
"github.com/m3db/m3/src/cluster/client/etcd"
"github.com/m3db/m3/src/cluster/generated/proto/commonpb"
"github.com/m3db/m3/src/cluster/kv"
"github.com/m3db/m3/src/cluster/kv/util"
"github.com/m3db/m3/src/cmd/services/m3dbnode/config"
"github.com/m3db/m3/src/dbnode/client"
"github.com/m3db/m3/src/dbnode/encoding"
"github.com/m3db/m3/src/dbnode/encoding/m3tsz"
"github.com/m3db/m3/src/dbnode/environment"
"github.com/m3db/m3/src/dbnode/kvconfig"
hjcluster "github.com/m3db/m3/src/dbnode/network/server/httpjson/cluster"
hjnode "github.com/m3db/m3/src/dbnode/network/server/httpjson/node"
"github.com/m3db/m3/src/dbnode/network/server/tchannelthrift"
ttcluster "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift/cluster"
ttnode "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift/node"
"github.com/m3db/m3/src/dbnode/persist/fs"
"github.com/m3db/m3/src/dbnode/persist/fs/commitlog"
"github.com/m3db/m3/src/dbnode/ratelimit"
"github.com/m3db/m3/src/dbnode/retention"
m3dbruntime "github.com/m3db/m3/src/dbnode/runtime"
"github.com/m3db/m3/src/dbnode/storage"
"github.com/m3db/m3/src/dbnode/storage/block"
"github.com/m3db/m3/src/dbnode/storage/cluster"
"github.com/m3db/m3/src/dbnode/storage/index"
"github.com/m3db/m3/src/dbnode/storage/namespace"
"github.com/m3db/m3/src/dbnode/storage/repair"
"github.com/m3db/m3/src/dbnode/storage/series"
"github.com/m3db/m3/src/dbnode/topology"
"github.com/m3db/m3/src/dbnode/ts"
"github.com/m3db/m3/src/dbnode/x/tchannel"
"github.com/m3db/m3/src/dbnode/x/xio"
"github.com/m3db/m3/src/m3ninx/postings"
"github.com/m3db/m3/src/m3ninx/postings/roaring"
xdocs "github.com/m3db/m3/src/x/docs"
"github.com/m3db/m3/src/x/mmap"
"github.com/m3db/m3/src/x/serialize"
xconfig "github.com/m3db/m3x/config"
"github.com/m3db/m3x/context"
"github.com/m3db/m3x/ident"
"github.com/m3db/m3x/instrument"
xlog "github.com/m3db/m3x/log"
"github.com/m3db/m3x/pool"
xsync "github.com/m3db/m3x/sync"
"github.com/coreos/etcd/embed"
"github.com/coreos/pkg/capnslog"
"github.com/uber-go/tally"
)
const (
bootstrapConfigInitTimeout = 10 * time.Second
serverGracefulCloseTimeout = 10 * time.Second
bgProcessLimitInterval = 10 * time.Second
maxBgProcessLimitMonitorDuration = 5 * time.Minute
)
// RunOptions provides options for running the server
// with backwards compatibility if only solely adding fields.
type RunOptions struct {
// ConfigFile is the YAML configuration file to use to run the server.
ConfigFile string
// Config is an alternate way to provide configuration and will be used
// instead of parsing ConfigFile if ConfigFile is not specified.
Config config.DBConfiguration
// BootstrapCh is a channel to listen on to be notified of bootstrap.
BootstrapCh chan<- struct{}
// EmbeddedKVCh is a channel to listen on to be notified that the embedded KV has bootstrapped.
EmbeddedKVCh chan<- struct{}
// ClientCh is a channel to listen on to share the same m3db client that this server uses.
ClientCh chan<- client.Client
// ClusterClientCh is a channel to listen on to share the same m3 cluster client that this server uses.
ClusterClientCh chan<- clusterclient.Client
// InterruptCh is a programmatic interrupt channel to supply to
// interrupt and shutdown the server.
InterruptCh <-chan error
}
// Run runs the server programmatically given a filename for the
// configuration file.
func Run(runOpts RunOptions) {
var cfg config.DBConfiguration
if runOpts.ConfigFile != "" {
var rootCfg config.Configuration
if err := xconfig.LoadFile(&rootCfg, runOpts.ConfigFile, xconfig.Options{}); err != nil {
fmt.Fprintf(os.Stderr, "unable to load %s: %v", runOpts.ConfigFile, err)
os.Exit(1)
}
cfg = *rootCfg.DB
} else {
cfg = runOpts.Config
}
logger, err := cfg.Logging.BuildLogger()
if err != nil {
fmt.Fprintf(os.Stderr, "unable to create logger: %v", err)
os.Exit(1)
}
go bgValidateProcessLimits(logger)
debug.SetGCPercent(cfg.GCPercentage)
scope, _, err := cfg.Metrics.NewRootScope()
if err != nil {
logger.Fatalf("could not connect to metrics: %v", err)
}
hostID, err := cfg.HostID.Resolve()
if err != nil {
logger.Fatalf("could not resolve local host ID: %v", err)
}
capnslog.SetGlobalLogLevel(capnslog.WARNING)
// Presence of KV server config indicates embedded etcd cluster
if cfg.EnvironmentConfig.SeedNodes == nil {
logger.Info("no seed nodes set, using dedicated etcd cluster")
} else {
// Default etcd client clusters if not set already
clusters := cfg.EnvironmentConfig.Service.ETCDClusters
seedNodes := cfg.EnvironmentConfig.SeedNodes.InitialCluster
if len(clusters) == 0 {
endpoints, err := config.InitialClusterEndpoints(seedNodes)
if err != nil {
logger.Fatalf("unable to create etcd clusters: %v", err)
}
zone := cfg.EnvironmentConfig.Service.Zone
logger.Infof("using seed nodes etcd cluster: zone=%s, endpoints=%v", zone, endpoints)
cfg.EnvironmentConfig.Service.ETCDClusters = []etcd.ClusterConfig{etcd.ClusterConfig{
Zone: zone,
Endpoints: endpoints,
}}
}
seedNodeHostIDs := make([]string, 0, len(seedNodes))
for _, entry := range seedNodes {
seedNodeHostIDs = append(seedNodeHostIDs, entry.HostID)
}
logger.WithFields(
xlog.NewField("hostID", hostID),
xlog.NewField("seedNodeHostIDs", fmt.Sprintf("%v", seedNodeHostIDs)),
).Info("resolving seed node configuration")
if !config.IsSeedNode(seedNodes, hostID) {
logger.Info("not a seed node, using cluster seed nodes")
} else {
logger.Info("seed node, starting etcd server")
etcdCfg, err := config.NewEtcdEmbedConfig(cfg)
if err != nil {
logger.Fatalf("unable to create etcd config: %v", err)
}
e, err := embed.StartEtcd(etcdCfg)
if err != nil {
logger.Fatalf("could not start embedded etcd: %v", err)
}
if runOpts.EmbeddedKVCh != nil {
// Notify on embedded KV bootstrap chan if specified
runOpts.EmbeddedKVCh <- struct{}{}
}
defer e.Close()
}
}
opts := storage.NewOptions()
iopts := opts.InstrumentOptions().
SetLogger(logger).
SetMetricsScope(scope).
SetMetricsSamplingRate(cfg.Metrics.SampleRate())
opts = opts.SetInstrumentOptions(iopts)
if cfg.Index.MaxQueryIDsConcurrency != 0 {
queryIDsWorkerPool := xsync.NewWorkerPool(cfg.Index.MaxQueryIDsConcurrency)
queryIDsWorkerPool.Init()
opts = opts.SetQueryIDsWorkerPool(queryIDsWorkerPool)
} else {
logger.Warnf("max index query IDs concurrency was not set, falling back to default value")
}
buildReporter := instrument.NewBuildReporter(iopts)
if err := buildReporter.Start(); err != nil {
logger.Fatalf("unable to start build reporter: %v", err)
}
defer buildReporter.Stop()
runtimeOpts := m3dbruntime.NewOptions().
SetPersistRateLimitOptions(ratelimit.NewOptions().
SetLimitEnabled(true).
SetLimitMbps(cfg.Filesystem.ThroughputLimitMbps).
SetLimitCheckEvery(cfg.Filesystem.ThroughputCheckEvery)).
SetWriteNewSeriesAsync(cfg.WriteNewSeriesAsync).
SetWriteNewSeriesBackoffDuration(cfg.WriteNewSeriesBackoffDuration)
if lruCfg := cfg.Cache.SeriesConfiguration().LRU; lruCfg != nil {
runtimeOpts = runtimeOpts.SetMaxWiredBlocks(lruCfg.MaxBlocks)
}
// FOLLOWUP(prateek): remove this once we have the runtime options<->index wiring done
indexOpts := opts.IndexOptions()
insertMode := index.InsertSync
if cfg.WriteNewSeriesAsync {
insertMode = index.InsertAsync
}
opts = opts.SetIndexOptions(
indexOpts.SetInsertMode(insertMode))
if tick := cfg.Tick; tick != nil {
runtimeOpts = runtimeOpts.
SetTickSeriesBatchSize(tick.SeriesBatchSize).
SetTickPerSeriesSleepDuration(tick.PerSeriesSleepDuration).
SetTickMinimumInterval(tick.MinimumInterval)
}
runtimeOptsMgr := m3dbruntime.NewOptionsManager()
if err := runtimeOptsMgr.Update(runtimeOpts); err != nil {
logger.Fatalf("could not set initial runtime options: %v", err)
}
defer runtimeOptsMgr.Close()
opts = opts.SetRuntimeOptionsManager(runtimeOptsMgr)
newFileMode, err := cfg.Filesystem.ParseNewFileMode()
if err != nil {
logger.Fatalf("could not parse new file mode: %v", err)
}
newDirectoryMode, err := cfg.Filesystem.ParseNewDirectoryMode()
if err != nil {
logger.Fatalf("could not parse new directory mode: %v", err)
}
mmapCfg := cfg.Filesystem.MmapConfiguration()
shouldUseHugeTLB := mmapCfg.HugeTLB.Enabled
if shouldUseHugeTLB {
// Make sure the host supports HugeTLB before proceeding with it to prevent
// excessive log spam.
shouldUseHugeTLB, err = hostSupportsHugeTLB()
if err != nil {
logger.Fatalf("could not determine if host supports HugeTLB: %v", err)
}
if !shouldUseHugeTLB {
logger.Warnf("host doesn't support HugeTLB, proceeding without it")
}
}
policy := cfg.PoolingPolicy
tagEncoderPool := serialize.NewTagEncoderPool(
serialize.NewTagEncoderOptions(),
poolOptions(policy.TagEncoderPool, scope.SubScope("tag-encoder-pool")))
tagEncoderPool.Init()
tagDecoderPool := serialize.NewTagDecoderPool(
serialize.NewTagDecoderOptions(),
poolOptions(policy.TagDecoderPool, scope.SubScope("tag-decoder-pool")))
tagDecoderPool.Init()
fsopts := fs.NewOptions().
SetClockOptions(opts.ClockOptions()).
SetInstrumentOptions(opts.InstrumentOptions().
SetMetricsScope(scope.SubScope("database.fs"))).
SetFilePathPrefix(cfg.Filesystem.FilePathPrefix).
SetNewFileMode(newFileMode).
SetNewDirectoryMode(newDirectoryMode).
SetWriterBufferSize(cfg.Filesystem.WriteBufferSize).
SetDataReaderBufferSize(cfg.Filesystem.DataReadBufferSize).
SetInfoReaderBufferSize(cfg.Filesystem.InfoReadBufferSize).
SetSeekReaderBufferSize(cfg.Filesystem.SeekReadBufferSize).
SetMmapEnableHugeTLB(shouldUseHugeTLB).
SetMmapHugeTLBThreshold(mmapCfg.HugeTLB.Threshold).
SetRuntimeOptionsManager(runtimeOptsMgr).
SetTagEncoderPool(tagEncoderPool).
SetTagDecoderPool(tagDecoderPool).
SetForceIndexSummariesMmapMemory(cfg.Filesystem.ForceIndexSummariesMmapMemory).
SetForceBloomFilterMmapMemory(cfg.Filesystem.ForceBloomFilterMmapMemory)
var commitLogQueueSize int
specified := cfg.CommitLog.Queue.Size
switch cfg.CommitLog.Queue.CalculationType {
case config.CalculationTypeFixed:
commitLogQueueSize = specified
case config.CalculationTypePerCPU:
commitLogQueueSize = specified * runtime.NumCPU()
default:
logger.Fatalf("unknown commit log queue size type: %v",
cfg.CommitLog.Queue.CalculationType)
}
var commitLogQueueChannelSize int
if cfg.CommitLog.QueueChannel != nil {
specified := cfg.CommitLog.QueueChannel.Size
switch cfg.CommitLog.Queue.CalculationType {
case config.CalculationTypeFixed:
commitLogQueueChannelSize = specified
case config.CalculationTypePerCPU:
commitLogQueueChannelSize = specified * runtime.NumCPU()
default:
logger.Fatalf("unknown commit log queue channel size type: %v",
cfg.CommitLog.Queue.CalculationType)
}
} else {
commitLogQueueChannelSize = int(float64(commitLogQueueSize) / commitlog.MaximumQueueSizeQueueChannelSizeRatio)
}
opts = opts.SetCommitLogOptions(opts.CommitLogOptions().
SetInstrumentOptions(opts.InstrumentOptions()).
SetFilesystemOptions(fsopts).
SetStrategy(commitlog.StrategyWriteBehind).
SetFlushSize(cfg.CommitLog.FlushMaxBytes).
SetFlushInterval(cfg.CommitLog.FlushEvery).
SetBacklogQueueSize(commitLogQueueSize).
SetBacklogQueueChannelSize(commitLogQueueChannelSize).
SetBlockSize(cfg.CommitLog.BlockSize))
// Set the series cache policy
seriesCachePolicy := cfg.Cache.SeriesConfiguration().Policy
opts = opts.SetSeriesCachePolicy(seriesCachePolicy)
// Apply pooling options
opts = withEncodingAndPoolingOptions(cfg, logger, opts, cfg.PoolingPolicy)
// Setup the block retriever
switch seriesCachePolicy {
case series.CacheAll:
// No options needed to be set
default:
// All other caching strategies require retrieving series from disk
// to service a cache miss
retrieverOpts := fs.NewBlockRetrieverOptions().
SetBytesPool(opts.BytesPool()).
SetSegmentReaderPool(opts.SegmentReaderPool()).
SetIdentifierPool(opts.IdentifierPool())
if blockRetrieveCfg := cfg.BlockRetrieve; blockRetrieveCfg != nil {
retrieverOpts = retrieverOpts.
SetFetchConcurrency(blockRetrieveCfg.FetchConcurrency)
}
blockRetrieverMgr := block.NewDatabaseBlockRetrieverManager(
func(md namespace.Metadata) (block.DatabaseBlockRetriever, error) {
retriever := fs.NewBlockRetriever(retrieverOpts, fsopts)
if err := retriever.Open(md); err != nil {
return nil, err
}
return retriever, nil
})
opts = opts.SetDatabaseBlockRetrieverManager(blockRetrieverMgr)
}
// Set the persistence manager
pm, err := fs.NewPersistManager(fsopts)
if err != nil {
logger.Fatalf("could not create persist manager: %v", err)
}
opts = opts.SetPersistManager(pm)
var (
envCfg environment.ConfigureResults
)
if cfg.EnvironmentConfig.Static == nil {
logger.Info("creating dynamic config service client with m3cluster")
envCfg, err = cfg.EnvironmentConfig.Configure(environment.ConfigurationParameters{
InstrumentOpts: iopts,
HashingSeed: cfg.Hashing.Seed,
})
if err != nil {
logger.Fatalf("could not initialize dynamic config: %v", err)
}
} else {
logger.Info("creating static config service client with m3cluster")
envCfg, err = cfg.EnvironmentConfig.Configure(environment.ConfigurationParameters{
InstrumentOpts: iopts,
HostID: hostID,
})
if err != nil {
logger.Fatalf("could not initialize static config: %v", err)
}
}
if runOpts.ClusterClientCh != nil {
runOpts.ClusterClientCh <- envCfg.ClusterClient
}
opts = opts.SetNamespaceInitializer(envCfg.NamespaceInitializer)
topo, err := envCfg.TopologyInitializer.Init()
if err != nil {
logger.Fatalf("could not initialize m3db topology: %v", err)
}
origin := topology.NewHost(hostID, "")
m3dbClient, err := cfg.Client.NewAdminClient(
client.ConfigurationParameters{
InstrumentOptions: iopts.
SetMetricsScope(iopts.MetricsScope().SubScope("m3dbclient")),
TopologyInitializer: envCfg.TopologyInitializer,
},
func(opts client.AdminOptions) client.AdminOptions {
return opts.SetRuntimeOptionsManager(runtimeOptsMgr).(client.AdminOptions)
},
func(opts client.AdminOptions) client.AdminOptions {
return opts.SetContextPool(opts.ContextPool()).(client.AdminOptions)
},
func(opts client.AdminOptions) client.AdminOptions {
return opts.SetOrigin(origin)
})
if err != nil {
logger.Fatalf("could not create m3db client: %v", err)
}
if runOpts.ClientCh != nil {
runOpts.ClientCh <- m3dbClient
}
// Kick off runtime options manager KV watches
clientAdminOpts := m3dbClient.Options().(client.AdminOptions)
kvWatchClientConsistencyLevels(envCfg.KVStore, logger,
clientAdminOpts, runtimeOptsMgr)
// Set repair options
hostBlockMetadataSlicePool := repair.NewHostBlockMetadataSlicePool(
capacityPoolOptions(policy.HostBlockMetadataSlicePool,
scope.SubScope("host-block-metadata-slice-pool")),
policy.HostBlockMetadataSlicePool.Capacity)
opts = opts.
SetRepairEnabled(cfg.Repair.Enabled).
SetRepairOptions(opts.RepairOptions().
SetAdminClient(m3dbClient).
SetRepairInterval(cfg.Repair.Interval).
SetRepairTimeOffset(cfg.Repair.Offset).
SetRepairTimeJitter(cfg.Repair.Jitter).
SetRepairThrottle(cfg.Repair.Throttle).
SetRepairCheckInterval(cfg.Repair.CheckInterval).
SetHostBlockMetadataSlicePool(hostBlockMetadataSlicePool))
// Set tchannelthrift options
ttopts := tchannelthrift.NewOptions().
SetInstrumentOptions(opts.InstrumentOptions()).
SetTagEncoderPool(tagEncoderPool).
SetTagDecoderPool(tagDecoderPool)
// Set bootstrap options - We need to create a topology map provider from the
// same topology that will be passed to the cluster so that when we make
// bootstrapping decisions they are in sync with the clustered database
// which is triggering the actual bootstraps. This way, when the clustered
// database receives a topology update and decides to kick off a bootstrap,
// the bootstrap process will receaive a topology map that is at least as
// recent as the one that triggered the bootstrap, if not newer.
// See GitHub issue #1013 for more details.
topoMapProvider := newTopoMapProvider(topo)
bs, err := cfg.Bootstrap.New(opts, topoMapProvider, origin, m3dbClient)
if err != nil {
logger.Fatalf("could not create bootstrap process: %v", err)
}
opts = opts.SetBootstrapProcessProvider(bs)
timeout := bootstrapConfigInitTimeout
kvWatchBootstrappers(envCfg.KVStore, logger, timeout, cfg.Bootstrap.Bootstrappers,
func(bootstrappers []string) {
if len(bootstrappers) == 0 {
logger.Errorf("updated bootstrapper list is empty")
return
}
cfg.Bootstrap.Bootstrappers = bootstrappers
updated, err := cfg.Bootstrap.New(opts, topoMapProvider, origin, m3dbClient)
if err != nil {
logger.Errorf("updated bootstrapper list failed: %v", err)
return
}
bs.SetBootstrapperProvider(updated.BootstrapperProvider())
})
// Initialize clustered database
clusterTopoWatch, err := topo.Watch()
if err != nil {
logger.Fatalf("could not create cluster topology watch: %v", err)
}
db, err := cluster.NewDatabase(hostID, topo, clusterTopoWatch, opts)
if err != nil {
logger.Fatalf("could not construct database: %v", err)
}
if err := db.Open(); err != nil {
logger.Fatalf("could not open database: %v", err)
}
contextPool := opts.ContextPool()
tchannelOpts := xtchannel.NewDefaultChannelOptions()
tchannelthriftNodeClose, err := ttnode.NewServer(db,
cfg.ListenAddress, contextPool, tchannelOpts, ttopts).ListenAndServe()
if err != nil {
logger.Fatalf("could not open tchannelthrift interface on %s: %v",
cfg.ListenAddress, err)
}
defer tchannelthriftNodeClose()
logger.Infof("node tchannelthrift: listening on %v", cfg.ListenAddress)
tchannelthriftClusterClose, err := ttcluster.NewServer(m3dbClient,
cfg.ClusterListenAddress, contextPool, tchannelOpts).ListenAndServe()
if err != nil {
logger.Fatalf("could not open tchannelthrift interface on %s: %v",
cfg.ClusterListenAddress, err)
}
defer tchannelthriftClusterClose()
logger.Infof("cluster tchannelthrift: listening on %v", cfg.ClusterListenAddress)
httpjsonNodeClose, err := hjnode.NewServer(db,
cfg.HTTPNodeListenAddress, contextPool, nil, ttopts).ListenAndServe()
if err != nil {
logger.Fatalf("could not open httpjson interface on %s: %v",
cfg.HTTPNodeListenAddress, err)
}
defer httpjsonNodeClose()
logger.Infof("node httpjson: listening on %v", cfg.HTTPNodeListenAddress)
httpjsonClusterClose, err := hjcluster.NewServer(m3dbClient,
cfg.HTTPClusterListenAddress, contextPool, nil).ListenAndServe()
if err != nil {
logger.Fatalf("could not open httpjson interface on %s: %v",
cfg.HTTPClusterListenAddress, err)
}
defer httpjsonClusterClose()
logger.Infof("cluster httpjson: listening on %v", cfg.HTTPClusterListenAddress)
if cfg.DebugListenAddress != "" {
go func() {
if err := http.ListenAndServe(cfg.DebugListenAddress, nil); err != nil {
logger.Errorf("debug server could not listen on %s: %v", cfg.DebugListenAddress, err)
}
}()
}
go func() {
if runOpts.BootstrapCh != nil {
// Notify on bootstrap chan if specified
defer func() {
runOpts.BootstrapCh <- struct{}{}
}()
}
// Bootstrap asynchronously so we can handle interrupt
if err := db.Bootstrap(); err != nil {
logger.Fatalf("could not bootstrap database: %v", err)
}
logger.Infof("bootstrapped")
// Only set the write new series limit after bootstrapping
kvWatchNewSeriesLimitPerShard(envCfg.KVStore, logger, topo,
runtimeOptsMgr, cfg.WriteNewSeriesLimitPerSecond)
}()
// Handle interrupt
interruptCh := runOpts.InterruptCh
if interruptCh == nil {
// Make a noop chan so we can always select
interruptCh = make(chan error)
}
var interruptErr error
select {
case err := <-interruptCh:
interruptErr = err
case sig := <-interrupt():
interruptErr = fmt.Errorf("%v", sig)
}
logger.Warnf("interrupt: %v", interruptErr)
// Attempt graceful server close
closedCh := make(chan struct{})
go func() {
err := db.Terminate()
if err != nil {
logger.Errorf("close database error: %v", err)
}
closedCh <- struct{}{}
}()
// Wait then close or hard close
closeTimeout := serverGracefulCloseTimeout
select {
case <-closedCh:
logger.Infof("server closed")
case <-time.After(closeTimeout):
logger.Errorf("server closed after %s timeout", closeTimeout.String())
}
}
func interrupt() <-chan os.Signal {
c := make(chan os.Signal)
signal.Notify(c, syscall.SIGINT, syscall.SIGTERM)
return c
}
func bgValidateProcessLimits(logger xlog.Logger) {
// If unable to validate process limits on the current configuration,
// do not run background validator task.
if canValidate, message := canValidateProcessLimits(); !canValidate {
logger.Warnf(`cannot validate process limits: invalid configuration found [%v]`, message)
return
}
start := time.Now()
t := time.NewTicker(bgProcessLimitInterval)
defer t.Stop()
for {
// only monitor for first `maxBgProcessLimitMonitorDuration` of process lifetime
if time.Since(start) > maxBgProcessLimitMonitorDuration {
return
}
err := validateProcessLimits()
if err == nil {
return
}
logger.WithFields(
xlog.NewField("url", xdocs.Path("operational_guide/kernel_configuration")),
).Warnf(`invalid configuration found [%v], refer to linked documentation for more information`, err)
<-t.C
}
}
func kvWatchNewSeriesLimitPerShard(
store kv.Store,
logger xlog.Logger,
topo topology.Topology,
runtimeOptsMgr m3dbruntime.OptionsManager,
defaultClusterNewSeriesLimit int,
) {
var initClusterLimit int
value, err := store.Get(kvconfig.ClusterNewSeriesInsertLimitKey)
if err == nil {
protoValue := &commonpb.Int64Proto{}
err = value.Unmarshal(protoValue)
if err == nil {
initClusterLimit = int(protoValue.Value)
}
}
if err != nil {
if err != kv.ErrNotFound {
logger.Warnf("error resolving cluster new series insert limit: %v", err)
}
initClusterLimit = defaultClusterNewSeriesLimit
}
err = setNewSeriesLimitPerShardOnChange(topo, runtimeOptsMgr, initClusterLimit)
if err != nil {
logger.Warnf("unable to set cluster new series insert limit: %v", err)
}
watch, err := store.Watch(kvconfig.ClusterNewSeriesInsertLimitKey)
if err != nil {
logger.Errorf("could not watch cluster new series insert limit: %v", err)
return
}
go func() {
protoValue := &commonpb.Int64Proto{}
for range watch.C() {
value := defaultClusterNewSeriesLimit
if newValue := watch.Get(); newValue != nil {
if err := newValue.Unmarshal(protoValue); err != nil {
logger.Warnf("unable to parse new cluster new series insert limit: %v", err)
continue
}
value = int(protoValue.Value)
}
err = setNewSeriesLimitPerShardOnChange(topo, runtimeOptsMgr, value)
if err != nil {
logger.Warnf("unable to set cluster new series insert limit: %v", err)
continue
}
}
}()
}
func kvWatchClientConsistencyLevels(
store kv.Store,
logger xlog.Logger,
clientOpts client.AdminOptions,
runtimeOptsMgr m3dbruntime.OptionsManager,
) {
setReadConsistencyLevel := func(
v string,
applyFn func(topology.ReadConsistencyLevel, m3dbruntime.Options) m3dbruntime.Options,
) error {
for _, level := range topology.ValidReadConsistencyLevels() {
if level.String() == v {
runtimeOpts := applyFn(level, runtimeOptsMgr.Get())
return runtimeOptsMgr.Update(runtimeOpts)
}
}
return fmt.Errorf("invalid read consistency level set: %s", v)
}
setConsistencyLevel := func(
v string,
applyFn func(topology.ConsistencyLevel, m3dbruntime.Options) m3dbruntime.Options,
) error {
for _, level := range topology.ValidConsistencyLevels() {
if level.String() == v {
runtimeOpts := applyFn(level, runtimeOptsMgr.Get())
return runtimeOptsMgr.Update(runtimeOpts)
}
}
return fmt.Errorf("invalid consistency level set: %s", v)
}
kvWatchStringValue(store, logger,
kvconfig.ClientBootstrapConsistencyLevel,
func(value string) error {
return setReadConsistencyLevel(value,
func(level topology.ReadConsistencyLevel, opts m3dbruntime.Options) m3dbruntime.Options {
return opts.SetClientBootstrapConsistencyLevel(level)
})
},
func() error {
return runtimeOptsMgr.Update(runtimeOptsMgr.Get().
SetClientBootstrapConsistencyLevel(clientOpts.BootstrapConsistencyLevel()))
})
kvWatchStringValue(store, logger,
kvconfig.ClientReadConsistencyLevel,
func(value string) error {
return setReadConsistencyLevel(value,
func(level topology.ReadConsistencyLevel, opts m3dbruntime.Options) m3dbruntime.Options {
return opts.SetClientReadConsistencyLevel(level)
})
},
func() error {
return runtimeOptsMgr.Update(runtimeOptsMgr.Get().
SetClientReadConsistencyLevel(clientOpts.ReadConsistencyLevel()))
})
kvWatchStringValue(store, logger,
kvconfig.ClientWriteConsistencyLevel,
func(value string) error {
return setConsistencyLevel(value,
func(level topology.ConsistencyLevel, opts m3dbruntime.Options) m3dbruntime.Options {
return opts.SetClientWriteConsistencyLevel(level)
})
},
func() error {
return runtimeOptsMgr.Update(runtimeOptsMgr.Get().
SetClientWriteConsistencyLevel(clientOpts.WriteConsistencyLevel()))
})
}
func kvWatchStringValue(
store kv.Store,
logger xlog.Logger,
key string,
onValue func(value string) error,
onDelete func() error,
) {
protoValue := &commonpb.StringProto{}
// First try to eagerly set the value so it doesn't flap if the
// watch returns but not immediately for an existing value
value, err := store.Get(key)
if err != nil && err != kv.ErrNotFound {
logger.Errorf("could not resolve KV key %s: %v", key, err)
}
if err == nil {
if err := value.Unmarshal(protoValue); err != nil {
logger.Errorf("could not unmarshal KV key %s: %v", key, err)
} else if err := onValue(protoValue.Value); err != nil {
logger.Errorf("could not process value of KV key %s: %v", key, err)
} else {
logger.Infof("set KV key %s: %v", key, protoValue.Value)
}
}
watch, err := store.Watch(key)
if err != nil {
logger.Errorf("could not watch KV key %s: %v", key, err)
return
}
go func() {
for range watch.C() {
newValue := watch.Get()
if newValue == nil {
if err := onDelete(); err != nil {
logger.Warnf("could not set default for KV key %s: %v", key, err)
}
continue
}
err := newValue.Unmarshal(protoValue)
if err != nil {
logger.Warnf("could not unmarshal KV key %s: %v", key, err)
continue
}
if err := onValue(protoValue.Value); err != nil {
logger.Warnf("could not process change for KV key %s: %v", key, err)
continue
}
logger.Infof("set KV key %s: %v", key, protoValue.Value)
}
}()
}
func setNewSeriesLimitPerShardOnChange(
topo topology.Topology,
runtimeOptsMgr m3dbruntime.OptionsManager,
clusterLimit int,
) error {
perPlacedShardLimit := clusterLimitToPlacedShardLimit(topo, clusterLimit)
runtimeOpts := runtimeOptsMgr.Get()
if runtimeOpts.WriteNewSeriesLimitPerShardPerSecond() == perPlacedShardLimit {
// Not changed, no need to set the value and trigger a runtime options update
return nil
}
newRuntimeOpts := runtimeOpts.
SetWriteNewSeriesLimitPerShardPerSecond(perPlacedShardLimit)
return runtimeOptsMgr.Update(newRuntimeOpts)
}
func clusterLimitToPlacedShardLimit(topo topology.Topology, clusterLimit int) int {
if clusterLimit < 1 {
return 0
}
topoMap := topo.Get()
numShards := len(topoMap.ShardSet().AllIDs())
numPlacedShards := numShards * topoMap.Replicas()
if numPlacedShards < 1 {
return 0
}
nodeLimit := int(math.Ceil(
float64(clusterLimit) / float64(numPlacedShards)))
return nodeLimit
}
// this function will block for at most waitTimeout to try to get an initial value
// before we kick off the bootstrap
func kvWatchBootstrappers(
kv kv.Store,
logger xlog.Logger,
waitTimeout time.Duration,
defaultBootstrappers []string,
onUpdate func(bootstrappers []string),
) {
vw, err := kv.Watch(kvconfig.BootstrapperKey)
if err != nil {
logger.Fatalf("could not watch value for key with KV: %s",
kvconfig.BootstrapperKey)
}
initializedCh := make(chan struct{})
var initialized bool
go func() {
opts := util.NewOptions().SetLogger(logger)
for range vw.C() {
v, err := util.StringArrayFromValue(vw.Get(),
kvconfig.BootstrapperKey, defaultBootstrappers, opts)
if err != nil {
logger.WithFields(
xlog.NewField("key", kvconfig.BootstrapperKey),
xlog.NewErrField(err),
).Error("error converting KV update to string array")
continue
}
onUpdate(v)
if !initialized {
initialized = true
close(initializedCh)
}
}
}()
select {
case <-time.After(waitTimeout):
case <-initializedCh:
}
}
func withEncodingAndPoolingOptions(
cfg config.DBConfiguration,
logger xlog.Logger,
opts storage.Options,
policy config.PoolingPolicy,
) storage.Options {
iopts := opts.InstrumentOptions()
scope := opts.InstrumentOptions().MetricsScope()
bytesPoolOpts := pool.NewObjectPoolOptions().
SetInstrumentOptions(iopts.SetMetricsScope(scope.SubScope("bytes-pool")))
checkedBytesPoolOpts := bytesPoolOpts.
SetInstrumentOptions(iopts.SetMetricsScope(scope.SubScope("checked-bytes-pool")))
buckets := make([]pool.Bucket, len(policy.BytesPool.Buckets))
for i, bucket := range policy.BytesPool.Buckets {
var b pool.Bucket
b.Capacity = bucket.Capacity
b.Count = bucket.Size
b.Options = bytesPoolOpts.
SetRefillLowWatermark(bucket.RefillLowWaterMark).
SetRefillHighWatermark(bucket.RefillHighWaterMark)
buckets[i] = b
logger.Infof("bytes pool registering bucket capacity=%d, size=%d, "+
"refillLowWatermark=%f, refillHighWatermark=%f",
bucket.Capacity, bucket.Size,
bucket.RefillLowWaterMark, bucket.RefillHighWaterMark)
}
var bytesPool pool.CheckedBytesPool
switch policy.Type {
case config.SimplePooling:
bytesPool = pool.NewCheckedBytesPool(
buckets,
checkedBytesPoolOpts,
func(s []pool.Bucket) pool.BytesPool {
return pool.NewBytesPool(s, bytesPoolOpts)
})
default:
logger.Fatalf("unrecognized pooling type: %s", policy.Type)
}
logger.Infof("bytes pool %s init", policy.Type)
bytesPool.Init()
segmentReaderPool := xio.NewSegmentReaderPool(
poolOptions(policy.SegmentReaderPool, scope.SubScope("segment-reader-pool")))
segmentReaderPool.Init()
encoderPool := encoding.NewEncoderPool(
poolOptions(policy.EncoderPool, scope.SubScope("encoder-pool")))
closersPoolOpts := poolOptions(policy.ClosersPool, scope.SubScope("closers-pool"))
contextPoolOpts := poolOptions(policy.ContextPool.PoolPolicy(), scope.SubScope("context-pool"))
contextPool := context.NewPool(context.NewOptions().
SetContextPoolOptions(contextPoolOpts).
SetFinalizerPoolOptions(closersPoolOpts).
SetMaxPooledFinalizerCapacity(policy.ContextPool.MaxFinalizerCapacityWithDefault()))
iteratorPool := encoding.NewReaderIteratorPool(
poolOptions(policy.IteratorPool, scope.SubScope("iterator-pool")))
multiIteratorPool := encoding.NewMultiReaderIteratorPool(
poolOptions(policy.IteratorPool, scope.SubScope("multi-iterator-pool")))
var writeBatchPoolInitialBatchSize *int
if policy.WriteBatchPool.InitialBatchSize != nil {
// Use config value if available.
writeBatchPoolInitialBatchSize = policy.WriteBatchPool.InitialBatchSize
} else {