-
Notifications
You must be signed in to change notification settings - Fork 0
/
server.go
999 lines (875 loc) · 31 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
// Copyright (c) 2018 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 (
"context"
"fmt"
"math/rand"
"net"
"net/http"
"os"
"sort"
"strings"
"time"
clusterclient "github.com/m3db/m3/src/cluster/client"
etcdclient "github.com/m3db/m3/src/cluster/client/etcd"
"github.com/m3db/m3/src/cmd/services/m3coordinator/downsample"
"github.com/m3db/m3/src/cmd/services/m3coordinator/ingest"
ingestcarbon "github.com/m3db/m3/src/cmd/services/m3coordinator/ingest/carbon"
dbconfig "github.com/m3db/m3/src/cmd/services/m3dbnode/config"
"github.com/m3db/m3/src/cmd/services/m3query/config"
"github.com/m3db/m3/src/dbnode/client"
"github.com/m3db/m3/src/metrics/aggregation"
"github.com/m3db/m3/src/metrics/policy"
"github.com/m3db/m3/src/query/api/v1/handler"
"github.com/m3db/m3/src/query/api/v1/httpd"
m3dbcluster "github.com/m3db/m3/src/query/cluster/m3db"
"github.com/m3db/m3/src/query/executor"
"github.com/m3db/m3/src/query/models"
"github.com/m3db/m3/src/query/policy/filter"
"github.com/m3db/m3/src/query/pools"
"github.com/m3db/m3/src/query/storage"
"github.com/m3db/m3/src/query/storage/fanout"
"github.com/m3db/m3/src/query/storage/m3"
"github.com/m3db/m3/src/query/storage/remote"
"github.com/m3db/m3/src/query/stores/m3db"
tsdbRemote "github.com/m3db/m3/src/query/tsdb/remote"
"github.com/m3db/m3/src/x/clock"
xconfig "github.com/m3db/m3/src/x/config"
"github.com/m3db/m3/src/x/instrument"
xos "github.com/m3db/m3/src/x/os"
"github.com/m3db/m3/src/x/pool"
"github.com/m3db/m3/src/x/serialize"
xserver "github.com/m3db/m3/src/x/server"
xsync "github.com/m3db/m3/src/x/sync"
xtime "github.com/m3db/m3/src/x/time"
opentracing "github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"github.com/uber-go/tally"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/reflection"
)
const (
serviceName = "m3query"
cpuProfileDuration = 5 * time.Second
)
var (
defaultLocalConfiguration = &config.LocalConfiguration{
Namespaces: []m3.ClusterStaticNamespaceConfiguration{
{
Namespace: "default",
Type: storage.UnaggregatedMetricsType,
Retention: 2 * 24 * time.Hour,
},
},
}
defaultDownsamplerAndWriterWorkerPoolSize = 1024
defaultCarbonIngesterWorkerPoolSize = 1024
)
type cleanupFn func() error
// RunOptions provides options for running the server
// with backwards compatibility if only solely adding fields.
type RunOptions struct {
// ConfigFiles is the array of config files to use. All files of the array
// get merged together.
ConfigFiles []string
// Config is an alternate way to provide configuration and will be used
// instead of parsing ConfigFile if ConfigFile is not specified.
Config config.Configuration
// DBConfig is the local M3DB config when running embedded.
DBConfig *dbconfig.DBConfiguration
// DBClient is the local M3DB client when running embedded.
DBClient <-chan client.Client
// ClusterClient is the local M3DB cluster client when running embedded.
ClusterClient <-chan clusterclient.Client
// InterruptCh is a programmatic interrupt channel to supply to
// interrupt and shutdown the server.
InterruptCh <-chan error
// ListenerCh is a programmatic channel to receive the server listener
// on once it has opened.
ListenerCh chan<- net.Listener
}
// Run runs the server programmatically given a filename for the configuration file.
func Run(runOpts RunOptions) {
rand.Seed(time.Now().UnixNano())
var cfg config.Configuration
if len(runOpts.ConfigFiles) > 0 {
err := xconfig.LoadFiles(&cfg, runOpts.ConfigFiles, xconfig.Options{})
if err != nil {
fmt.Fprintf(os.Stderr, "unable to load %s: %v", runOpts.ConfigFiles, err)
os.Exit(1)
}
fmt.Fprintf(os.Stdout, "using %s config files: %v",
serviceName, runOpts.ConfigFiles)
} 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)
}
defer logger.Sync()
xconfig.WarnOnDeprecation(cfg, logger)
scope, closer, err := cfg.Metrics.NewRootScope()
if err != nil {
logger.Fatal("could not connect to metrics", zap.Error(err))
}
tracer, traceCloser, err := cfg.Tracing.NewTracer(serviceName, scope, logger)
if err != nil {
logger.Fatal("could not initialize tracing", zap.Error(err))
}
defer traceCloser.Close()
if _, ok := tracer.(opentracing.NoopTracer); ok {
logger.Info("tracing disabled for m3query; set `tracing.backend` to enable")
}
instrumentOptions := instrument.NewOptions().
SetMetricsScope(scope).
SetLogger(logger).
SetTracer(tracer)
opentracing.SetGlobalTracer(tracer)
// Close metrics scope
defer func() {
if e := recover(); e != nil {
logger.Warn("recovered from panic", zap.String("e", fmt.Sprintf("%v", e)))
}
logger.Info("closing metrics scope")
if err := closer.Close(); err != nil {
logger.Error("unable to close metrics scope", zap.Error(err))
}
}()
buildInfoOpts := instrumentOptions.SetMetricsScope(
instrumentOptions.MetricsScope().SubScope("build_info"))
buildReporter := instrument.NewBuildReporter(buildInfoOpts)
if err := buildReporter.Start(); err != nil {
logger.Fatal("could not start build reporter", zap.Error(err))
}
defer buildReporter.Stop()
var (
backendStorage storage.Storage
clusterClient clusterclient.Client
downsampler downsample.Downsampler
fetchOptsBuilderCfg = cfg.Limits.PerQuery.AsFetchOptionsBuilderOptions()
fetchOptsBuilder = handler.NewFetchOptionsBuilder(fetchOptsBuilderCfg)
queryCtxOpts = models.QueryContextOptions{
LimitMaxTimeseries: fetchOptsBuilderCfg.Limit,
}
)
tagOptions, err := config.TagOptionsFromConfig(cfg.TagOptions)
if err != nil {
logger.Fatal("could not create tag options", zap.Error(err))
}
lookbackDuration, err := cfg.LookbackDurationOrDefault()
if err != nil {
logger.Fatal("error validating LookbackDuration", zap.Error(err))
}
cfg.LookbackDuration = &lookbackDuration
readWorkerPool, writeWorkerPool, err := pools.BuildWorkerPools(
instrumentOptions,
cfg.ReadWorkerPool,
cfg.WriteWorkerPool,
scope,
)
if err != nil {
logger.Fatal("could not create worker pools", zap.Error(err))
}
var (
m3dbClusters m3.Clusters
m3dbPoolWrapper *pools.PoolWrapper
)
if cfg.Backend == config.GRPCStorageType {
// For grpc backend, we need to setup only the grpc client and a storage
// accompanying that client.
poolWrapper := pools.NewPoolsWrapper(pools.BuildIteratorPools())
opts := config.RemoteOptionsFromConfig(cfg.RPC)
remotes, enabled, err := remoteClient(lookbackDuration, opts, tagOptions,
poolWrapper, readWorkerPool, instrumentOptions)
if err != nil {
logger.Fatal("unable to setup grpc backend", zap.Error(err))
}
if !enabled {
logger.Fatal("need remote clients for grpc backend")
}
var (
r = filter.AllowAll
w = filter.AllowAll
c = filter.CompleteTagsAllowAll
)
backendStorage = fanout.NewStorage(remotes, r, w, c,
instrumentOptions)
logger.Info("setup grpc backend")
} else {
// For m3db backend, we need to make connections to the m3db cluster
// which generates a session and use the storage with the session.
m3dbClusters, m3dbPoolWrapper, err = initClusters(cfg,
runOpts.DBClient, instrumentOptions)
if err != nil {
logger.Fatal("unable to init clusters", zap.Error(err))
}
var cleanup cleanupFn
backendStorage, clusterClient, downsampler, cleanup, err = newM3DBStorage(
runOpts, cfg, tagOptions, m3dbClusters, m3dbPoolWrapper,
readWorkerPool, writeWorkerPool, queryCtxOpts, instrumentOptions)
if err != nil {
logger.Fatal("unable to setup m3db backend", zap.Error(err))
}
defer cleanup()
}
perQueryEnforcer, err := newConfiguredChainedEnforcer(&cfg, instrumentOptions)
if err != nil {
logger.Fatal("unable to setup perQueryEnforcer", zap.Error(err))
}
engineOpts := executor.NewEngineOptions().
SetStore(backendStorage).
SetLookbackDuration(*cfg.LookbackDuration).
SetGlobalEnforcer(perQueryEnforcer).
SetInstrumentOptions(instrumentOptions.
SetMetricsScope(instrumentOptions.MetricsScope().SubScope("engine")))
engine := executor.NewEngine(engineOpts)
downsamplerAndWriter, err := newDownsamplerAndWriter(backendStorage, downsampler)
if err != nil {
logger.Fatal("unable to create new downsampler and writer", zap.Error(err))
}
var serviceOptionDefaults []handler.ServiceOptionsDefault
if dbCfg := runOpts.DBConfig; dbCfg != nil {
cluster, err := dbCfg.EnvironmentConfig.Services.SyncCluster()
if err != nil {
logger.Fatal("could not resolve embedded db cluster info",
zap.Error(err))
}
if svcCfg := cluster.Service; svcCfg != nil {
serviceOptionDefaults = append(serviceOptionDefaults,
handler.WithDefaultServiceEnvironment(svcCfg.Env))
serviceOptionDefaults = append(serviceOptionDefaults,
handler.WithDefaultServiceZone(svcCfg.Zone))
}
}
handler, err := httpd.NewHandler(downsamplerAndWriter, tagOptions, engine,
m3dbClusters, clusterClient, cfg, runOpts.DBConfig, perQueryEnforcer,
fetchOptsBuilder, queryCtxOpts, instrumentOptions, cpuProfileDuration,
[]string{handler.M3DBServiceName}, serviceOptionDefaults)
if err != nil {
logger.Fatal("unable to set up handlers", zap.Error(err))
}
if err := handler.RegisterRoutes(); err != nil {
logger.Fatal("unable to register routes", zap.Error(err))
}
listenAddress, err := cfg.ListenAddress.Resolve()
if err != nil {
logger.Fatal("unable to get listen address", zap.Error(err))
}
srv := &http.Server{Addr: listenAddress, Handler: handler.Router()}
defer func() {
logger.Info("closing server")
if err := srv.Shutdown(context.Background()); err != nil {
logger.Error("error closing server", zap.Error(err))
}
}()
listener, err := net.Listen("tcp", listenAddress)
if err != nil {
logger.Fatal("unable to listen on listen address",
zap.String("address", listenAddress),
zap.Error(err))
}
if runOpts.ListenerCh != nil {
runOpts.ListenerCh <- listener
}
go func() {
logger.Info("starting API server", zap.String("address", listenAddress))
if err := srv.Serve(listener); err != nil && err != http.ErrServerClosed {
logger.Fatal("server serve error",
zap.String("address", listenAddress),
zap.Error(err))
}
}()
if cfg.Ingest != nil {
logger.Info("starting m3msg server",
zap.String("address", cfg.Ingest.M3Msg.Server.ListenAddress))
ingester, err := cfg.Ingest.Ingester.NewIngester(backendStorage, instrumentOptions)
if err != nil {
logger.Fatal("unable to create ingester", zap.Error(err))
}
server, err := cfg.Ingest.M3Msg.NewServer(
ingester.Ingest,
instrumentOptions.SetMetricsScope(scope.SubScope("ingest-m3msg")),
)
if err != nil {
logger.Fatal("unable to create m3msg server", zap.Error(err))
}
if err := server.ListenAndServe(); err != nil {
logger.Fatal("unable to listen on ingest server", zap.Error(err))
}
logger.Info("started m3msg server ")
defer server.Close()
} else {
logger.Info("no m3msg server configured")
}
if cfg.Carbon != nil && cfg.Carbon.Ingester != nil {
server, ok := startCarbonIngestion(cfg.Carbon, instrumentOptions,
logger, m3dbClusters, downsamplerAndWriter)
if ok {
defer server.Close()
}
}
// Wait for process interrupt.
xos.WaitForInterrupt(logger, xos.InterruptOptions{
InterruptCh: runOpts.InterruptCh,
})
}
// make connections to the m3db cluster(s) and generate sessions for those clusters along with the storage
func newM3DBStorage(
runOpts RunOptions,
cfg config.Configuration,
tagOptions models.TagOptions,
clusters m3.Clusters,
poolWrapper *pools.PoolWrapper,
readWorkerPool xsync.PooledWorkerPool,
writeWorkerPool xsync.PooledWorkerPool,
queryContextOptions models.QueryContextOptions,
instrumentOptions instrument.Options,
) (storage.Storage, clusterclient.Client, downsample.Downsampler, cleanupFn, error) {
var (
logger = instrumentOptions.Logger()
clusterClient clusterclient.Client
clusterClientWaitCh <-chan struct{}
)
if clusterClientCh := runOpts.ClusterClient; clusterClientCh != nil {
// Only use a cluster client if we are going to receive one, that
// way passing nil to httpd NewHandler disables the endpoints entirely
clusterClientDoneCh := make(chan struct{}, 1)
clusterClientWaitCh = clusterClientDoneCh
clusterClient = m3dbcluster.NewAsyncClient(func() (clusterclient.Client, error) {
return <-clusterClientCh, nil
}, clusterClientDoneCh)
} else {
var etcdCfg *etcdclient.Configuration
switch {
case cfg.ClusterManagement != nil:
etcdCfg = &cfg.ClusterManagement.Etcd
case len(cfg.Clusters) == 1 &&
cfg.Clusters[0].Client.EnvironmentConfig != nil:
syncCfg, err := cfg.Clusters[0].Client.EnvironmentConfig.Services.SyncCluster()
if err != nil {
return nil, nil, nil, nil, errors.Wrap(err, "unable to get etcd sync cluster config")
}
etcdCfg = syncCfg.Service
}
if etcdCfg != nil {
// We resolved an etcd configuration for cluster management endpoints
var (
clusterSvcClientOpts = etcdCfg.NewOptions()
err error
)
clusterClient, err = etcdclient.NewConfigServiceClient(clusterSvcClientOpts)
if err != nil {
return nil, nil, nil, nil, errors.Wrap(err, "unable to create cluster management etcd client")
}
}
}
fanoutStorage, storageCleanup, err := newStorages(clusters, cfg, tagOptions,
poolWrapper, readWorkerPool, writeWorkerPool, queryContextOptions,
instrumentOptions)
if err != nil {
return nil, nil, nil, nil, errors.Wrap(err, "unable to set up storages")
}
var (
namespaces = clusters.ClusterNamespaces()
downsampler downsample.Downsampler
)
if n := namespaces.NumAggregatedClusterNamespaces(); n > 0 {
logger.Info("configuring downsampler to use with aggregated cluster namespaces",
zap.Int("numAggregatedClusterNamespaces", n))
autoMappingRules, err := newDownsamplerAutoMappingRules(namespaces)
if err != nil {
return nil, nil, nil, nil, err
}
newDownsamplerFn := func() (downsample.Downsampler, error) {
return newDownsampler(cfg.Downsample, clusterClient,
fanoutStorage, autoMappingRules, tagOptions, instrumentOptions)
}
if clusterClientWaitCh != nil {
// Need to wait before constructing and instead return an async downsampler
// since the cluster client will return errors until it's initialized itself
// and will fail constructing the downsampler consequently
downsampler = downsample.NewAsyncDownsampler(func() (downsample.Downsampler, error) {
<-clusterClientWaitCh
return newDownsamplerFn()
}, nil)
} else {
// Otherwise we already have a client and can immediately construct the downsampler
downsampler, err = newDownsamplerFn()
if err != nil {
return nil, nil, nil, nil, err
}
}
}
cleanup := func() error {
lastErr := storageCleanup()
// Don't want to quit on the first error since the full cleanup is important
if lastErr != nil {
logger.Error("error during storage cleanup", zap.Error(lastErr))
}
if err := clusters.Close(); err != nil {
lastErr = errors.Wrap(err, "unable to close M3DB cluster sessions")
// Make sure the previous error is at least logged
logger.Error("error during cluster cleanup", zap.Error(err))
}
return lastErr
}
return fanoutStorage, clusterClient, downsampler, cleanup, nil
}
func newDownsampler(
cfg downsample.Configuration,
clusterManagementClient clusterclient.Client,
storage storage.Storage,
autoMappingRules []downsample.MappingRule,
tagOptions models.TagOptions,
instrumentOpts instrument.Options,
) (downsample.Downsampler, error) {
if clusterManagementClient == nil {
return nil, fmt.Errorf("no configured cluster management config, " +
"must set this config for downsampler")
}
kvStore, err := clusterManagementClient.KV()
if err != nil {
return nil, errors.Wrap(err, "unable to create KV store from the "+
"cluster management config client")
}
tagEncoderOptions := serialize.NewTagEncoderOptions()
tagDecoderOptions := serialize.NewTagDecoderOptions()
tagEncoderPoolOptions := pool.NewObjectPoolOptions().
SetInstrumentOptions(instrumentOpts.
SetMetricsScope(instrumentOpts.MetricsScope().
SubScope("tag-encoder-pool")))
tagDecoderPoolOptions := pool.NewObjectPoolOptions().
SetInstrumentOptions(instrumentOpts.
SetMetricsScope(instrumentOpts.MetricsScope().
SubScope("tag-decoder-pool")))
downsampler, err := cfg.NewDownsampler(downsample.DownsamplerOptions{
Storage: storage,
ClusterClient: clusterManagementClient,
RulesKVStore: kvStore,
AutoMappingRules: autoMappingRules,
ClockOptions: clock.NewOptions(),
// TODO: remove after https://github.com/m3db/m3/issues/992 is fixed
InstrumentOptions: instrumentOpts.SetMetricsScope(tally.NoopScope),
TagEncoderOptions: tagEncoderOptions,
TagDecoderOptions: tagDecoderOptions,
TagEncoderPoolOptions: tagEncoderPoolOptions,
TagDecoderPoolOptions: tagDecoderPoolOptions,
TagOptions: tagOptions,
})
if err != nil {
return nil, errors.Wrap(err, "unable to create downsampler")
}
return downsampler, nil
}
func newDownsamplerAutoMappingRules(
namespaces []m3.ClusterNamespace,
) ([]downsample.MappingRule, error) {
var autoMappingRules []downsample.MappingRule
for _, namespace := range namespaces {
opts := namespace.Options()
attrs := opts.Attributes()
if attrs.MetricsType == storage.AggregatedMetricsType {
downsampleOpts, err := opts.DownsampleOptions()
if err != nil {
errFmt := "unable to resolve downsample options for namespace: %v"
return nil, fmt.Errorf(errFmt, namespace.NamespaceID().String())
}
if downsampleOpts.All {
storagePolicy := policy.NewStoragePolicy(attrs.Resolution,
xtime.Second, attrs.Retention)
autoMappingRules = append(autoMappingRules, downsample.MappingRule{
// NB(r): By default we will apply just keep all last values
// since coordinator only uses downsampling with Prometheus
// remote write endpoint.
// More rich static configuration mapping rules can be added
// in the future but they are currently not required.
Aggregations: []aggregation.Type{aggregation.Last},
Policies: policy.StoragePolicies{storagePolicy},
})
}
}
}
return autoMappingRules, nil
}
func initClusters(
cfg config.Configuration,
dbClientCh <-chan client.Client,
instrumentOpts instrument.Options,
) (m3.Clusters, *pools.PoolWrapper, error) {
instrumentOpts = instrumentOpts.
SetMetricsScope(instrumentOpts.MetricsScope().SubScope("m3db-client"))
var (
logger = instrumentOpts.Logger()
clusters m3.Clusters
poolWrapper *pools.PoolWrapper
err error
)
if len(cfg.Clusters) > 0 {
clusters, err = cfg.Clusters.NewClusters(instrumentOpts,
m3.ClustersStaticConfigurationOptions{
AsyncSessions: true,
})
if err != nil {
return nil, nil, errors.Wrap(err, "unable to connect to clusters")
}
poolWrapper = pools.NewPoolsWrapper(pools.BuildIteratorPools())
} else {
localCfg := cfg.Local
if localCfg == nil {
localCfg = defaultLocalConfiguration
}
if dbClientCh == nil {
return nil, nil, errors.New("no clusters configured and not running local cluster")
}
sessionInitChan := make(chan struct{})
session := m3db.NewAsyncSession(func() (client.Client, error) {
return <-dbClientCh, nil
}, sessionInitChan)
clustersCfg := m3.ClustersStaticConfiguration{
m3.ClusterStaticConfiguration{
Namespaces: localCfg.Namespaces,
},
}
clusters, err = clustersCfg.NewClusters(instrumentOpts,
m3.ClustersStaticConfigurationOptions{
ProvidedSession: session,
})
if err != nil {
return nil, nil, errors.Wrap(err, "unable to connect to clusters")
}
poolWrapper = pools.NewAsyncPoolsWrapper()
go func() {
<-sessionInitChan
poolWrapper.Init(session.IteratorPools())
}()
}
for _, namespace := range clusters.ClusterNamespaces() {
logger.Info("resolved cluster namespace",
zap.String("namespace", namespace.NamespaceID().String()))
}
return clusters, poolWrapper, nil
}
func newStorages(
clusters m3.Clusters,
cfg config.Configuration,
tagOptions models.TagOptions,
poolWrapper *pools.PoolWrapper,
readWorkerPool xsync.PooledWorkerPool,
writeWorkerPool xsync.PooledWorkerPool,
queryContextOptions models.QueryContextOptions,
instrumentOpts instrument.Options,
) (storage.Storage, cleanupFn, error) {
var (
logger = instrumentOpts.Logger()
cleanup = func() error { return nil }
)
lookback, err := cfg.LookbackDurationOrDefault()
if err != nil {
return nil, cleanup, err
}
localStorage, err := m3.NewStorage(clusters, readWorkerPool,
writeWorkerPool, tagOptions, lookback, instrumentOpts)
if err != nil {
return nil, nil, err
}
stores := []storage.Storage{localStorage}
remoteEnabled := false
rpcOpts := config.RemoteOptionsFromConfig(cfg.RPC)
if rpcOpts.ServeEnabled() {
logger.Info("rpc serve enabled")
server, err := startGRPCServer(localStorage, queryContextOptions,
poolWrapper, rpcOpts, instrumentOpts)
if err != nil {
return nil, nil, err
}
cleanup = func() error {
server.GracefulStop()
return nil
}
}
if rpcOpts.ListenEnabled() {
remoteStorages, enabled, err := remoteClient(
lookback,
rpcOpts,
tagOptions,
poolWrapper,
readWorkerPool,
instrumentOpts,
)
if err != nil {
return nil, nil, err
}
if enabled {
stores = append(stores, remoteStorages...)
remoteEnabled = enabled
}
}
readFilter := filter.LocalOnly
writeFilter := filter.LocalOnly
completeTagsFilter := filter.CompleteTagsLocalOnly
if remoteEnabled {
// If remote enabled, allow all for read and complete tags
// but continue to only send writes locally
readFilter = filter.AllowAll
completeTagsFilter = filter.CompleteTagsAllowAll
}
switch cfg.Filter.Read {
case config.FilterLocalOnly:
readFilter = filter.LocalOnly
case config.FilterRemoteOnly:
readFilter = filter.RemoteOnly
case config.FilterAllowAll:
readFilter = filter.AllowAll
case config.FilterAllowNone:
readFilter = filter.AllowNone
}
switch cfg.Filter.Write {
case config.FilterLocalOnly:
writeFilter = filter.LocalOnly
case config.FilterRemoteOnly:
writeFilter = filter.RemoteOnly
case config.FilterAllowAll:
writeFilter = filter.AllowAll
case config.FilterAllowNone:
writeFilter = filter.AllowNone
}
switch cfg.Filter.CompleteTags {
case config.FilterLocalOnly:
completeTagsFilter = filter.CompleteTagsLocalOnly
case config.FilterRemoteOnly:
completeTagsFilter = filter.CompleteTagsRemoteOnly
case config.FilterAllowAll:
completeTagsFilter = filter.CompleteTagsAllowAll
case config.FilterAllowNone:
completeTagsFilter = filter.CompleteTagsAllowNone
}
fanoutStorage := fanout.NewStorage(stores, readFilter, writeFilter,
completeTagsFilter, instrumentOpts)
return fanoutStorage, cleanup, nil
}
func remoteZoneStorage(
zone config.Remote,
lookbackDuration time.Duration,
tagOptions models.TagOptions,
poolWrapper *pools.PoolWrapper,
readWorkerPool xsync.PooledWorkerPool,
instrumentOpts instrument.Options,
) (storage.Storage, error) {
if len(zone.Addresses) == 0 {
// No addresses; skip.
return nil, nil
}
client, err := tsdbRemote.NewGRPCClient(
zone.Addresses,
poolWrapper,
readWorkerPool,
tagOptions,
lookbackDuration,
)
if err != nil {
return nil, err
}
remoteOpts := remote.RemoteOptions{
Name: zone.Name,
ErrorBehavior: zone.ErrorBehavior,
}
remoteStorage := remote.NewStorage(client, remoteOpts)
return remoteStorage, nil
}
func remoteClient(
lookback time.Duration,
remoteOpts config.RemoteOptions,
tagOptions models.TagOptions,
poolWrapper *pools.PoolWrapper,
readWorkerPool xsync.PooledWorkerPool,
instrumentOpts instrument.Options,
) ([]storage.Storage, bool, error) {
logger := instrumentOpts.Logger()
remotes := remoteOpts.Remotes()
remoteStores := make([]storage.Storage, 0, len(remotes))
for _, zone := range remotes {
logger.Info(
"creating RPC client with remotes",
zap.String("name", zone.Name),
zap.Strings("addresses", zone.Addresses),
)
remote, err := remoteZoneStorage(zone, lookback, tagOptions, poolWrapper,
readWorkerPool, instrumentOpts)
if err != nil {
return nil, false, err
}
remoteStores = append(remoteStores, remote)
}
return remoteStores, true, nil
}
func startGRPCServer(
storage m3.Storage,
queryContextOptions models.QueryContextOptions,
poolWrapper *pools.PoolWrapper,
opts config.RemoteOptions,
instrumentOpts instrument.Options,
) (*grpc.Server, error) {
logger := instrumentOpts.Logger()
logger.Info("creating gRPC server")
server := tsdbRemote.NewGRPCServer(storage,
queryContextOptions, poolWrapper, instrumentOpts)
if opts.ReflectionEnabled() {
reflection.Register(server)
}
logger.Info("gRPC server reflection configured",
zap.Bool("enabled", opts.ReflectionEnabled()))
listener, err := net.Listen("tcp", opts.ServeAddress())
if err != nil {
return nil, err
}
go func() {
if err := server.Serve(listener); err != nil {
logger.Error("error from serving gRPC server", zap.Error(err))
}
}()
return server, nil
}
func startCarbonIngestion(
cfg *config.CarbonConfiguration,
iOpts instrument.Options,
logger *zap.Logger,
m3dbClusters m3.Clusters,
downsamplerAndWriter ingest.DownsamplerAndWriter,
) (xserver.Server, bool) {
ingesterCfg := cfg.Ingester
logger.Info("carbon ingestion enabled, configuring ingester")
// Setup worker pool.
var (
carbonIOpts = iOpts.SetMetricsScope(
iOpts.MetricsScope().SubScope("ingest-carbon"))
carbonWorkerPoolOpts xsync.PooledWorkerPoolOptions
carbonWorkerPoolSize int
)
if ingesterCfg.MaxConcurrency > 0 {
// Use a bounded worker pool if they requested a specific maximum concurrency.
carbonWorkerPoolOpts = xsync.NewPooledWorkerPoolOptions().
SetGrowOnDemand(false).
SetInstrumentOptions(carbonIOpts)
carbonWorkerPoolSize = ingesterCfg.MaxConcurrency
} else {
carbonWorkerPoolOpts = xsync.NewPooledWorkerPoolOptions().
SetGrowOnDemand(true).
SetKillWorkerProbability(0.001)
carbonWorkerPoolSize = defaultCarbonIngesterWorkerPoolSize
}
workerPool, err := xsync.NewPooledWorkerPool(carbonWorkerPoolSize, carbonWorkerPoolOpts)
if err != nil {
logger.Fatal("unable to create worker pool for carbon ingester", zap.Error(err))
}
workerPool.Init()
if m3dbClusters == nil {
logger.Fatal("carbon ingestion is only supported when connecting to M3DB clusters directly")
}
// Validate provided rules.
var (
clusterNamespaces = m3dbClusters.ClusterNamespaces()
rules = ingestcarbon.CarbonIngesterRules{
Rules: ingesterCfg.RulesOrDefault(clusterNamespaces),
}
)
for _, rule := range rules.Rules {
// Sort so we can detect duplicates.
sort.Slice(rule.Policies, func(i, j int) bool {
if rule.Policies[i].Resolution == rule.Policies[j].Resolution {
return rule.Policies[i].Retention < rule.Policies[j].Retention
}
return rule.Policies[i].Resolution < rule.Policies[j].Resolution
})
var lastPolicy config.CarbonIngesterStoragePolicyConfiguration
for i, policy := range rule.Policies {
if i > 0 && policy == lastPolicy {
logger.Fatal(
"cannot include the same storage policy multiple times for a single carbon ingestion rule",
zap.String("pattern", rule.Pattern), zap.Duration("resolution", policy.Resolution), zap.Duration("retention", policy.Retention))
}
if i > 0 && !rule.Aggregation.EnabledOrDefault() && policy.Resolution != lastPolicy.Resolution {
logger.Fatal(
"cannot include multiple storage policies with different resolutions if aggregation is disabled",
zap.String("pattern", rule.Pattern), zap.Duration("resolution", policy.Resolution), zap.Duration("retention", policy.Retention))
}
_, ok := m3dbClusters.AggregatedClusterNamespace(m3.RetentionResolution{
Resolution: policy.Resolution,
Retention: policy.Retention,
})
// Disallow storage policies that don't match any known M3DB clusters.
if !ok {
logger.Fatal(
"cannot enable carbon ingestion without a corresponding aggregated M3DB namespace",
zap.String("resolution", policy.Resolution.String()), zap.String("retention", policy.Retention.String()))
}
}
}
if len(rules.Rules) == 0 {
logger.Warn("no carbon ingestion rules were provided and no aggregated M3DB namespaces exist, carbon metrics will not be ingested")
return nil, false
}
if len(ingesterCfg.Rules) == 0 {
logger.Info("no carbon ingestion rules were provided, all carbon metrics will be written to all aggregated M3DB namespaces")
}
// Create ingester.
ingester, err := ingestcarbon.NewIngester(
downsamplerAndWriter, rules, ingestcarbon.Options{
InstrumentOptions: carbonIOpts,
WorkerPool: workerPool,
})
if err != nil {
logger.Fatal("unable to create carbon ingester", zap.Error(err))
}
// Start server.
var (
serverOpts = xserver.NewOptions().SetInstrumentOptions(carbonIOpts)
carbonListenAddress = ingesterCfg.ListenAddressOrDefault()
carbonServer = xserver.NewServer(carbonListenAddress, ingester, serverOpts)
)
if strings.TrimSpace(carbonListenAddress) == "" {
logger.Fatal("no listen address specified for carbon ingester")
}
logger.Info("starting carbon ingestion server", zap.String("listenAddress", carbonListenAddress))
err = carbonServer.ListenAndServe()
if err != nil {
logger.Fatal("unable to start carbon ingestion server at listen address",
zap.String("listenAddress", carbonListenAddress), zap.Error(err))
}
logger.Info("started carbon ingestion server", zap.String("listenAddress", carbonListenAddress))
return carbonServer, true
}
func newDownsamplerAndWriter(storage storage.Storage, downsampler downsample.Downsampler) (ingest.DownsamplerAndWriter, error) {
// Make sure the downsampler and writer gets its own PooledWorkerPool and that its not shared with any other
// codepaths because PooledWorkerPools can deadlock if used recursively.
downAndWriterWorkerPoolOpts := xsync.NewPooledWorkerPoolOptions().
SetGrowOnDemand(true).
SetKillWorkerProbability(0.001)
downAndWriteWorkerPool, err := xsync.NewPooledWorkerPool(
defaultDownsamplerAndWriterWorkerPoolSize, downAndWriterWorkerPoolOpts)
if err != nil {
return nil, err
}
downAndWriteWorkerPool.Init()
return ingest.NewDownsamplerAndWriter(storage, downsampler, downAndWriteWorkerPool), nil
}