github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/query/server/query.go (about) 1 // Copyright (c) 2018 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package server 22 23 import ( 24 "context" 25 "fmt" 26 "math/rand" 27 "net" 28 "net/http" 29 "os" 30 "strings" 31 "time" 32 33 "github.com/m3db/m3/src/aggregator/server" 34 clusterclient "github.com/m3db/m3/src/cluster/client" 35 etcdclient "github.com/m3db/m3/src/cluster/client/etcd" 36 "github.com/m3db/m3/src/cluster/kv" 37 memcluster "github.com/m3db/m3/src/cluster/mem" 38 handleroptions3 "github.com/m3db/m3/src/cluster/placementhandler/handleroptions" 39 "github.com/m3db/m3/src/cmd/services/m3aggregator/serve" 40 "github.com/m3db/m3/src/cmd/services/m3coordinator/downsample" 41 "github.com/m3db/m3/src/cmd/services/m3coordinator/ingest" 42 ingestcarbon "github.com/m3db/m3/src/cmd/services/m3coordinator/ingest/carbon" 43 dbconfig "github.com/m3db/m3/src/cmd/services/m3dbnode/config" 44 "github.com/m3db/m3/src/cmd/services/m3query/config" 45 "github.com/m3db/m3/src/dbnode/client" 46 "github.com/m3db/m3/src/dbnode/encoding" 47 "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" 48 "github.com/m3db/m3/src/query/api/v1/httpd" 49 "github.com/m3db/m3/src/query/api/v1/options" 50 m3dbcluster "github.com/m3db/m3/src/query/cluster/m3db" 51 "github.com/m3db/m3/src/query/executor" 52 graphite "github.com/m3db/m3/src/query/graphite/storage" 53 "github.com/m3db/m3/src/query/models" 54 "github.com/m3db/m3/src/query/parser/promql" 55 "github.com/m3db/m3/src/query/policy/filter" 56 "github.com/m3db/m3/src/query/pools" 57 "github.com/m3db/m3/src/query/promqlengine" 58 tsdbremote "github.com/m3db/m3/src/query/remote" 59 "github.com/m3db/m3/src/query/storage" 60 "github.com/m3db/m3/src/query/storage/fanout" 61 "github.com/m3db/m3/src/query/storage/m3" 62 "github.com/m3db/m3/src/query/storage/m3/consolidators" 63 "github.com/m3db/m3/src/query/storage/promremote" 64 "github.com/m3db/m3/src/query/storage/remote" 65 "github.com/m3db/m3/src/query/stores/m3db" 66 "github.com/m3db/m3/src/x/clock" 67 xconfig "github.com/m3db/m3/src/x/config" 68 "github.com/m3db/m3/src/x/instrument" 69 xio "github.com/m3db/m3/src/x/io" 70 xnet "github.com/m3db/m3/src/x/net" 71 xos "github.com/m3db/m3/src/x/os" 72 "github.com/m3db/m3/src/x/pool" 73 "github.com/m3db/m3/src/x/serialize" 74 xserver "github.com/m3db/m3/src/x/server" 75 xsync "github.com/m3db/m3/src/x/sync" 76 77 "github.com/go-kit/kit/log" 78 kitlogzap "github.com/go-kit/kit/log/zap" 79 "github.com/opentracing/opentracing-go" 80 "github.com/pkg/errors" 81 extprom "github.com/prometheus/client_golang/prometheus" 82 prometheuspromql "github.com/prometheus/prometheus/promql" 83 "go.uber.org/zap" 84 "go.uber.org/zap/zapcore" 85 "golang.org/x/net/http2" 86 "golang.org/x/net/http2/h2c" 87 "google.golang.org/grpc" 88 "google.golang.org/grpc/reflection" 89 ) 90 91 const ( 92 serviceName = "m3query" 93 cpuProfileDuration = 5 * time.Second 94 ) 95 96 var ( 97 defaultCarbonIngesterWorkerPoolSize = 1024 98 defaultPerCPUMultiProcess = 0.5 99 ) 100 101 type cleanupFn func() error 102 103 // RunOptions provides options for running the server 104 // with backwards compatibility if only solely adding fields. 105 type RunOptions struct { 106 // Config is an alternate way to provide configuration and will be used 107 // instead of parsing ConfigFile if ConfigFile is not specified. 108 Config config.Configuration 109 110 // DBConfig is the local M3DB config when running embedded. 111 DBConfig *dbconfig.DBConfiguration 112 113 // DBClient is the local M3DB client when running embedded. 114 DBClient <-chan client.Client 115 116 // ClusterClient is the local M3DB cluster client when running embedded. 117 ClusterClient <-chan clusterclient.Client 118 119 // InterruptCh is a programmatic interrupt channel to supply to 120 // interrupt and shutdown the server. 121 InterruptCh <-chan error 122 123 // ShutdownCh is an optional channel to supply if interested in receiving 124 // a notification that the server has shutdown. 125 ShutdownCh chan<- struct{} 126 127 // ListenerCh is a programmatic channel to receive the server listener 128 // on once it has opened. 129 ListenerCh chan<- net.Listener 130 131 // M3MsgListenerCh is a programmatic channel to receive the M3Msg server 132 // listener on once it has opened. 133 M3MsgListenerCh chan<- net.Listener 134 135 // DownsamplerReadyCh is a programmatic channel to receive the downsampler 136 // ready signal once it is open. 137 DownsamplerReadyCh chan<- struct{} 138 139 // LocalSessionReadyCh is a programmatic channel to receive the 140 // local DB session ready signal once it is open. 141 LocalSessionReadyCh chan struct{} 142 143 // InstrumentOptionsReadyCh is a programmatic channel to receive a set of 144 // instrument options and metric reporters that is delivered when 145 // constructed. 146 InstrumentOptionsReadyCh chan<- InstrumentOptionsReady 147 148 // ClockOptions is an optional clock to use instead of the default one. 149 ClockOptions clock.Options 150 151 // CustomHandlerOptions creates custom handler options. 152 CustomHandlerOptions CustomHandlerOptionsFn 153 154 // CustomPromQLParseFunction is a custom PromQL parsing function. 155 CustomPromQLParseFunction promql.ParseFn 156 157 // ApplyCustomTSDBOptions is a transform that allows for custom tsdb options. 158 ApplyCustomTSDBOptions CustomTSDBOptionsFn 159 160 // BackendStorageTransform is a custom backend storage transform. 161 BackendStorageTransform BackendStorageTransformFn 162 163 // AggregatorServerOptions are server options for aggregator. 164 AggregatorServerOptions []server.AdminOption 165 166 // CustomBuildTags are additional tags to be added to the instrument build 167 // reporter. 168 CustomBuildTags map[string]string 169 170 // ApplyCustomRuleStore provides an option to swap the backend used for the rule stores. 171 ApplyCustomRuleStore downsample.CustomRuleStoreFn 172 } 173 174 // InstrumentOptionsReady is a set of instrument options 175 // and metric reporters that is delivered when constructed. 176 type InstrumentOptionsReady struct { 177 InstrumentOptions instrument.Options 178 MetricsReporters instrument.MetricsConfigurationReporters 179 } 180 181 // CustomTSDBOptionsFn is a transformation function for TSDB Options. 182 type CustomTSDBOptionsFn func(m3.Options, instrument.Options) (m3.Options, error) 183 184 // CustomHandlerOptionsFn is a factory for options.CustomHandlerOptions. 185 type CustomHandlerOptionsFn func(instrument.Options) (options.CustomHandlerOptions, error) 186 187 // BackendStorageTransformFn is a transformation function for backend storage. 188 type BackendStorageTransformFn func( 189 storage.Storage, 190 m3.Options, 191 instrument.Options, 192 ) (storage.Storage, error) 193 194 // RunResult returns metadata about the process run. 195 type RunResult struct { 196 MultiProcessRun bool 197 MultiProcessIsParentCleanExit bool 198 } 199 200 // Run runs the server programmatically given a filename for the configuration file. 201 func Run(runOpts RunOptions) RunResult { 202 rand.Seed(time.Now().UnixNano()) 203 204 var ( 205 cfg = runOpts.Config 206 listenerOpts = xnet.NewListenerOptions() 207 runResult RunResult 208 ) 209 210 logger, err := cfg.LoggingOrDefault().BuildLogger() 211 if err != nil { 212 // NB(r): Use fmt.Fprintf(os.Stderr, ...) to avoid etcd.SetGlobals() 213 // sending stdlib "log" to black hole. Don't remove unless with good reason. 214 fmt.Fprintf(os.Stderr, "unable to create logger: %v", err) 215 os.Exit(1) 216 } 217 218 // NB(nate): Register shutdown notification defer function first so that 219 // it's the last defer to fire before terminating. This allows other defer methods 220 // that clean up resources to execute first. 221 if runOpts.ShutdownCh != nil { 222 defer func() { 223 select { 224 case runOpts.ShutdownCh <- struct{}{}: 225 break 226 default: 227 logger.Warn("could not send shutdown notification as channel was full") 228 } 229 }() 230 } 231 232 interruptOpts := xos.NewInterruptOptions() 233 if runOpts.InterruptCh != nil { 234 interruptOpts.InterruptCh = runOpts.InterruptCh 235 } 236 intWatchCancel := xos.WatchForInterrupt(logger, interruptOpts) 237 defer intWatchCancel() 238 239 defer logger.Sync() 240 241 cfg.Debug.SetRuntimeValues(logger) 242 243 xconfig.WarnOnDeprecation(cfg, logger) 244 245 var commonLabels map[string]string 246 if cfg.MultiProcess.Enabled { 247 // Mark as a multi-process run result. 248 runResult.MultiProcessRun = true 249 250 // Execute multi-process parent spawn or child setup code path. 251 multiProcessRunResult, err := multiProcessRun(cfg, logger, listenerOpts) 252 if err != nil { 253 logger = logger.With(zap.String("processID", multiProcessProcessID())) 254 logger.Fatal("failed to run", zap.Error(err)) 255 } 256 if multiProcessRunResult.isParentCleanExit { 257 // Parent process clean exit. 258 runResult.MultiProcessIsParentCleanExit = true 259 return runResult 260 } 261 262 cfg = multiProcessRunResult.cfg 263 logger = multiProcessRunResult.logger 264 listenerOpts = multiProcessRunResult.listenerOpts 265 commonLabels = multiProcessRunResult.commonLabels 266 } 267 268 prometheusEngineRegistry := extprom.NewRegistry() 269 scope, closer, reporters, err := cfg.MetricsOrDefault().NewRootScopeAndReporters( 270 instrument.NewRootScopeAndReportersOptions{ 271 PrometheusExternalRegistries: []instrument.PrometheusExternalRegistry{ 272 { 273 Registry: prometheusEngineRegistry, 274 SubScope: "coordinator_prometheus_engine", 275 }, 276 }, 277 PrometheusOnError: func(err error) { 278 // NB(r): Required otherwise collisions when registering metrics will 279 // cause a panic. 280 logger.Error("register metric error", zap.Error(err)) 281 }, 282 CommonLabels: commonLabels, 283 }) 284 if err != nil { 285 logger.Fatal("could not connect to metrics", zap.Error(err)) 286 } 287 288 tracer, traceCloser, err := cfg.Tracing.NewTracer(serviceName, scope, logger) 289 if err != nil { 290 logger.Fatal("could not initialize tracing", zap.Error(err)) 291 } 292 293 defer traceCloser.Close() 294 295 if _, ok := tracer.(opentracing.NoopTracer); ok { 296 logger.Info("tracing disabled for m3query; set `tracing.backend` to enable") 297 } 298 299 opentracing.SetGlobalTracer(tracer) 300 301 clockOpts := clock.NewOptions() 302 if runOpts.ClockOptions != nil { 303 clockOpts = runOpts.ClockOptions 304 } 305 306 instrumentOptions := instrument.NewOptions(). 307 SetMetricsScope(scope). 308 SetLogger(logger). 309 SetTracer(tracer). 310 SetCustomBuildTags(runOpts.CustomBuildTags) 311 312 if runOpts.InstrumentOptionsReadyCh != nil { 313 runOpts.InstrumentOptionsReadyCh <- InstrumentOptionsReady{ 314 InstrumentOptions: instrumentOptions, 315 MetricsReporters: reporters, 316 } 317 } 318 319 // Close metrics scope 320 defer func() { 321 logger.Info("closing metrics scope") 322 if err := closer.Close(); err != nil { 323 logger.Error("unable to close metrics scope", zap.Error(err)) 324 } 325 }() 326 327 buildReporter := instrument.NewBuildReporter(instrumentOptions) 328 if err := buildReporter.Start(); err != nil { 329 logger.Fatal("could not start build reporter", zap.Error(err)) 330 } 331 332 defer buildReporter.Stop() 333 334 storageRestrictByTags, _, err := cfg.Query.RestrictTagsAsStorageRestrictByTag() 335 if err != nil { 336 logger.Fatal("could not parse query restrict tags config", zap.Error(err)) 337 } 338 339 timeout := cfg.Query.TimeoutOrDefault() 340 if runOpts.DBConfig != nil && 341 runOpts.DBConfig.Client.FetchTimeout != nil && 342 *runOpts.DBConfig.Client.FetchTimeout > timeout { 343 timeout = *runOpts.DBConfig.Client.FetchTimeout 344 } 345 346 fetchOptsBuilderLimitsOpts := cfg.Limits.PerQuery.AsFetchOptionsBuilderLimitsOptions() 347 fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder( 348 handleroptions.FetchOptionsBuilderOptions{ 349 Limits: fetchOptsBuilderLimitsOpts, 350 RestrictByTag: storageRestrictByTags, 351 Timeout: timeout, 352 }) 353 if err != nil { 354 logger.Fatal("could not set fetch options parser", zap.Error(err)) 355 } 356 357 var ( 358 clusterNamespacesWatcher = m3.NewClusterNamespacesWatcher() 359 backendStorage storage.Storage 360 clusterClient clusterclient.Client 361 downsampler downsample.Downsampler 362 queryCtxOpts = models.QueryContextOptions{ 363 LimitMaxTimeseries: fetchOptsBuilderLimitsOpts.SeriesLimit, 364 LimitMaxDocs: fetchOptsBuilderLimitsOpts.DocsLimit, 365 LimitMaxReturnedSeries: fetchOptsBuilderLimitsOpts.ReturnedSeriesLimit, 366 LimitMaxReturnedDatapoints: fetchOptsBuilderLimitsOpts.ReturnedDatapointsLimit, 367 LimitMaxReturnedSeriesMetadata: fetchOptsBuilderLimitsOpts.ReturnedSeriesMetadataLimit, 368 RequireExhaustive: fetchOptsBuilderLimitsOpts.RequireExhaustive, 369 } 370 371 matchOptions = consolidators.MatchOptions{ 372 MatchType: cfg.Query.ConsolidationConfiguration.MatchType, 373 } 374 ) 375 defer clusterNamespacesWatcher.Close() 376 377 tagOptions, err := config.TagOptionsFromConfig(cfg.TagOptions) 378 if err != nil { 379 logger.Fatal("could not create tag options", zap.Error(err)) 380 } 381 382 lookbackDuration, err := cfg.LookbackDurationOrDefault() 383 if err != nil { 384 logger.Fatal("error validating LookbackDuration", zap.Error(err)) 385 } 386 cfg.LookbackDuration = &lookbackDuration 387 388 promConvertOptions := cfg.Query.Prometheus.ConvertOptionsOrDefault() 389 390 readWorkerPool, writeWorkerPool, err := pools.BuildWorkerPools( 391 instrumentOptions, 392 cfg.ReadWorkerPool, 393 cfg.WriteWorkerPoolOrDefault(), 394 scope) 395 if err != nil { 396 logger.Fatal("could not create worker pools", zap.Error(err)) 397 } 398 399 var ( 400 encodingOpts = encoding.NewOptions() 401 m3dbClusters m3.Clusters 402 m3dbPoolWrapper *pools.PoolWrapper 403 ) 404 405 tsdbOpts := m3.NewOptions(encodingOpts). 406 SetTagOptions(tagOptions). 407 SetLookbackDuration(lookbackDuration). 408 SetConsolidationFunc(consolidators.TakeLast). 409 SetReadWorkerPool(readWorkerPool). 410 SetWriteWorkerPool(writeWorkerPool). 411 SetSeriesConsolidationMatchOptions(matchOptions). 412 SetPromConvertOptions(promConvertOptions) 413 414 if runOpts.ApplyCustomTSDBOptions != nil { 415 tsdbOpts, err = runOpts.ApplyCustomTSDBOptions(tsdbOpts, instrumentOptions) 416 if err != nil { 417 logger.Fatal("could not apply ApplyCustomTSDBOptions", zap.Error(err)) 418 } 419 } 420 421 serveOptions := serve.NewOptions(instrumentOptions) 422 for i, transform := range runOpts.AggregatorServerOptions { 423 if opts, err := transform(serveOptions); err != nil { 424 logger.Fatal("could not apply transform", 425 zap.Int("index", i), zap.Error(err)) 426 } else { 427 serveOptions = opts 428 } 429 } 430 431 rwOpts := serveOptions.RWOptions() 432 switch cfg.Backend { 433 case config.GRPCStorageType: 434 // For grpc backend, we need to setup only the grpc client and a storage 435 // accompanying that client. 436 poolWrapper := pools.NewPoolsWrapper( 437 pools.BuildIteratorPools(encodingOpts, pools.BuildIteratorPoolsOptions{})) 438 remoteOpts := config.RemoteOptionsFromConfig(cfg.RPC) 439 remotes, enabled, err := remoteClient(poolWrapper, remoteOpts, 440 tsdbOpts, instrumentOptions) 441 if err != nil { 442 logger.Fatal("unable to setup grpc backend", zap.Error(err)) 443 } 444 if !enabled { 445 logger.Fatal("need remote clients for grpc backend") 446 } 447 448 var ( 449 r = filter.AllowAll 450 w = filter.AllowAll 451 c = filter.CompleteTagsAllowAll 452 ) 453 454 backendStorage = fanout.NewStorage(remotes, r, w, c, 455 tagOptions, tsdbOpts, instrumentOptions) 456 logger.Info("setup grpc backend") 457 458 case config.NoopEtcdStorageType: 459 backendStorage = storage.NewNoopStorage() 460 etcd := cfg.ClusterManagement.Etcd 461 462 if etcd == nil || len(etcd.ETCDClusters) == 0 { 463 logger.Fatal("must specify cluster management config and at least one etcd cluster") 464 } 465 466 opts := etcd.NewOptions() 467 clusterClient, err = etcdclient.NewConfigServiceClient(opts) 468 if err != nil { 469 logger.Fatal("error constructing etcd client", zap.Error(err)) 470 } 471 logger.Info("setup noop storage backend with etcd") 472 473 // Empty backend defaults to M3DB. 474 case "", config.M3DBStorageType: 475 // For m3db backend, we need to make connections to the m3db cluster 476 // which generates a session and use the storage with the session. 477 m3dbClusters, m3dbPoolWrapper, err = initClusters(cfg, runOpts.DBConfig, 478 clusterNamespacesWatcher, runOpts.DBClient, encodingOpts, 479 runOpts.LocalSessionReadyCh, instrumentOptions, 480 tsdbOpts.CustomAdminOptions()) 481 if err != nil { 482 logger.Fatal("unable to init clusters", zap.Error(err)) 483 } 484 485 var cleanup cleanupFn 486 backendStorage, cleanup, err = newM3DBStorage( 487 cfg, m3dbClusters, m3dbPoolWrapper, queryCtxOpts, tsdbOpts, instrumentOptions, 488 ) 489 if err != nil { 490 logger.Fatal("unable to setup m3db backend", zap.Error(err)) 491 } 492 493 defer cleanup() 494 495 etcdConfig, err := resolveEtcdForM3DB(cfg) 496 if err != nil { 497 logger.Fatal("unable to resolve etcd config for m3db backend", zap.Error(err)) 498 } 499 500 logger.Info("configuring downsampler to use with aggregated cluster namespaces", 501 zap.Int("numAggregatedClusterNamespaces", len(m3dbClusters.ClusterNamespaces()))) 502 503 downsampler, clusterClient, err = newDownsamplerAsync(cfg.Downsample, etcdConfig, backendStorage, 504 clusterNamespacesWatcher, tsdbOpts.TagOptions(), clockOpts, instrumentOptions, rwOpts, runOpts, 505 interruptOpts, 506 ) 507 if err != nil { 508 var interruptErr *xos.InterruptError 509 if errors.As(err, &interruptErr) { 510 logger.Warn("interrupt received. closing server", zap.Error(err)) 511 return runResult 512 } 513 logger.Fatal("unable to setup downsampler for m3db backend", zap.Error(err)) 514 } 515 case config.PromRemoteStorageType: 516 opts, err := promremote.NewOptions(cfg.PrometheusRemoteBackend, scope, instrumentOptions.Logger()) 517 if err != nil { 518 logger.Fatal("invalid configuration", zap.Error(err)) 519 } 520 backendStorage, err = promremote.NewStorage(opts) 521 if err != nil { 522 logger.Fatal("unable to setup prom remote backend", zap.Error(err)) 523 } 524 defer func() { 525 if err := backendStorage.Close(); err != nil { 526 logger.Error("error when closing storage", zap.Error(err)) 527 } 528 }() 529 530 logger.Info("configuring downsampler to use with aggregated namespaces", 531 zap.Int("numAggregatedClusterNamespaces", opts.Namespaces().NumAggregatedClusterNamespaces())) 532 err = clusterNamespacesWatcher.Update(opts.Namespaces()) 533 if err != nil { 534 logger.Fatal("unable to update namespaces", zap.Error(err)) 535 } 536 537 downsampler, clusterClient, err = newDownsamplerAsync(cfg.Downsample, cfg.ClusterManagement.Etcd, backendStorage, 538 clusterNamespacesWatcher, tsdbOpts.TagOptions(), clockOpts, instrumentOptions, rwOpts, runOpts, 539 interruptOpts, 540 ) 541 if err != nil { 542 logger.Fatal("unable to setup downsampler for prom remote backend", zap.Error(err)) 543 } 544 default: 545 logger.Fatal("unrecognized backend", zap.String("backend", string(cfg.Backend))) 546 } 547 548 if fn := runOpts.BackendStorageTransform; fn != nil { 549 backendStorage, err = fn(backendStorage, tsdbOpts, instrumentOptions) 550 if err != nil { 551 logger.Fatal("could not apply BackendStorageTransform", zap.Error(err)) 552 } 553 } 554 555 engineOpts := executor.NewEngineOptions(). 556 SetStore(backendStorage). 557 SetLookbackDuration(*cfg.LookbackDuration). 558 SetInstrumentOptions(instrumentOptions. 559 SetMetricsScope(instrumentOptions.MetricsScope().SubScope("engine"))) 560 if fn := runOpts.CustomPromQLParseFunction; fn != nil { 561 engineOpts = engineOpts. 562 SetParseOptions(engineOpts.ParseOptions().SetParseFn(fn)) 563 } 564 565 engine := executor.NewEngine(engineOpts) 566 downsamplerAndWriter, err := newDownsamplerAndWriter( 567 backendStorage, 568 downsampler, 569 cfg.WriteWorkerPoolOrDefault(), 570 instrumentOptions, 571 ) 572 if err != nil { 573 logger.Fatal("unable to create new downsampler and writer", zap.Error(err)) 574 } 575 576 var serviceOptionDefaults []handleroptions3.ServiceOptionsDefault 577 if dbCfg := runOpts.DBConfig; dbCfg != nil { 578 hostID, err := dbCfg.HostIDOrDefault().Resolve() 579 if err != nil { 580 logger.Fatal("could not resolve hostID", 581 zap.Error(err)) 582 } 583 584 discoveryCfg := dbCfg.DiscoveryOrDefault() 585 envCfg, err := discoveryCfg.EnvironmentConfig(hostID) 586 if err != nil { 587 logger.Fatal("could not get env config from discovery config", 588 zap.Error(err)) 589 } 590 591 cluster, err := envCfg.Services.SyncCluster() 592 if err != nil { 593 logger.Fatal("could not resolve embedded db cluster info", 594 zap.Error(err)) 595 } 596 if svcCfg := cluster.Service; svcCfg != nil { 597 serviceOptionDefaults = append(serviceOptionDefaults, 598 handleroptions3.WithDefaultServiceEnvironment(svcCfg.Env)) 599 serviceOptionDefaults = append(serviceOptionDefaults, 600 handleroptions3.WithDefaultServiceZone(svcCfg.Zone)) 601 } 602 } 603 604 var ( 605 graphiteFindFetchOptsBuilder = fetchOptsBuilder 606 graphiteRenderFetchOptsBuilder = fetchOptsBuilder 607 graphiteStorageOpts graphite.M3WrappedStorageOptions 608 ) 609 if cfg.Carbon != nil { 610 graphiteStorageOpts = graphite.M3WrappedStorageOptions{ 611 AggregateNamespacesAllData: cfg.Carbon.AggregateNamespacesAllData, 612 ShiftTimeStart: cfg.Carbon.ShiftTimeStart, 613 ShiftTimeEnd: cfg.Carbon.ShiftTimeEnd, 614 ShiftStepsStart: cfg.Carbon.ShiftStepsStart, 615 ShiftStepsEnd: cfg.Carbon.ShiftStepsEnd, 616 ShiftStepsStartWhenAtResolutionBoundary: cfg.Carbon.ShiftStepsStartWhenAtResolutionBoundary, 617 ShiftStepsEndWhenAtResolutionBoundary: cfg.Carbon.ShiftStepsEndWhenAtResolutionBoundary, 618 ShiftStepsEndWhenStartAtResolutionBoundary: cfg.Carbon.ShiftStepsEndWhenStartAtResolutionBoundary, 619 ShiftStepsStartWhenEndAtResolutionBoundary: cfg.Carbon.ShiftStepsStartWhenEndAtResolutionBoundary, 620 RenderPartialStart: cfg.Carbon.RenderPartialStart, 621 RenderPartialEnd: cfg.Carbon.RenderPartialEnd, 622 RenderSeriesAllNaNs: cfg.Carbon.RenderSeriesAllNaNs, 623 CompileEscapeAllNotOnlyQuotes: cfg.Carbon.CompileEscapeAllNotOnlyQuotes, 624 FindResultsIncludeBothExpandableAndLeaf: cfg.Carbon.FindResultsIncludeBothExpandableAndLeaf, 625 } 626 if limits := cfg.Carbon.LimitsFind; limits != nil { 627 fetchOptsBuilderLimitsOpts := limits.PerQuery.AsFetchOptionsBuilderLimitsOptions() 628 graphiteFindFetchOptsBuilder, err = handleroptions.NewFetchOptionsBuilder( 629 handleroptions.FetchOptionsBuilderOptions{ 630 Limits: fetchOptsBuilderLimitsOpts, 631 RestrictByTag: storageRestrictByTags, 632 Timeout: timeout, 633 }) 634 if err != nil { 635 logger.Fatal("could not set graphite find fetch options parser", zap.Error(err)) 636 } 637 } 638 if limits := cfg.Carbon.LimitsRender; limits != nil { 639 fetchOptsBuilderLimitsOpts := limits.PerQuery.AsFetchOptionsBuilderLimitsOptions() 640 graphiteRenderFetchOptsBuilder, err = handleroptions.NewFetchOptionsBuilder( 641 handleroptions.FetchOptionsBuilderOptions{ 642 Limits: fetchOptsBuilderLimitsOpts, 643 RestrictByTag: storageRestrictByTags, 644 Timeout: timeout, 645 }) 646 if err != nil { 647 logger.Fatal("could not set graphite find fetch options parser", zap.Error(err)) 648 } 649 } 650 } 651 652 defaultPrometheusEngine, err := newPromQLEngine(lookbackDuration, cfg, prometheusEngineRegistry, instrumentOptions) 653 if err != nil { 654 logger.Fatal("unable to create PromQL engine", zap.Error(err)) 655 } 656 prometheusEngineFn := func(lookbackDuration time.Duration) (*prometheuspromql.Engine, error) { 657 // NB: use nil metric registry to avoid duplicate metric registration when creating multiple engines 658 return newPromQLEngine(lookbackDuration, cfg, nil, instrumentOptions) 659 } 660 661 enginesByLookback, err := createEnginesWithResolutionBasedLookbacks( 662 lookbackDuration, 663 defaultPrometheusEngine, 664 runOpts.Config.Clusters, 665 cfg.Middleware.Prometheus.ResolutionMultiplier, 666 prometheusEngineFn, 667 ) 668 if err != nil { 669 logger.Fatal("failed creating PromgQL engines with resolution based lookback durations", zap.Error(err)) 670 } 671 engineCache := promqlengine.NewCache(enginesByLookback, prometheusEngineFn) 672 673 handlerOptions, err := options.NewHandlerOptions(downsamplerAndWriter, 674 tagOptions, engine, engineCache.Get, m3dbClusters, clusterClient, cfg, 675 runOpts.DBConfig, fetchOptsBuilder, graphiteFindFetchOptsBuilder, graphiteRenderFetchOptsBuilder, 676 queryCtxOpts, instrumentOptions, cpuProfileDuration, 677 serviceOptionDefaults, httpd.NewQueryRouter(), httpd.NewQueryRouter(), 678 graphiteStorageOpts, tsdbOpts, httpd.NewGraphiteRenderRouter(), httpd.NewGraphiteFindRouter(), lookbackDuration) 679 if err != nil { 680 logger.Fatal("unable to set up handler options", zap.Error(err)) 681 } 682 683 var customHandlerOpts options.CustomHandlerOptions 684 if runOpts.CustomHandlerOptions != nil { 685 customHandlerOpts, err = runOpts.CustomHandlerOptions(instrumentOptions) 686 if err != nil { 687 logger.Fatal("could not create custom handlers", zap.Error(err)) 688 } 689 } 690 691 if fn := customHandlerOpts.OptionTransformFn; fn != nil { 692 handlerOptions = fn(handlerOptions) 693 } 694 695 customHandlers := customHandlerOpts.CustomHandlers 696 handler := httpd.NewHandler(handlerOptions, cfg.Middleware, customHandlers...) 697 if err := handler.RegisterRoutes(); err != nil { 698 logger.Fatal("unable to register routes", zap.Error(err)) 699 } 700 701 listenAddress := cfg.ListenAddressOrDefault() 702 srvHandler := handler.Router() 703 if cfg.HTTP.EnableH2C { 704 srvHandler = h2c.NewHandler(handler.Router(), &http2.Server{}) 705 } 706 srv := &http.Server{Addr: listenAddress, Handler: srvHandler} 707 defer func() { 708 logger.Info("closing server") 709 if err := srv.Shutdown(context.Background()); err != nil { 710 logger.Error("error closing server", zap.Error(err)) 711 } 712 }() 713 714 listener, err := listenerOpts.Listen("tcp", listenAddress) 715 if err != nil { 716 logger.Fatal("unable to listen on listen address", 717 zap.String("address", listenAddress), 718 zap.Error(err)) 719 } 720 if runOpts.ListenerCh != nil { 721 runOpts.ListenerCh <- listener 722 } 723 go func() { 724 logger.Info("starting API server", zap.Stringer("address", listener.Addr())) 725 if err := srv.Serve(listener); err != nil && err != http.ErrServerClosed { 726 logger.Fatal("server serve error", 727 zap.String("address", listenAddress), 728 zap.Error(err)) 729 } 730 }() 731 732 if cfg.Ingest != nil { 733 logger.Info("starting m3msg server", 734 zap.String("address", cfg.Ingest.M3Msg.Server.ListenAddress)) 735 ingester, err := cfg.Ingest.Ingester.NewIngester(backendStorage, 736 tagOptions, instrumentOptions) 737 if err != nil { 738 logger.Fatal("unable to create ingester", zap.Error(err)) 739 } 740 741 server, err := cfg.Ingest.M3Msg.NewServer( 742 ingester.Ingest, rwOpts, 743 instrumentOptions.SetMetricsScope(scope.SubScope("ingest-m3msg"))) 744 if err != nil { 745 logger.Fatal("unable to create m3msg server", zap.Error(err)) 746 } 747 748 listener, err := listenerOpts.Listen("tcp", cfg.Ingest.M3Msg.Server.ListenAddress) 749 if err != nil { 750 logger.Fatal("unable to open m3msg server", zap.Error(err)) 751 } 752 753 if runOpts.M3MsgListenerCh != nil { 754 runOpts.M3MsgListenerCh <- listener 755 } 756 757 if err := server.Serve(listener); err != nil { 758 logger.Fatal("unable to listen on ingest server", zap.Error(err)) 759 } 760 761 logger.Info("started m3msg server", zap.Stringer("addr", listener.Addr())) 762 defer server.Close() 763 } else { 764 logger.Info("no m3msg server configured") 765 } 766 767 if cfg.Carbon != nil && cfg.Carbon.Ingester != nil { 768 server := startCarbonIngestion(*cfg.Carbon.Ingester, listenerOpts, 769 instrumentOptions, logger, m3dbClusters, clusterNamespacesWatcher, 770 downsamplerAndWriter) 771 defer server.Close() 772 } 773 774 // Stop our async watch and now block waiting for the interrupt. 775 intWatchCancel() 776 select { 777 case <-interruptOpts.InterruptedCh: 778 logger.Warn("interrupt already received. closing") 779 default: 780 xos.WaitForInterrupt(logger, interruptOpts) 781 } 782 783 return runResult 784 } 785 786 // make connections to the m3db cluster(s) and generate sessions for those clusters along with the storage 787 func newM3DBStorage(cfg config.Configuration, clusters m3.Clusters, poolWrapper *pools.PoolWrapper, 788 queryContextOptions models.QueryContextOptions, tsdbOpts m3.Options, 789 instrumentOptions instrument.Options, 790 ) (storage.Storage, cleanupFn, error) { 791 fanoutStorage, storageCleanup, err := newStorages(clusters, cfg, 792 poolWrapper, queryContextOptions, tsdbOpts, instrumentOptions) 793 if err != nil { 794 return nil, nil, errors.Wrap(err, "unable to set up storages") 795 } 796 logger := instrumentOptions.Logger() 797 cleanup := func() error { 798 lastErr := storageCleanup() 799 // Don't want to quit on the first error since the full cleanup is important 800 if lastErr != nil { 801 logger.Error("error during storage cleanup", zap.Error(lastErr)) 802 } 803 804 if err := clusters.Close(); err != nil { 805 lastErr = errors.Wrap(err, "unable to close M3DB cluster sessions") 806 // Make sure the previous error is at least logged 807 logger.Error("error during cluster cleanup", zap.Error(err)) 808 } 809 810 return lastErr 811 } 812 813 return fanoutStorage, cleanup, err 814 } 815 816 func resolveEtcdForM3DB(cfg config.Configuration) (*etcdclient.Configuration, error) { 817 etcdConfig := cfg.ClusterManagement.Etcd 818 if etcdConfig == nil && len(cfg.Clusters) == 1 && 819 cfg.Clusters[0].Client.EnvironmentConfig != nil { 820 syncCfg, err := cfg.Clusters[0].Client.EnvironmentConfig.Services.SyncCluster() 821 if err != nil { 822 return nil, errors.Wrap(err, "unable to get etcd sync cluster config") 823 } 824 etcdConfig = syncCfg.Service 825 } 826 return etcdConfig, nil 827 } 828 829 func newDownsamplerAsync( 830 cfg downsample.Configuration, etcdCfg *etcdclient.Configuration, storage storage.Appender, 831 clusterNamespacesWatcher m3.ClusterNamespacesWatcher, tagOptions models.TagOptions, clockOpts clock.Options, 832 instrumentOptions instrument.Options, rwOpts xio.Options, runOpts RunOptions, interruptOpts xos.InterruptOptions, 833 ) (downsample.Downsampler, clusterclient.Client, error) { 834 var ( 835 clusterClient clusterclient.Client 836 clusterClientWaitCh <-chan struct{} 837 err error 838 ) 839 if clusterClientCh := runOpts.ClusterClient; clusterClientCh != nil { 840 // Only use a cluster client if we are going to receive one, that 841 // way passing nil to httpd NewHandler disables the endpoints entirely 842 clusterClientDoneCh := make(chan struct{}, 1) 843 clusterClientWaitCh = clusterClientDoneCh 844 clusterClient = m3dbcluster.NewAsyncClient(func() (clusterclient.Client, error) { 845 return <-clusterClientCh, nil 846 }, clusterClientDoneCh) 847 } else if etcdCfg != nil { 848 // We resolved an etcd configuration for cluster management endpoints 849 var ( 850 clusterSvcClientOpts = etcdCfg.NewOptions() 851 err error 852 ) 853 clusterClient, err = etcdclient.NewConfigServiceClient(clusterSvcClientOpts) 854 if err != nil { 855 return nil, nil, errors.Wrap(err, "unable to create cluster management etcd client") 856 } 857 } else if cfg.RemoteAggregator == nil { 858 // NB(antanas): M3 Coordinator with in process aggregator can run with in memory cluster client. 859 instrumentOptions.Logger().Info("no etcd config and no remote aggregator - will run with in memory cluster client") 860 clusterClient = memcluster.New(kv.NewOverrideOptions()) 861 } else { 862 return nil, nil, fmt.Errorf("no configured cluster management config, must set this config for remote aggregator") 863 } 864 865 newDownsamplerFn := func() (downsample.Downsampler, error) { 866 ds, err := newDownsampler( 867 cfg, clusterClient, 868 storage, clusterNamespacesWatcher, 869 tagOptions, clockOpts, instrumentOptions, rwOpts, runOpts.ApplyCustomRuleStore, 870 interruptOpts.InterruptedCh) 871 if err != nil { 872 return nil, err 873 } 874 875 // Notify the downsampler ready channel that 876 // the downsampler has now been created and is ready. 877 if runOpts.DownsamplerReadyCh != nil { 878 runOpts.DownsamplerReadyCh <- struct{}{} 879 } 880 881 return ds, nil 882 } 883 884 if clusterClientWaitCh != nil { 885 // Need to wait before constructing and instead return an async downsampler 886 // since the cluster client will return errors until it's initialized itself 887 // and will fail constructing the downsampler consequently 888 downsampler := downsample.NewAsyncDownsampler(func() (downsample.Downsampler, error) { 889 <-clusterClientWaitCh 890 return newDownsamplerFn() 891 }, nil) 892 return downsampler, clusterClient, err 893 } else { 894 // Otherwise we already have a client and can immediately construct the downsampler 895 downsampler, err := newDownsamplerFn() 896 return downsampler, clusterClient, err 897 } 898 } 899 900 func newDownsampler( 901 cfg downsample.Configuration, 902 clusterClient clusterclient.Client, 903 storage storage.Appender, 904 clusterNamespacesWatcher m3.ClusterNamespacesWatcher, 905 tagOptions models.TagOptions, 906 clockOpts clock.Options, 907 instrumentOpts instrument.Options, 908 rwOpts xio.Options, 909 applyCustomRuleStore downsample.CustomRuleStoreFn, 910 interruptedCh <-chan struct{}, 911 ) (downsample.Downsampler, error) { 912 // Namespace the downsampler metrics. 913 instrumentOpts = instrumentOpts.SetMetricsScope( 914 instrumentOpts.MetricsScope().SubScope("downsampler")) 915 916 var kvStore kv.Store 917 var err error 918 919 if applyCustomRuleStore == nil { 920 kvStore, err = clusterClient.KV() 921 if err != nil { 922 return nil, errors.Wrap(err, "unable to create KV store from the "+ 923 "cluster management config client") 924 } 925 } else { 926 kvStore, err = applyCustomRuleStore(clusterClient, instrumentOpts) 927 if err != nil { 928 return nil, errors.Wrap(err, "unable to apply custom rule store") 929 } 930 } 931 932 tagEncoderOptions := serialize.NewTagEncoderOptions() 933 tagDecoderOptions := serialize.NewTagDecoderOptions(serialize.TagDecoderOptionsConfig{}) 934 tagEncoderPoolOptions := pool.NewObjectPoolOptions(). 935 SetInstrumentOptions(instrumentOpts. 936 SetMetricsScope(instrumentOpts.MetricsScope(). 937 SubScope("tag-encoder-pool"))) 938 tagDecoderPoolOptions := pool.NewObjectPoolOptions(). 939 SetInstrumentOptions(instrumentOpts. 940 SetMetricsScope(instrumentOpts.MetricsScope(). 941 SubScope("tag-decoder-pool"))) 942 metricsAppenderPoolOptions := pool.NewObjectPoolOptions(). 943 SetInstrumentOptions(instrumentOpts. 944 SetMetricsScope(instrumentOpts.MetricsScope(). 945 SubScope("metrics-appender-pool"))) 946 947 downsampler, err := cfg.NewDownsampler(downsample.DownsamplerOptions{ 948 Storage: storage, 949 ClusterClient: clusterClient, 950 RulesKVStore: kvStore, 951 ClusterNamespacesWatcher: clusterNamespacesWatcher, 952 ClockOptions: clockOpts, 953 InstrumentOptions: instrumentOpts, 954 TagEncoderOptions: tagEncoderOptions, 955 TagDecoderOptions: tagDecoderOptions, 956 TagEncoderPoolOptions: tagEncoderPoolOptions, 957 TagDecoderPoolOptions: tagDecoderPoolOptions, 958 TagOptions: tagOptions, 959 MetricsAppenderPoolOptions: metricsAppenderPoolOptions, 960 RWOptions: rwOpts, 961 InterruptedCh: interruptedCh, 962 }) 963 if err != nil { 964 return nil, fmt.Errorf("unable to create downsampler: %w", err) 965 } 966 967 return downsampler, nil 968 } 969 970 func initClusters( 971 cfg config.Configuration, 972 dbCfg *dbconfig.DBConfiguration, 973 clusterNamespacesWatcher m3.ClusterNamespacesWatcher, 974 dbClientCh <-chan client.Client, 975 encodingOpts encoding.Options, 976 localSessionReadyCh chan struct{}, 977 instrumentOpts instrument.Options, 978 customAdminOptions []client.CustomAdminOption, 979 ) (m3.Clusters, *pools.PoolWrapper, error) { 980 instrumentOpts = instrumentOpts. 981 SetMetricsScope(instrumentOpts.MetricsScope().SubScope("m3db-client")) 982 983 var ( 984 logger = instrumentOpts.Logger() 985 clusters m3.Clusters 986 poolWrapper *pools.PoolWrapper 987 staticNamespaceConfig bool 988 err error 989 ) 990 if len(cfg.Clusters) > 0 { 991 for _, clusterCfg := range cfg.Clusters { 992 if len(clusterCfg.Namespaces) > 0 { 993 staticNamespaceConfig = true 994 break 995 } 996 } 997 opts := m3.ClustersStaticConfigurationOptions{ 998 AsyncSessions: true, 999 CustomAdminOptions: customAdminOptions, 1000 EncodingOptions: encodingOpts, 1001 } 1002 if staticNamespaceConfig { 1003 clusters, err = cfg.Clusters.NewStaticClusters(instrumentOpts, opts, clusterNamespacesWatcher) 1004 } else { 1005 // No namespaces defined in config -- pull namespace data from etcd. 1006 clusters, err = cfg.Clusters.NewDynamicClusters(instrumentOpts, opts, clusterNamespacesWatcher) 1007 } 1008 if err != nil { 1009 return nil, nil, errors.Wrap(err, "unable to connect to clusters") 1010 } 1011 1012 poolWrapper = pools.NewPoolsWrapper( 1013 pools.BuildIteratorPools(encodingOpts, pools.BuildIteratorPoolsOptions{})) 1014 } else { 1015 localCfg := cfg.Local 1016 if localCfg == nil { 1017 localCfg = &config.LocalConfiguration{} 1018 } 1019 if len(localCfg.Namespaces) > 0 { 1020 staticNamespaceConfig = true 1021 } 1022 1023 if dbClientCh == nil { 1024 return nil, nil, errors.New("no clusters configured and not running local cluster") 1025 } 1026 1027 if localSessionReadyCh == nil { 1028 localSessionReadyCh = make(chan struct{}) 1029 } 1030 session := m3db.NewAsyncSession(func() (client.Client, error) { 1031 return <-dbClientCh, nil 1032 }, localSessionReadyCh) 1033 1034 clusterStaticConfig := m3.ClusterStaticConfiguration{ 1035 Namespaces: localCfg.Namespaces, 1036 } 1037 if !staticNamespaceConfig { 1038 if dbCfg == nil { 1039 return nil, nil, errors.New("environment config required when dynamically fetching namespaces") 1040 } 1041 1042 hostID, err := dbCfg.HostIDOrDefault().Resolve() 1043 if err != nil { 1044 logger.Fatal("could not resolve hostID", 1045 zap.Error(err)) 1046 } 1047 1048 discoveryCfg := dbCfg.DiscoveryOrDefault() 1049 envCfg, err := discoveryCfg.EnvironmentConfig(hostID) 1050 if err != nil { 1051 logger.Fatal("could not get env config from discovery config", 1052 zap.Error(err)) 1053 } 1054 1055 clusterStaticConfig.Client = client.Configuration{EnvironmentConfig: &envCfg} 1056 } 1057 1058 clustersCfg := m3.ClustersStaticConfiguration{clusterStaticConfig} 1059 cfgOptions := m3.ClustersStaticConfigurationOptions{ 1060 ProvidedSession: session, 1061 CustomAdminOptions: customAdminOptions, 1062 EncodingOptions: encodingOpts, 1063 } 1064 1065 if staticNamespaceConfig { 1066 clusters, err = clustersCfg.NewStaticClusters(instrumentOpts, cfgOptions, clusterNamespacesWatcher) 1067 } else { 1068 clusters, err = clustersCfg.NewDynamicClusters(instrumentOpts, cfgOptions, clusterNamespacesWatcher) 1069 } 1070 if err != nil { 1071 return nil, nil, errors.Wrap(err, "unable to connect to clusters") 1072 } 1073 1074 poolWrapper = pools.NewAsyncPoolsWrapper() 1075 go func() { 1076 <-localSessionReadyCh 1077 poolWrapper.Init(session.IteratorPools()) 1078 }() 1079 } 1080 1081 for _, namespace := range clusters.ClusterNamespaces() { 1082 logger.Info("resolved cluster namespace", 1083 zap.String("namespace", namespace.NamespaceID().String())) 1084 } 1085 1086 return clusters, poolWrapper, nil 1087 } 1088 1089 func newStorages( 1090 clusters m3.Clusters, 1091 cfg config.Configuration, 1092 poolWrapper *pools.PoolWrapper, 1093 queryContextOptions models.QueryContextOptions, 1094 opts m3.Options, 1095 instrumentOpts instrument.Options, 1096 ) (storage.Storage, cleanupFn, error) { 1097 var ( 1098 logger = instrumentOpts.Logger() 1099 cleanup = func() error { return nil } 1100 ) 1101 1102 localStorage, err := m3.NewStorage(clusters, opts, instrumentOpts) 1103 if err != nil { 1104 return nil, nil, err 1105 } 1106 1107 stores := []storage.Storage{localStorage} 1108 remoteEnabled := false 1109 remoteOpts := config.RemoteOptionsFromConfig(cfg.RPC) 1110 if remoteOpts.ServeEnabled() { 1111 logger.Info("rpc serve enabled") 1112 server, err := startGRPCServer(localStorage, queryContextOptions, 1113 poolWrapper, remoteOpts, instrumentOpts) 1114 if err != nil { 1115 return nil, nil, err 1116 } 1117 1118 cleanup = func() error { 1119 server.GracefulStop() 1120 return nil 1121 } 1122 } 1123 1124 if remoteOpts.ListenEnabled() { 1125 remoteStorages, enabled, err := remoteClient(poolWrapper, remoteOpts, 1126 opts, instrumentOpts) 1127 if err != nil { 1128 return nil, nil, err 1129 } 1130 1131 if enabled { 1132 stores = append(stores, remoteStorages...) 1133 remoteEnabled = enabled 1134 } 1135 } 1136 1137 readFilter := filter.LocalOnly 1138 writeFilter := filter.LocalOnly 1139 completeTagsFilter := filter.CompleteTagsLocalOnly 1140 if remoteEnabled { 1141 // If remote enabled, allow all for read and complete tags 1142 // but continue to only send writes locally 1143 readFilter = filter.AllowAll 1144 completeTagsFilter = filter.CompleteTagsAllowAll 1145 } 1146 1147 switch cfg.Filter.Read { 1148 case config.FilterLocalOnly: 1149 readFilter = filter.LocalOnly 1150 case config.FilterRemoteOnly: 1151 readFilter = filter.RemoteOnly 1152 case config.FilterAllowAll: 1153 readFilter = filter.AllowAll 1154 case config.FilterAllowNone: 1155 readFilter = filter.AllowNone 1156 } 1157 1158 switch cfg.Filter.Write { 1159 case config.FilterLocalOnly: 1160 writeFilter = filter.LocalOnly 1161 case config.FilterRemoteOnly: 1162 writeFilter = filter.RemoteOnly 1163 case config.FilterAllowAll: 1164 writeFilter = filter.AllowAll 1165 case config.FilterAllowNone: 1166 writeFilter = filter.AllowNone 1167 } 1168 1169 switch cfg.Filter.CompleteTags { 1170 case config.FilterLocalOnly: 1171 completeTagsFilter = filter.CompleteTagsLocalOnly 1172 case config.FilterRemoteOnly: 1173 completeTagsFilter = filter.CompleteTagsRemoteOnly 1174 case config.FilterAllowAll: 1175 completeTagsFilter = filter.CompleteTagsAllowAll 1176 case config.FilterAllowNone: 1177 completeTagsFilter = filter.CompleteTagsAllowNone 1178 } 1179 1180 fanoutStorage := fanout.NewStorage(stores, readFilter, writeFilter, 1181 completeTagsFilter, opts.TagOptions(), opts, instrumentOpts) 1182 return fanoutStorage, cleanup, nil 1183 } 1184 1185 func remoteZoneStorage( 1186 zone config.Remote, 1187 poolWrapper *pools.PoolWrapper, 1188 opts m3.Options, 1189 instrumentOpts instrument.Options, 1190 ) (storage.Storage, error) { 1191 if len(zone.Addresses) == 0 { 1192 // No addresses; skip. 1193 return nil, nil 1194 } 1195 1196 client, err := tsdbremote.NewGRPCClient(zone.Name, zone.Addresses, 1197 poolWrapper, opts, instrumentOpts) 1198 if err != nil { 1199 return nil, err 1200 } 1201 1202 remoteOpts := remote.Options{ 1203 Name: zone.Name, 1204 ErrorBehavior: zone.ErrorBehavior, 1205 } 1206 1207 remoteStorage := remote.NewStorage(client, remoteOpts) 1208 return remoteStorage, nil 1209 } 1210 1211 func remoteClient( 1212 poolWrapper *pools.PoolWrapper, 1213 remoteOpts config.RemoteOptions, 1214 opts m3.Options, 1215 instrumentOpts instrument.Options, 1216 ) ([]storage.Storage, bool, error) { 1217 logger := instrumentOpts.Logger() 1218 remotes := remoteOpts.Remotes() 1219 remoteStores := make([]storage.Storage, 0, len(remotes)) 1220 for _, zone := range remotes { 1221 logger.Info( 1222 "creating RPC client with remotes", 1223 zap.String("name", zone.Name), 1224 zap.Strings("addresses", zone.Addresses), 1225 ) 1226 1227 remote, err := remoteZoneStorage(zone, poolWrapper, opts, 1228 instrumentOpts) 1229 if err != nil { 1230 return nil, false, err 1231 } 1232 1233 remoteStores = append(remoteStores, remote) 1234 } 1235 1236 return remoteStores, true, nil 1237 } 1238 1239 func startGRPCServer( 1240 storage m3.Storage, 1241 queryContextOptions models.QueryContextOptions, 1242 poolWrapper *pools.PoolWrapper, 1243 opts config.RemoteOptions, 1244 instrumentOpts instrument.Options, 1245 ) (*grpc.Server, error) { 1246 logger := instrumentOpts.Logger() 1247 1248 logger.Info("creating gRPC server") 1249 server := tsdbremote.NewGRPCServer(storage, 1250 queryContextOptions, poolWrapper, instrumentOpts) 1251 1252 if opts.ReflectionEnabled() { 1253 reflection.Register(server) 1254 } 1255 1256 logger.Info("gRPC server reflection configured", 1257 zap.Bool("enabled", opts.ReflectionEnabled())) 1258 1259 listener, err := net.Listen("tcp", opts.ServeAddress()) 1260 if err != nil { 1261 return nil, err 1262 } 1263 go func() { 1264 if err := server.Serve(listener); err != nil { 1265 logger.Error("error from serving gRPC server", zap.Error(err)) 1266 } 1267 }() 1268 1269 return server, nil 1270 } 1271 1272 func startCarbonIngestion( 1273 ingesterCfg config.CarbonIngesterConfiguration, 1274 listenerOpts xnet.ListenerOptions, 1275 iOpts instrument.Options, 1276 logger *zap.Logger, 1277 m3dbClusters m3.Clusters, 1278 clusterNamespacesWatcher m3.ClusterNamespacesWatcher, 1279 downsamplerAndWriter ingest.DownsamplerAndWriter, 1280 ) xserver.Server { 1281 logger.Info("carbon ingestion enabled, configuring ingester") 1282 1283 // Setup worker pool. 1284 var ( 1285 carbonIOpts = iOpts.SetMetricsScope( 1286 iOpts.MetricsScope().SubScope("ingest-carbon")) 1287 carbonWorkerPoolOpts xsync.PooledWorkerPoolOptions 1288 carbonWorkerPoolSize int 1289 ) 1290 if ingesterCfg.MaxConcurrency > 0 { 1291 // Use a bounded worker pool if they requested a specific maximum concurrency. 1292 carbonWorkerPoolOpts = xsync.NewPooledWorkerPoolOptions(). 1293 SetGrowOnDemand(false). 1294 SetInstrumentOptions(carbonIOpts) 1295 carbonWorkerPoolSize = ingesterCfg.MaxConcurrency 1296 } else { 1297 carbonWorkerPoolOpts = xsync.NewPooledWorkerPoolOptions(). 1298 SetGrowOnDemand(true). 1299 SetKillWorkerProbability(0.001) 1300 carbonWorkerPoolSize = defaultCarbonIngesterWorkerPoolSize 1301 } 1302 workerPool, err := xsync.NewPooledWorkerPool(carbonWorkerPoolSize, carbonWorkerPoolOpts) 1303 if err != nil { 1304 logger.Fatal("unable to create worker pool for carbon ingester", zap.Error(err)) 1305 } 1306 workerPool.Init() 1307 1308 if m3dbClusters == nil { 1309 logger.Fatal("carbon ingestion is only supported when connecting to M3DB clusters directly") 1310 } 1311 1312 // Create ingester. 1313 ingester, err := ingestcarbon.NewIngester( 1314 downsamplerAndWriter, clusterNamespacesWatcher, ingestcarbon.Options{ 1315 InstrumentOptions: carbonIOpts, 1316 WorkerPool: workerPool, 1317 IngesterConfig: ingesterCfg, 1318 }) 1319 if err != nil { 1320 logger.Fatal("unable to create carbon ingester", zap.Error(err)) 1321 } 1322 1323 // Start server. 1324 var ( 1325 serverOpts = xserver.NewOptions(). 1326 SetInstrumentOptions(carbonIOpts). 1327 SetListenerOptions(listenerOpts) 1328 carbonListenAddress = ingesterCfg.ListenAddressOrDefault() 1329 carbonServer = xserver.NewServer(carbonListenAddress, ingester, serverOpts) 1330 ) 1331 if strings.TrimSpace(carbonListenAddress) == "" { 1332 logger.Fatal("no listen address specified for carbon ingester") 1333 } 1334 1335 logger.Info("starting carbon ingestion server", zap.String("listenAddress", carbonListenAddress)) 1336 err = carbonServer.ListenAndServe() 1337 if err != nil { 1338 logger.Fatal("unable to start carbon ingestion server at listen address", 1339 zap.String("listenAddress", carbonListenAddress), zap.Error(err)) 1340 } 1341 1342 logger.Info("started carbon ingestion server", zap.String("listenAddress", carbonListenAddress)) 1343 1344 return carbonServer 1345 } 1346 1347 func newDownsamplerAndWriter( 1348 storage storage.Storage, 1349 downsampler downsample.Downsampler, 1350 workerPoolPolicy xconfig.WorkerPoolPolicy, 1351 iOpts instrument.Options, 1352 ) (ingest.DownsamplerAndWriter, error) { 1353 // Make sure the downsampler and writer gets its own PooledWorkerPool and that its not shared with any other 1354 // codepaths because PooledWorkerPools can deadlock if used recursively. 1355 downAndWriterWorkerPoolOpts, writePoolSize := workerPoolPolicy.Options() 1356 downAndWriterWorkerPoolOpts = downAndWriterWorkerPoolOpts.SetInstrumentOptions(iOpts. 1357 SetMetricsScope(iOpts.MetricsScope().SubScope("ingest-writer-worker-pool"))) 1358 downAndWriteWorkerPool, err := xsync.NewPooledWorkerPool(writePoolSize, downAndWriterWorkerPoolOpts) 1359 if err != nil { 1360 return nil, err 1361 } 1362 downAndWriteWorkerPool.Init() 1363 1364 return ingest.NewDownsamplerAndWriter(storage, downsampler, downAndWriteWorkerPool, iOpts), nil 1365 } 1366 1367 func newPromQLEngine( 1368 lookbackDelta time.Duration, 1369 cfg config.Configuration, 1370 registry extprom.Registerer, 1371 instrumentOpts instrument.Options, 1372 ) (*prometheuspromql.Engine, error) { 1373 if lookbackDelta < 0 { 1374 return nil, errors.New("lookbackDelta cannot be negative") 1375 } 1376 1377 instrumentOpts.Logger().Debug("creating new PromQL engine", zap.Duration("lookbackDelta", lookbackDelta)) 1378 var ( 1379 kitLogger = kitlogzap.NewZapSugarLogger(instrumentOpts.Logger(), zapcore.InfoLevel) 1380 opts = prometheuspromql.EngineOpts{ 1381 Logger: log.With(kitLogger, "component", "prometheus_engine"), 1382 Reg: registry, 1383 MaxSamples: cfg.Query.Prometheus.MaxSamplesPerQueryOrDefault(), 1384 Timeout: cfg.Query.TimeoutOrDefault(), 1385 LookbackDelta: lookbackDelta, 1386 NoStepSubqueryIntervalFn: func(rangeMillis int64) int64 { 1387 return durationMilliseconds(1 * time.Minute) 1388 }, 1389 } 1390 ) 1391 return prometheuspromql.NewEngine(opts), nil 1392 } 1393 1394 func createEnginesWithResolutionBasedLookbacks( 1395 defaultLookback time.Duration, 1396 defaultEngine *prometheuspromql.Engine, 1397 clusters m3.ClustersStaticConfiguration, 1398 resolutionMultiplier int, 1399 prometheusEngineFn func(time.Duration) (*prometheuspromql.Engine, error), 1400 ) (map[time.Duration]*prometheuspromql.Engine, error) { 1401 enginesByLookback := make(map[time.Duration]*prometheuspromql.Engine) 1402 enginesByLookback[defaultLookback] = defaultEngine 1403 if resolutionMultiplier > 0 { 1404 for _, cluster := range clusters { 1405 for _, ns := range cluster.Namespaces { 1406 if res := ns.Resolution; res > 0 { 1407 resolutionBasedLookback := res * time.Duration(resolutionMultiplier) 1408 if _, ok := enginesByLookback[resolutionBasedLookback]; !ok { 1409 eng, err := prometheusEngineFn(resolutionBasedLookback) 1410 if err != nil { 1411 return nil, err 1412 } 1413 enginesByLookback[resolutionBasedLookback] = eng 1414 } 1415 } 1416 } 1417 } 1418 } 1419 return enginesByLookback, nil 1420 } 1421 1422 func durationMilliseconds(d time.Duration) int64 { 1423 return int64(d / (time.Millisecond / time.Nanosecond)) 1424 }