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