github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/server/server.go (about) 1 // Copyright (c) 2017 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 contains the code to run the dbnode server. 22 package server 23 24 import ( 25 "context" 26 "errors" 27 "fmt" 28 "io" 29 "math" 30 "net/http" 31 "os" 32 "path" 33 "runtime" 34 "runtime/debug" 35 "strings" 36 "sync" 37 "time" 38 39 clusterclient "github.com/m3db/m3/src/cluster/client" 40 "github.com/m3db/m3/src/cluster/client/etcd" 41 "github.com/m3db/m3/src/cluster/generated/proto/commonpb" 42 "github.com/m3db/m3/src/cluster/generated/proto/kvpb" 43 "github.com/m3db/m3/src/cluster/kv" 44 "github.com/m3db/m3/src/cluster/placement" 45 "github.com/m3db/m3/src/cluster/placementhandler" 46 "github.com/m3db/m3/src/cluster/placementhandler/handleroptions" 47 "github.com/m3db/m3/src/cmd/services/m3dbnode/config" 48 "github.com/m3db/m3/src/dbnode/client" 49 "github.com/m3db/m3/src/dbnode/encoding" 50 "github.com/m3db/m3/src/dbnode/encoding/m3tsz" 51 "github.com/m3db/m3/src/dbnode/encoding/proto" 52 "github.com/m3db/m3/src/dbnode/environment" 53 "github.com/m3db/m3/src/dbnode/kvconfig" 54 "github.com/m3db/m3/src/dbnode/namespace" 55 hjcluster "github.com/m3db/m3/src/dbnode/network/server/httpjson/cluster" 56 hjnode "github.com/m3db/m3/src/dbnode/network/server/httpjson/node" 57 "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift" 58 ttcluster "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift/cluster" 59 ttnode "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift/node" 60 "github.com/m3db/m3/src/dbnode/persist/fs" 61 "github.com/m3db/m3/src/dbnode/persist/fs/commitlog" 62 "github.com/m3db/m3/src/dbnode/ratelimit" 63 "github.com/m3db/m3/src/dbnode/retention" 64 m3dbruntime "github.com/m3db/m3/src/dbnode/runtime" 65 "github.com/m3db/m3/src/dbnode/sharding" 66 "github.com/m3db/m3/src/dbnode/storage" 67 "github.com/m3db/m3/src/dbnode/storage/block" 68 "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" 69 "github.com/m3db/m3/src/dbnode/storage/cluster" 70 "github.com/m3db/m3/src/dbnode/storage/index" 71 "github.com/m3db/m3/src/dbnode/storage/limits" 72 "github.com/m3db/m3/src/dbnode/storage/limits/permits" 73 "github.com/m3db/m3/src/dbnode/storage/series" 74 "github.com/m3db/m3/src/dbnode/topology" 75 "github.com/m3db/m3/src/dbnode/ts" 76 "github.com/m3db/m3/src/dbnode/ts/writes" 77 xtchannel "github.com/m3db/m3/src/dbnode/x/tchannel" 78 "github.com/m3db/m3/src/dbnode/x/xio" 79 "github.com/m3db/m3/src/dbnode/x/xpool" 80 m3ninxindex "github.com/m3db/m3/src/m3ninx/index" 81 "github.com/m3db/m3/src/m3ninx/postings" 82 "github.com/m3db/m3/src/m3ninx/postings/roaring" 83 "github.com/m3db/m3/src/x/clock" 84 xconfig "github.com/m3db/m3/src/x/config" 85 xcontext "github.com/m3db/m3/src/x/context" 86 xdebug "github.com/m3db/m3/src/x/debug" 87 extdebug "github.com/m3db/m3/src/x/debug/ext" 88 xdocs "github.com/m3db/m3/src/x/docs" 89 "github.com/m3db/m3/src/x/ident" 90 "github.com/m3db/m3/src/x/instrument" 91 "github.com/m3db/m3/src/x/mmap" 92 xos "github.com/m3db/m3/src/x/os" 93 "github.com/m3db/m3/src/x/pool" 94 "github.com/m3db/m3/src/x/serialize" 95 tbinarypool "github.com/m3db/m3/src/x/thrift" 96 97 "github.com/m3dbx/vellum/levenshtein" 98 "github.com/m3dbx/vellum/levenshtein2" 99 "github.com/m3dbx/vellum/regexp" 100 "github.com/opentracing/opentracing-go" 101 "github.com/uber-go/tally" 102 "github.com/uber/tchannel-go" 103 "go.etcd.io/etcd/server/v3/embed" 104 "go.uber.org/zap" 105 ) 106 107 const ( 108 bootstrapConfigInitTimeout = 10 * time.Second 109 serverGracefulCloseTimeout = 10 * time.Second 110 debugServerGracefulCloseTimeout = 2 * time.Second 111 bgProcessLimitInterval = 10 * time.Second 112 maxBgProcessLimitMonitorDuration = 5 * time.Minute 113 cpuProfileDuration = 5 * time.Second 114 filePathPrefixLockFile = ".lock" 115 defaultServiceName = "m3dbnode" 116 skipRaiseProcessLimitsEnvVar = "SKIP_PROCESS_LIMITS_RAISE" 117 skipRaiseProcessLimitsEnvVarTrue = "true" 118 mmapReporterMetricName = "mmap-mapped-bytes" 119 mmapReporterTagName = "map-name" 120 ) 121 122 // RunOptions provides options for running the server 123 // with backwards compatibility if only solely adding fields. 124 type RunOptions struct { 125 // ConfigFile is the YAML configuration file to use to run the server. 126 ConfigFile string 127 128 // Config is an alternate way to provide configuration and will be used 129 // instead of parsing ConfigFile if ConfigFile is not specified. 130 Config config.DBConfiguration 131 132 // BootstrapCh is a channel to listen on to be notified of bootstrap. 133 BootstrapCh chan<- struct{} 134 135 // EmbeddedKVCh is a channel to listen on to be notified that the embedded KV has bootstrapped. 136 EmbeddedKVCh chan<- struct{} 137 138 // ClientCh is a channel to listen on to share the same m3db client that this server uses. 139 ClientCh chan<- client.Client 140 141 // ClusterClientCh is a channel to listen on to share the same m3 cluster client that this server uses. 142 ClusterClientCh chan<- clusterclient.Client 143 144 // KVStoreCh is a channel to listen on to share the same m3 kv store client that this server uses. 145 KVStoreCh chan<- kv.Store 146 147 // InterruptCh is a programmatic interrupt channel to supply to 148 // interrupt and shutdown the server. 149 InterruptCh <-chan error 150 151 // ShutdownCh is an optional channel to supply if interested in receiving 152 // a notification that the server has shutdown. 153 ShutdownCh chan<- struct{} 154 155 // CustomOptions are custom options to apply to the session. 156 CustomOptions []client.CustomAdminOption 157 158 // Transform is a function to transform the Options. 159 Transform storage.OptionTransform 160 161 // StorageOptions are additional storage options. 162 StorageOptions StorageOptions 163 164 // CustomBuildTags are additional tags to be added to the instrument build 165 // reporter. 166 CustomBuildTags map[string]string 167 } 168 169 // Run runs the server programmatically given a filename for the 170 // configuration file. 171 func Run(runOpts RunOptions) { 172 var cfg config.DBConfiguration 173 if runOpts.ConfigFile != "" { 174 var rootCfg config.Configuration 175 if err := xconfig.LoadFile(&rootCfg, runOpts.ConfigFile, xconfig.Options{}); err != nil { 176 // NB(r): Use fmt.Fprintf(os.Stderr, ...) to avoid etcd.SetGlobals() 177 // sending stdlib "log" to black hole. Don't remove unless with good reason. 178 fmt.Fprintf(os.Stderr, "unable to load %s: %v", runOpts.ConfigFile, err) 179 os.Exit(1) 180 } 181 182 cfg = *rootCfg.DB 183 } else { 184 cfg = runOpts.Config 185 } 186 187 err := cfg.Validate() 188 if err != nil { 189 // NB(r): Use fmt.Fprintf(os.Stderr, ...) to avoid etcd.SetGlobals() 190 // sending stdlib "log" to black hole. Don't remove unless with good reason. 191 fmt.Fprintf(os.Stderr, "error initializing config defaults and validating config: %v", err) 192 os.Exit(1) 193 } 194 195 logger, err := cfg.LoggingOrDefault().BuildLogger() 196 if err != nil { 197 // NB(r): Use fmt.Fprintf(os.Stderr, ...) to avoid etcd.SetGlobals() 198 // sending stdlib "log" to black hole. Don't remove unless with good reason. 199 fmt.Fprintf(os.Stderr, "unable to create logger: %v", err) 200 os.Exit(1) 201 } 202 203 // NB(nate): Register shutdown notification defer function first so that 204 // it's the last defer to fire before terminating. This allows other defer methods 205 // that clean up resources to execute first. 206 if runOpts.ShutdownCh != nil { 207 defer func() { 208 select { 209 case runOpts.ShutdownCh <- struct{}{}: 210 break 211 default: 212 logger.Warn("could not send shutdown notification as channel was full") 213 } 214 }() 215 } 216 217 interruptOpts := xos.NewInterruptOptions() 218 if runOpts.InterruptCh != nil { 219 interruptOpts.InterruptCh = runOpts.InterruptCh 220 } 221 intWatchCancel := xos.WatchForInterrupt(logger, interruptOpts) 222 defer intWatchCancel() 223 224 defer logger.Sync() 225 226 cfg.Debug.SetRuntimeValues(logger) 227 228 xconfig.WarnOnDeprecation(cfg, logger) 229 230 // By default attempt to raise process limits, which is a benign operation. 231 skipRaiseLimits := strings.TrimSpace(os.Getenv(skipRaiseProcessLimitsEnvVar)) 232 if skipRaiseLimits != skipRaiseProcessLimitsEnvVarTrue { 233 // Raise fd limits to nr_open system limit 234 result, err := xos.RaiseProcessNoFileToNROpen() 235 if err != nil { 236 logger.Warn("unable to raise rlimit", zap.Error(err)) 237 } else { 238 logger.Info("raised rlimit no file fds limit", 239 zap.Bool("required", result.RaisePerformed), 240 zap.Uint64("sysNROpenValue", result.NROpenValue), 241 zap.Uint64("noFileMaxValue", result.NoFileMaxValue), 242 zap.Uint64("noFileCurrValue", result.NoFileCurrValue)) 243 } 244 } 245 246 // Parse file and directory modes 247 newFileMode, err := cfg.Filesystem.ParseNewFileMode() 248 if err != nil { 249 logger.Fatal("could not parse new file mode", zap.Error(err)) 250 } 251 252 newDirectoryMode, err := cfg.Filesystem.ParseNewDirectoryMode() 253 if err != nil { 254 logger.Fatal("could not parse new directory mode", zap.Error(err)) 255 } 256 257 // Obtain a lock on `filePathPrefix`, or exit if another process already has it. 258 // The lock consists of a lock file (on the file system) and a lock in memory. 259 // When the process exits gracefully, both the lock file and the lock will be removed. 260 // If the process exits ungracefully, only the lock in memory will be removed, the lock 261 // file will remain on the file system. When a dbnode starts after an ungracefully stop, 262 // it will be able to acquire the lock despite the fact the the lock file exists. 263 lockPath := path.Join(cfg.Filesystem.FilePathPrefixOrDefault(), filePathPrefixLockFile) 264 fslock, err := createAndAcquireLockfile(lockPath, newDirectoryMode) 265 if err != nil { 266 logger.Fatal("could not acquire lock", zap.String("path", lockPath), zap.Error(err)) 267 } 268 // nolint: errcheck 269 defer fslock.releaseLockfile() 270 271 go bgValidateProcessLimits(logger) 272 debug.SetGCPercent(cfg.GCPercentageOrDefault()) 273 274 defaultServeMux := http.NewServeMux() 275 scope, _, _, err := cfg.MetricsOrDefault().NewRootScopeAndReporters( 276 instrument.NewRootScopeAndReportersOptions{ 277 PrometheusDefaultServeMux: defaultServeMux, 278 }) 279 if err != nil { 280 logger.Fatal("could not connect to metrics", zap.Error(err)) 281 } 282 283 hostID, err := cfg.HostIDOrDefault().Resolve() 284 if err != nil { 285 logger.Fatal("could not resolve local host ID", zap.Error(err)) 286 } 287 288 var ( 289 tracer opentracing.Tracer 290 traceCloser io.Closer 291 ) 292 293 if cfg.Tracing == nil { 294 tracer = opentracing.NoopTracer{} 295 logger.Info("tracing disabled; set `tracing.backend` to enable") 296 } else { 297 // setup tracer 298 serviceName := cfg.Tracing.ServiceName 299 if serviceName == "" { 300 serviceName = defaultServiceName 301 } 302 tracer, traceCloser, err = cfg.Tracing.NewTracer(serviceName, scope.SubScope("jaeger"), logger) 303 if err != nil { 304 tracer = opentracing.NoopTracer{} 305 logger.Warn("could not initialize tracing; using no-op tracer instead", 306 zap.String("service", serviceName), zap.Error(err)) 307 } else { 308 defer traceCloser.Close() 309 logger.Info("tracing enabled", zap.String("service", serviceName)) 310 } 311 } 312 313 // Presence of KV server config indicates embedded etcd cluster 314 discoveryConfig := cfg.DiscoveryOrDefault() 315 envConfig, err := discoveryConfig.EnvironmentConfig(hostID) 316 if err != nil { 317 logger.Fatal("could not get env config from discovery config", zap.Error(err)) 318 } 319 320 if envConfig.SeedNodes == nil { 321 logger.Info("no seed nodes set, using dedicated etcd cluster") 322 } else { 323 // Default etcd client clusters if not set already 324 service, err := envConfig.Services.SyncCluster() 325 if err != nil { 326 logger.Fatal("invalid cluster configuration", zap.Error(err)) 327 } 328 329 clusters := service.Service.ETCDClusters 330 seedNodes := envConfig.SeedNodes.InitialCluster 331 if len(clusters) == 0 { 332 endpoints, err := config.InitialClusterEndpoints(seedNodes) 333 if err != nil { 334 logger.Fatal("unable to create etcd clusters", zap.Error(err)) 335 } 336 337 zone := service.Service.Zone 338 339 logger.Info("using seed nodes etcd cluster", 340 zap.String("zone", zone), zap.Strings("endpoints", endpoints)) 341 service.Service.ETCDClusters = []etcd.ClusterConfig{{ 342 Zone: zone, 343 Endpoints: endpoints, 344 }} 345 } 346 347 seedNodeHostIDs := make([]string, 0, len(seedNodes)) 348 for _, entry := range seedNodes { 349 seedNodeHostIDs = append(seedNodeHostIDs, entry.HostID) 350 } 351 logger.Info("resolving seed node configuration", 352 zap.String("hostID", hostID), zap.Strings("seedNodeHostIDs", seedNodeHostIDs), 353 ) 354 355 if !config.IsSeedNode(seedNodes, hostID) { 356 logger.Info("not a seed node, using cluster seed nodes") 357 } else { 358 logger.Info("seed node, starting etcd server") 359 360 etcdCfg, err := config.NewEtcdEmbedConfig(cfg) 361 if err != nil { 362 logger.Fatal("unable to create etcd config", zap.Error(err)) 363 } 364 365 e, err := embed.StartEtcd(etcdCfg) 366 if err != nil { 367 logger.Fatal("could not start embedded etcd", zap.Error(err)) 368 } 369 370 if runOpts.EmbeddedKVCh != nil { 371 // Notify on embedded KV bootstrap chan if specified 372 runOpts.EmbeddedKVCh <- struct{}{} 373 } 374 375 defer e.Close() 376 } 377 } 378 379 // By default use histogram timers for timers that 380 // are constructed allowing for type to be picked 381 // by the caller using instrument.NewTimer(...). 382 timerOpts := instrument.NewHistogramTimerOptions(instrument.HistogramTimerOptions{}) 383 timerOpts.StandardSampleRate = cfg.MetricsOrDefault().SampleRate() 384 385 var ( 386 opts = storage.NewOptions() 387 iOpts = opts.InstrumentOptions(). 388 SetLogger(logger). 389 SetMetricsScope(scope). 390 SetTimerOptions(timerOpts). 391 SetTracer(tracer). 392 SetCustomBuildTags(runOpts.CustomBuildTags) 393 ) 394 opts = opts.SetInstrumentOptions(iOpts) 395 396 // Only override the default MemoryTracker (which has default limits) if a custom limit has 397 // been set. 398 if cfg.Limits.MaxOutstandingRepairedBytes > 0 { 399 memTrackerOptions := storage.NewMemoryTrackerOptions(cfg.Limits.MaxOutstandingRepairedBytes) 400 memTracker := storage.NewMemoryTracker(memTrackerOptions) 401 opts = opts.SetMemoryTracker(memTracker) 402 } 403 404 opentracing.SetGlobalTracer(tracer) 405 406 // Set global index options. 407 if n := cfg.Index.RegexpDFALimitOrDefault(); n > 0 { 408 regexp.SetStateLimit(n) 409 levenshtein.SetStateLimit(n) 410 levenshtein2.SetStateLimit(n) 411 } 412 if n := cfg.Index.RegexpFSALimitOrDefault(); n > 0 { 413 regexp.SetDefaultLimit(n) 414 } 415 416 buildReporter := instrument.NewBuildReporter(iOpts) 417 if err := buildReporter.Start(); err != nil { 418 logger.Fatal("unable to start build reporter", zap.Error(err)) 419 } 420 defer buildReporter.Stop() 421 422 mmapCfg := cfg.Filesystem.MmapConfigurationOrDefault() 423 shouldUseHugeTLB := mmapCfg.HugeTLB.Enabled 424 if shouldUseHugeTLB { 425 // Make sure the host supports HugeTLB before proceeding with it to prevent 426 // excessive log spam. 427 shouldUseHugeTLB, err = hostSupportsHugeTLB() 428 if err != nil { 429 logger.Fatal("could not determine if host supports HugeTLB", zap.Error(err)) 430 } 431 if !shouldUseHugeTLB { 432 logger.Warn("host doesn't support HugeTLB, proceeding without it") 433 } 434 } 435 436 mmapReporter := newMmapReporter(scope) 437 mmapReporterCtx, cancel := context.WithCancel(context.Background()) 438 defer cancel() 439 go mmapReporter.Run(mmapReporterCtx) 440 opts = opts.SetMmapReporter(mmapReporter) 441 442 runtimeOpts := m3dbruntime.NewOptions(). 443 SetPersistRateLimitOptions(ratelimit.NewOptions(). 444 SetLimitEnabled(true). 445 SetLimitMbps(cfg.Filesystem.ThroughputLimitMbpsOrDefault()). 446 SetLimitCheckEvery(cfg.Filesystem.ThroughputCheckEveryOrDefault())). 447 SetWriteNewSeriesAsync(cfg.WriteNewSeriesAsyncOrDefault()). 448 SetWriteNewSeriesBackoffDuration(cfg.WriteNewSeriesBackoffDurationOrDefault()) 449 450 if lruCfg := cfg.Cache.SeriesConfiguration().LRU; lruCfg != nil { 451 runtimeOpts = runtimeOpts.SetMaxWiredBlocks(lruCfg.MaxBlocks) 452 } 453 454 // Setup query stats tracking. 455 var ( 456 docsLimit = limits.DefaultLookbackLimitOptions() 457 bytesReadLimit = limits.DefaultLookbackLimitOptions() 458 diskSeriesReadLimit = limits.DefaultLookbackLimitOptions() 459 aggDocsLimit = limits.DefaultLookbackLimitOptions() 460 ) 461 462 if limitConfig := runOpts.Config.Limits.MaxRecentlyQueriedSeriesBlocks; limitConfig != nil { 463 docsLimit.Limit = limitConfig.Value 464 docsLimit.Lookback = limitConfig.Lookback 465 } 466 if limitConfig := runOpts.Config.Limits.MaxRecentlyQueriedSeriesDiskBytesRead; limitConfig != nil { 467 bytesReadLimit.Limit = limitConfig.Value 468 bytesReadLimit.Lookback = limitConfig.Lookback 469 } 470 if limitConfig := runOpts.Config.Limits.MaxRecentlyQueriedSeriesDiskRead; limitConfig != nil { 471 diskSeriesReadLimit.Limit = limitConfig.Value 472 diskSeriesReadLimit.Lookback = limitConfig.Lookback 473 } 474 if limitConfig := runOpts.Config.Limits.MaxRecentlyQueriedMetadata; limitConfig != nil { 475 aggDocsLimit.Limit = limitConfig.Value 476 aggDocsLimit.Lookback = limitConfig.Lookback 477 } 478 limitOpts := limits.NewOptions(). 479 SetDocsLimitOpts(docsLimit). 480 SetBytesReadLimitOpts(bytesReadLimit). 481 SetDiskSeriesReadLimitOpts(diskSeriesReadLimit). 482 SetAggregateDocsLimitOpts(aggDocsLimit). 483 SetInstrumentOptions(iOpts) 484 if builder := opts.SourceLoggerBuilder(); builder != nil { 485 limitOpts = limitOpts.SetSourceLoggerBuilder(builder) 486 } 487 opts = opts.SetLimitsOptions(limitOpts) 488 489 seriesReadPermits := permits.NewLookbackLimitPermitsManager( 490 "disk-series-read", 491 diskSeriesReadLimit, 492 iOpts, 493 limitOpts.SourceLoggerBuilder(), 494 ) 495 496 permitOptions := opts.PermitsOptions().SetSeriesReadPermitsManager(seriesReadPermits) 497 maxIdxConcurrency := int(math.Ceil(float64(runtime.GOMAXPROCS(0)) / 2)) 498 if cfg.Index.MaxQueryIDsConcurrency > 0 { 499 maxIdxConcurrency = cfg.Index.MaxQueryIDsConcurrency 500 logger.Info("max index query IDs concurrency set", 501 zap.Int("maxIdxConcurrency", maxIdxConcurrency)) 502 } else { 503 logger.Info("max index query IDs concurrency was not set, falling back to default value", 504 zap.Int("maxIdxConcurrency", maxIdxConcurrency)) 505 } 506 maxWorkerTime := time.Second 507 if cfg.Index.MaxWorkerTime > 0 { 508 maxWorkerTime = cfg.Index.MaxWorkerTime 509 logger.Info("max index worker time set", 510 zap.Duration("maxWorkerTime", maxWorkerTime)) 511 } else { 512 logger.Info("max index worker time was not set, falling back to default value", 513 zap.Duration("maxWorkerTime", maxWorkerTime)) 514 } 515 opts = opts.SetPermitsOptions(permitOptions.SetIndexQueryPermitsManager( 516 permits.NewFixedPermitsManager(maxIdxConcurrency, int64(maxWorkerTime), iOpts))) 517 518 // Setup postings list cache. 519 var ( 520 plCacheConfig = cfg.Cache.PostingsListConfiguration() 521 plCacheSize = plCacheConfig.SizeOrDefault() 522 plCacheOptions = index.PostingsListCacheOptions{ 523 InstrumentOptions: opts.InstrumentOptions(). 524 SetMetricsScope(scope.SubScope("postings-list-cache")), 525 } 526 ) 527 segmentPostingsListCache, err := index.NewPostingsListCache(plCacheSize, plCacheOptions) 528 if err != nil { 529 logger.Fatal("could not construct segment postings list cache", zap.Error(err)) 530 } 531 532 segmentStopReporting := segmentPostingsListCache.Start() 533 defer segmentStopReporting() 534 535 searchPostingsListCache, err := index.NewPostingsListCache(plCacheSize, plCacheOptions) 536 if err != nil { 537 logger.Fatal("could not construct searches postings list cache", zap.Error(err)) 538 } 539 540 searchStopReporting := searchPostingsListCache.Start() 541 defer searchStopReporting() 542 543 // Setup index regexp compilation cache. 544 m3ninxindex.SetRegexpCacheOptions(m3ninxindex.RegexpCacheOptions{ 545 Size: cfg.Cache.RegexpConfiguration().SizeOrDefault(), 546 Scope: iOpts.MetricsScope(), 547 }) 548 549 if runOpts.Transform != nil { 550 opts = runOpts.Transform(opts) 551 } 552 553 queryLimits, err := limits.NewQueryLimits(opts.LimitsOptions()) 554 if err != nil { 555 logger.Fatal("could not construct docs query limits from config", zap.Error(err)) 556 } 557 558 queryLimits.Start() 559 defer queryLimits.Stop() 560 seriesReadPermits.Start() 561 defer seriesReadPermits.Stop() 562 563 // FOLLOWUP(prateek): remove this once we have the runtime options<->index wiring done 564 indexOpts := opts.IndexOptions() 565 insertMode := index.InsertSync 566 567 if cfg.WriteNewSeriesAsyncOrDefault() { 568 insertMode = index.InsertAsync 569 } 570 indexOpts = indexOpts.SetInsertMode(insertMode). 571 SetPostingsListCache(segmentPostingsListCache). 572 SetSearchPostingsListCache(searchPostingsListCache). 573 SetReadThroughSegmentOptions(index.ReadThroughSegmentOptions{ 574 CacheRegexp: plCacheConfig.CacheRegexpOrDefault(), 575 CacheTerms: plCacheConfig.CacheTermsOrDefault(), 576 CacheSearches: plCacheConfig.CacheSearchOrDefault(), 577 }). 578 SetMmapReporter(mmapReporter). 579 SetQueryLimits(queryLimits) 580 581 opts = opts.SetIndexOptions(indexOpts) 582 583 if tick := cfg.Tick; tick != nil { 584 runtimeOpts = runtimeOpts. 585 SetTickSeriesBatchSize(tick.SeriesBatchSize). 586 SetTickPerSeriesSleepDuration(tick.PerSeriesSleepDuration). 587 SetTickMinimumInterval(tick.MinimumInterval) 588 } 589 590 runtimeOptsMgr := m3dbruntime.NewOptionsManager() 591 if err := runtimeOptsMgr.Update(runtimeOpts); err != nil { 592 logger.Fatal("could not set initial runtime options", zap.Error(err)) 593 } 594 defer runtimeOptsMgr.Close() 595 596 opts = opts.SetRuntimeOptionsManager(runtimeOptsMgr) 597 598 policy, err := cfg.PoolingPolicyOrDefault() 599 if err != nil { 600 logger.Fatal("could not get pooling policy", zap.Error(err)) 601 } 602 603 tagEncoderPool := serialize.NewTagEncoderPool( 604 serialize.NewTagEncoderOptions(), 605 poolOptions( 606 policy.TagEncoderPool, 607 scope.SubScope("tag-encoder-pool"))) 608 tagEncoderPool.Init() 609 tagDecoderPool := serialize.NewTagDecoderPool( 610 serialize.NewTagDecoderOptions(serialize.TagDecoderOptionsConfig{}), 611 poolOptions( 612 policy.TagDecoderPool, 613 scope.SubScope("tag-decoder-pool"))) 614 tagDecoderPool.Init() 615 616 // Pass nil for block.LeaseVerifier for now and it will be set after the 617 // db is constructed (since the db is required to construct a 618 // block.LeaseVerifier). Initialized here because it needs to be propagated 619 // to both the DB and the blockRetriever. 620 blockLeaseManager := block.NewLeaseManager(nil) 621 opts = opts.SetBlockLeaseManager(blockLeaseManager) 622 fsopts := fs.NewOptions(). 623 SetClockOptions(opts.ClockOptions()). 624 SetInstrumentOptions(opts.InstrumentOptions(). 625 SetMetricsScope(scope.SubScope("database.fs"))). 626 SetFilePathPrefix(cfg.Filesystem.FilePathPrefixOrDefault()). 627 SetNewFileMode(newFileMode). 628 SetNewDirectoryMode(newDirectoryMode). 629 SetWriterBufferSize(cfg.Filesystem.WriteBufferSizeOrDefault()). 630 SetDataReaderBufferSize(cfg.Filesystem.DataReadBufferSizeOrDefault()). 631 SetInfoReaderBufferSize(cfg.Filesystem.InfoReadBufferSizeOrDefault()). 632 SetSeekReaderBufferSize(cfg.Filesystem.SeekReadBufferSizeOrDefault()). 633 SetMmapEnableHugeTLB(shouldUseHugeTLB). 634 SetMmapHugeTLBThreshold(mmapCfg.HugeTLB.Threshold). 635 SetRuntimeOptionsManager(runtimeOptsMgr). 636 SetTagEncoderPool(tagEncoderPool). 637 SetTagDecoderPool(tagDecoderPool). 638 SetForceIndexSummariesMmapMemory(cfg.Filesystem.ForceIndexSummariesMmapMemoryOrDefault()). 639 SetForceBloomFilterMmapMemory(cfg.Filesystem.ForceBloomFilterMmapMemoryOrDefault()). 640 SetIndexBloomFilterFalsePositivePercent(cfg.Filesystem.BloomFilterFalsePositivePercentOrDefault()). 641 SetMmapReporter(mmapReporter) 642 643 var commitLogQueueSize int 644 cfgCommitLog := cfg.CommitLogOrDefault() 645 specified := cfgCommitLog.Queue.Size 646 switch cfgCommitLog.Queue.CalculationType { 647 case config.CalculationTypeFixed: 648 commitLogQueueSize = specified 649 case config.CalculationTypePerCPU: 650 commitLogQueueSize = specified * runtime.GOMAXPROCS(0) 651 default: 652 logger.Fatal("unknown commit log queue size type", 653 zap.Any("type", cfgCommitLog.Queue.CalculationType)) 654 } 655 656 var commitLogQueueChannelSize int 657 if cfgCommitLog.QueueChannel != nil { 658 specified := cfgCommitLog.QueueChannel.Size 659 switch cfgCommitLog.Queue.CalculationType { 660 case config.CalculationTypeFixed: 661 commitLogQueueChannelSize = specified 662 case config.CalculationTypePerCPU: 663 commitLogQueueChannelSize = specified * runtime.GOMAXPROCS(0) 664 default: 665 logger.Fatal("unknown commit log queue channel size type", 666 zap.Any("type", cfgCommitLog.Queue.CalculationType)) 667 } 668 } else { 669 commitLogQueueChannelSize = int(float64(commitLogQueueSize) / commitlog.MaximumQueueSizeQueueChannelSizeRatio) 670 } 671 672 // Set the series cache policy. 673 seriesCachePolicy := cfg.Cache.SeriesConfiguration().Policy 674 opts = opts.SetSeriesCachePolicy(seriesCachePolicy) 675 676 // Apply pooling options. 677 poolingPolicy, err := cfg.PoolingPolicyOrDefault() 678 if err != nil { 679 logger.Fatal("could not get pooling policy", zap.Error(err)) 680 } 681 682 opts = withEncodingAndPoolingOptions(cfg, logger, opts, poolingPolicy) 683 opts = opts.SetCommitLogOptions(opts.CommitLogOptions(). 684 SetInstrumentOptions(opts.InstrumentOptions()). 685 SetFilesystemOptions(fsopts). 686 SetStrategy(commitlog.StrategyWriteBehind). 687 SetFlushSize(cfgCommitLog.FlushMaxBytes). 688 SetFlushInterval(cfgCommitLog.FlushEvery). 689 SetBacklogQueueSize(commitLogQueueSize). 690 SetBacklogQueueChannelSize(commitLogQueueChannelSize)) 691 692 // Setup the block retriever 693 switch seriesCachePolicy { 694 case series.CacheAll: 695 // No options needed to be set 696 default: 697 // All other caching strategies require retrieving series from disk 698 // to service a cache miss 699 retrieverOpts := fs.NewBlockRetrieverOptions(). 700 SetBytesPool(opts.BytesPool()). 701 SetRetrieveRequestPool(opts.RetrieveRequestPool()). 702 SetIdentifierPool(opts.IdentifierPool()). 703 SetBlockLeaseManager(blockLeaseManager). 704 SetQueryLimits(queryLimits) 705 if blockRetrieveCfg := cfg.BlockRetrieve; blockRetrieveCfg != nil { 706 if v := blockRetrieveCfg.FetchConcurrency; v != nil { 707 retrieverOpts = retrieverOpts.SetFetchConcurrency(*v) 708 } 709 if v := blockRetrieveCfg.CacheBlocksOnRetrieve; v != nil { 710 retrieverOpts = retrieverOpts.SetCacheBlocksOnRetrieve(*v) 711 } 712 } 713 blockRetrieverMgr := block.NewDatabaseBlockRetrieverManager( 714 func(md namespace.Metadata, shardSet sharding.ShardSet) (block.DatabaseBlockRetriever, error) { 715 retriever, err := fs.NewBlockRetriever(retrieverOpts, fsopts) 716 if err != nil { 717 return nil, err 718 } 719 if err := retriever.Open(md, shardSet); err != nil { 720 return nil, err 721 } 722 return retriever, nil 723 }) 724 opts = opts.SetDatabaseBlockRetrieverManager(blockRetrieverMgr) 725 } 726 727 // Set the persistence manager 728 pm, err := fs.NewPersistManager(fsopts) 729 if err != nil { 730 logger.Fatal("could not create persist manager", zap.Error(err)) 731 } 732 opts = opts.SetPersistManager(pm) 733 734 // Set the index claims manager 735 icm, err := fs.NewIndexClaimsManager(fsopts) 736 if err != nil { 737 logger.Fatal("could not create index claims manager", zap.Error(err)) 738 } 739 defer func() { 740 // Reset counter of index claims managers after server teardown. 741 fs.ResetIndexClaimsManagersUnsafe() 742 }() 743 opts = opts.SetIndexClaimsManager(icm) 744 745 if value := cfg.ForceColdWritesEnabled; value != nil { 746 // Allow forcing cold writes to be enabled by config. 747 opts = opts.SetForceColdWritesEnabled(*value) 748 } 749 750 forceColdWrites := opts.ForceColdWritesEnabled() 751 var envCfgResults environment.ConfigureResults 752 if len(envConfig.Statics) == 0 { 753 logger.Info("creating dynamic config service client with m3cluster") 754 755 envCfgResults, err = envConfig.Configure(environment.ConfigurationParameters{ 756 InterruptedCh: interruptOpts.InterruptedCh, 757 InstrumentOpts: iOpts, 758 HashingSeed: cfg.Hashing.Seed, 759 NewDirectoryMode: newDirectoryMode, 760 ForceColdWritesEnabled: forceColdWrites, 761 }) 762 if err != nil { 763 logger.Fatal("could not initialize dynamic config", zap.Error(err)) 764 } 765 } else { 766 logger.Info("creating static config service client with m3cluster") 767 768 envCfgResults, err = envConfig.Configure(environment.ConfigurationParameters{ 769 InterruptedCh: interruptOpts.InterruptedCh, 770 InstrumentOpts: iOpts, 771 HostID: hostID, 772 ForceColdWritesEnabled: forceColdWrites, 773 }) 774 if err != nil { 775 logger.Fatal("could not initialize static config", zap.Error(err)) 776 } 777 } 778 779 syncCfg, err := envCfgResults.SyncCluster() 780 if err != nil { 781 logger.Fatal("invalid cluster config", zap.Error(err)) 782 } 783 if runOpts.ClusterClientCh != nil { 784 runOpts.ClusterClientCh <- syncCfg.ClusterClient 785 } 786 if runOpts.KVStoreCh != nil { 787 runOpts.KVStoreCh <- syncCfg.KVStore 788 } 789 790 opts = opts.SetNamespaceInitializer(syncCfg.NamespaceInitializer) 791 792 // Set tchannelthrift options. 793 ttopts := tchannelthrift.NewOptions(). 794 SetClockOptions(opts.ClockOptions()). 795 SetInstrumentOptions(opts.InstrumentOptions()). 796 SetTopologyInitializer(syncCfg.TopologyInitializer). 797 SetIdentifierPool(opts.IdentifierPool()). 798 SetTagEncoderPool(tagEncoderPool). 799 SetCheckedBytesWrapperPool(opts.CheckedBytesWrapperPool()). 800 SetMaxOutstandingWriteRequests(cfg.Limits.MaxOutstandingWriteRequests). 801 SetMaxOutstandingReadRequests(cfg.Limits.MaxOutstandingReadRequests). 802 SetQueryLimits(queryLimits). 803 SetPermitsOptions(opts.PermitsOptions()) 804 805 // Start servers before constructing the DB so orchestration tools can check health endpoints 806 // before topology is set. 807 var ( 808 contextPool = opts.ContextPool() 809 tchannelOpts = xtchannel.NewDefaultChannelOptions() 810 // Pass nil for the database argument because we haven't constructed it yet. We'll call 811 // SetDatabase() once we've initialized it. 812 service = ttnode.NewService(nil, ttopts) 813 ) 814 if cfg.TChannel != nil { 815 tchannelOpts.MaxIdleTime = cfg.TChannel.MaxIdleTime 816 tchannelOpts.IdleCheckInterval = cfg.TChannel.IdleCheckInterval 817 } 818 tchanOpts := ttnode.NewOptions(tchannelOpts). 819 SetInstrumentOptions(opts.InstrumentOptions()) 820 if fn := runOpts.StorageOptions.TChanChannelFn; fn != nil { 821 tchanOpts = tchanOpts.SetTChanChannelFn(fn) 822 } 823 if fn := runOpts.StorageOptions.TChanNodeServerFn; fn != nil { 824 tchanOpts = tchanOpts.SetTChanNodeServerFn(fn) 825 } 826 827 listenAddress := cfg.ListenAddressOrDefault() 828 tchannelthriftNodeClose, err := ttnode.NewServer(service, 829 listenAddress, contextPool, tchanOpts).ListenAndServe() 830 if err != nil { 831 logger.Fatal("could not open tchannelthrift interface", 832 zap.String("address", listenAddress), zap.Error(err)) 833 } 834 defer tchannelthriftNodeClose() 835 logger.Info("node tchannelthrift: listening", zap.String("address", listenAddress)) 836 837 httpListenAddress := cfg.HTTPNodeListenAddressOrDefault() 838 httpjsonNodeClose, err := hjnode.NewServer(service, 839 httpListenAddress, contextPool, nil).ListenAndServe() 840 if err != nil { 841 logger.Fatal("could not open httpjson interface", 842 zap.String("address", httpListenAddress), zap.Error(err)) 843 } 844 defer httpjsonNodeClose() 845 logger.Info("node httpjson: listening", zap.String("address", httpListenAddress)) 846 847 debugListenAddress := cfg.DebugListenAddressOrDefault() 848 if debugListenAddress != "" { 849 var debugWriter xdebug.ZipWriter 850 handlerOpts, err := placementhandler.NewHandlerOptions(syncCfg.ClusterClient, 851 placement.Configuration{}, nil, iOpts) 852 if err != nil { 853 logger.Warn("could not create handler options for debug writer", zap.Error(err)) 854 } else { 855 if envConfig.Services == nil || len(envConfig.Services) == 0 { 856 logger.Warn("no DynamicConfiguration set; not configuring debug writer", 857 zap.Error(err), 858 ) 859 } else { 860 envCfgCluster, err := envConfig.Services.SyncCluster() 861 if err != nil { 862 logger.Warn("could not get cluster config for debug writer", 863 zap.Error(err), 864 ) 865 } else { 866 debugWriter, err = extdebug.NewPlacementAndNamespaceZipWriterWithDefaultSources( 867 cpuProfileDuration, 868 syncCfg.ClusterClient, 869 handlerOpts, 870 []handleroptions.ServiceNameAndDefaults{ 871 { 872 ServiceName: handleroptions.M3DBServiceName, 873 Defaults: []handleroptions.ServiceOptionsDefault{ 874 handleroptions.WithDefaultServiceEnvironment(envCfgCluster.Service.Env), 875 handleroptions.WithDefaultServiceZone(envCfgCluster.Service.Zone), 876 }, 877 }, 878 }, 879 iOpts) 880 if err != nil { 881 logger.Error("unable to create debug writer", zap.Error(err)) 882 } 883 } 884 } 885 } 886 887 debugClose := startDebugServer(debugWriter, logger, debugListenAddress, defaultServeMux) 888 defer debugClose() 889 } 890 891 topo, err := syncCfg.TopologyInitializer.Init() 892 if err != nil { 893 var interruptErr *xos.InterruptError 894 if errors.As(err, &interruptErr) { 895 logger.Warn("interrupt received. closing server", zap.Error(err)) 896 // NB(nate): Have not attempted to start the actual database yet so 897 // it's safe for us to just return here. 898 return 899 } 900 901 logger.Fatal("could not initialize m3db topology", zap.Error(err)) 902 } 903 904 var protoEnabled bool 905 if cfg.Proto != nil && cfg.Proto.Enabled { 906 protoEnabled = true 907 } 908 schemaRegistry := namespace.NewSchemaRegistry(protoEnabled, logger) 909 // For application m3db client integration test convenience (where a local dbnode is started as a docker container), 910 // we allow loading user schema from local file into schema registry. 911 if protoEnabled { 912 for nsID, protoConfig := range cfg.Proto.SchemaRegistry { 913 dummyDeployID := "fromconfig" 914 if err := namespace.LoadSchemaRegistryFromFile(schemaRegistry, ident.StringID(nsID), 915 dummyDeployID, 916 protoConfig.SchemaFilePath, protoConfig.MessageName); err != nil { 917 logger.Fatal("could not load schema from configuration", zap.Error(err)) 918 } 919 } 920 } 921 922 origin := topology.NewHost(hostID, "") 923 m3dbClient, err := newAdminClient( 924 cfg.Client, opts.ClockOptions(), iOpts, tchannelOpts, syncCfg.TopologyInitializer, 925 runtimeOptsMgr, origin, protoEnabled, schemaRegistry, 926 syncCfg.KVStore, opts.ContextPool(), opts.BytesPool(), opts.IdentifierPool(), 927 logger, runOpts.CustomOptions) 928 if err != nil { 929 logger.Fatal("could not create m3db client", zap.Error(err)) 930 } 931 932 if runOpts.ClientCh != nil { 933 runOpts.ClientCh <- m3dbClient 934 } 935 936 documentsBuilderAlloc := index.NewBootstrapResultDocumentsBuilderAllocator( 937 opts.IndexOptions()) 938 rsOpts := result.NewOptions(). 939 SetClockOptions(opts.ClockOptions()). 940 SetInstrumentOptions(opts.InstrumentOptions()). 941 SetDatabaseBlockOptions(opts.DatabaseBlockOptions()). 942 SetSeriesCachePolicy(opts.SeriesCachePolicy()). 943 SetIndexDocumentsBuilderAllocator(documentsBuilderAlloc) 944 945 var repairClients []client.AdminClient 946 if cfg.Repair != nil && cfg.Repair.Enabled { 947 repairClients = append(repairClients, m3dbClient) 948 } 949 if cfg.Replication != nil { 950 for _, cluster := range cfg.Replication.Clusters { 951 if !cluster.RepairEnabled { 952 continue 953 } 954 955 // Pass nil for the topology initializer because we want to create 956 // a new one for the cluster we wish to replicate from, not use the 957 // same one as the cluster this node belongs to. 958 var topologyInitializer topology.Initializer 959 // Guaranteed to not be nil if repair is enabled by config validation. 960 clientCfg := *cluster.Client 961 clusterClient, err := newAdminClient( 962 clientCfg, opts.ClockOptions(), iOpts, tchannelOpts, topologyInitializer, 963 runtimeOptsMgr, origin, protoEnabled, schemaRegistry, 964 syncCfg.KVStore, opts.ContextPool(), opts.BytesPool(), 965 opts.IdentifierPool(), logger, runOpts.CustomOptions) 966 if err != nil { 967 logger.Fatal( 968 "unable to create client for replicated cluster", 969 zap.String("clusterName", cluster.Name), zap.Error(err)) 970 } 971 repairClients = append(repairClients, clusterClient) 972 } 973 } 974 repairEnabled := len(repairClients) > 0 975 if repairEnabled { 976 repairOpts := opts.RepairOptions(). 977 SetAdminClients(repairClients) 978 979 if repairCfg := cfg.Repair; repairCfg != nil { 980 repairOpts = repairOpts. 981 SetType(repairCfg.Type). 982 SetStrategy(repairCfg.Strategy). 983 SetForce(repairCfg.Force). 984 SetResultOptions(rsOpts). 985 SetDebugShadowComparisonsEnabled(cfg.Repair.DebugShadowComparisonsEnabled) 986 if cfg.Repair.Throttle > 0 { 987 repairOpts = repairOpts.SetRepairThrottle(cfg.Repair.Throttle) 988 } 989 if cfg.Repair.CheckInterval > 0 { 990 repairOpts = repairOpts.SetRepairCheckInterval(cfg.Repair.CheckInterval) 991 } 992 if cfg.Repair.Concurrency > 0 { 993 repairOpts = repairOpts.SetRepairShardConcurrency(cfg.Repair.Concurrency) 994 } 995 996 if cfg.Repair.DebugShadowComparisonsPercentage > 0 { 997 // Set conditionally to avoid stomping on the default value of 1.0. 998 repairOpts = repairOpts.SetDebugShadowComparisonsPercentage(cfg.Repair.DebugShadowComparisonsPercentage) 999 } 1000 } 1001 1002 opts = opts. 1003 SetRepairEnabled(true). 1004 SetRepairOptions(repairOpts) 1005 } else { 1006 opts = opts.SetRepairEnabled(false) 1007 } 1008 1009 // Set bootstrap options - We need to create a topology map provider from the 1010 // same topology that will be passed to the cluster so that when we make 1011 // bootstrapping decisions they are in sync with the clustered database 1012 // which is triggering the actual bootstraps. This way, when the clustered 1013 // database receives a topology update and decides to kick off a bootstrap, 1014 // the bootstrap process will receaive a topology map that is at least as 1015 // recent as the one that triggered the bootstrap, if not newer. 1016 // See GitHub issue #1013 for more details. 1017 topoMapProvider := newTopoMapProvider(topo) 1018 bs, err := cfg.Bootstrap.New( 1019 rsOpts, opts, topoMapProvider, origin, m3dbClient, 1020 ) 1021 if err != nil { 1022 logger.Fatal("could not create bootstrap process", zap.Error(err)) 1023 } 1024 opts = opts.SetBootstrapProcessProvider(bs) 1025 1026 // Start the cluster services now that the M3DB client is available. 1027 clusterListenAddress := cfg.ClusterListenAddressOrDefault() 1028 tchannelthriftClusterClose, err := ttcluster.NewServer(m3dbClient, 1029 clusterListenAddress, contextPool, tchannelOpts).ListenAndServe() 1030 if err != nil { 1031 logger.Fatal("could not open tchannelthrift interface", 1032 zap.String("address", clusterListenAddress), zap.Error(err)) 1033 } 1034 defer tchannelthriftClusterClose() 1035 logger.Info("cluster tchannelthrift: listening", zap.String("address", clusterListenAddress)) 1036 1037 httpClusterListenAddress := cfg.HTTPClusterListenAddressOrDefault() 1038 httpjsonClusterClose, err := hjcluster.NewServer(m3dbClient, 1039 httpClusterListenAddress, contextPool, nil).ListenAndServe() 1040 if err != nil { 1041 logger.Fatal("could not open httpjson interface", 1042 zap.String("address", httpClusterListenAddress), zap.Error(err)) 1043 } 1044 defer httpjsonClusterClose() 1045 logger.Info("cluster httpjson: listening", zap.String("address", httpClusterListenAddress)) 1046 1047 // Initialize clustered database. 1048 clusterTopoWatch, err := topo.Watch() 1049 if err != nil { 1050 logger.Fatal("could not create cluster topology watch", zap.Error(err)) 1051 } 1052 1053 opts = opts.SetSchemaRegistry(schemaRegistry). 1054 SetAdminClient(m3dbClient) 1055 1056 db, err := cluster.NewDatabase(hostID, topo, clusterTopoWatch, opts) 1057 if err != nil { 1058 logger.Fatal("could not construct database", zap.Error(err)) 1059 } 1060 1061 // Now that the database has been created it can be set as the block lease verifier 1062 // on the block lease manager. 1063 leaseVerifier := storage.NewLeaseVerifier(db) 1064 blockLeaseManager.SetLeaseVerifier(leaseVerifier) 1065 1066 if err := db.Open(); err != nil { 1067 logger.Fatal("could not open database", zap.Error(err)) 1068 } 1069 1070 // Now that we've initialized the database we can set it on the service. 1071 service.SetDatabase(db) 1072 1073 go func() { 1074 if runOpts.BootstrapCh != nil { 1075 // Notify on bootstrap chan if specified. 1076 defer func() { 1077 runOpts.BootstrapCh <- struct{}{} 1078 }() 1079 } 1080 1081 // Bootstrap asynchronously so we can handle interrupt. 1082 if err := db.Bootstrap(); err != nil { 1083 logger.Fatal("could not bootstrap database", zap.Error(err)) 1084 } 1085 logger.Info("bootstrapped") 1086 1087 // Only set the write new series limit after bootstrapping 1088 kvWatchNewSeriesLimitPerShard(syncCfg.KVStore, logger, topo, 1089 runtimeOptsMgr, cfg.Limits.WriteNewSeriesPerSecond) 1090 kvWatchEncodersPerBlockLimit(syncCfg.KVStore, logger, 1091 runtimeOptsMgr, cfg.Limits.MaxEncodersPerBlock) 1092 kvWatchQueryLimit(syncCfg.KVStore, logger, 1093 queryLimits.FetchDocsLimit(), 1094 queryLimits.BytesReadLimit(), 1095 // For backwards compatibility as M3 moves toward permits instead of time-based limits, 1096 // the series-read path uses permits which are implemented with limits, and so we support 1097 // dynamic updates to this limit-based permit still be passing downstream the limit itself. 1098 seriesReadPermits.Limit, 1099 queryLimits.AggregateDocsLimit(), 1100 limitOpts, 1101 ) 1102 }() 1103 1104 // Stop our async watch and now block waiting for the interrupt. 1105 intWatchCancel() 1106 select { 1107 case <-interruptOpts.InterruptedCh: 1108 logger.Warn("interrupt already received. closing") 1109 default: 1110 xos.WaitForInterrupt(logger, interruptOpts) 1111 } 1112 1113 // Attempt graceful server close. 1114 closedCh := make(chan struct{}) 1115 go func() { 1116 err := db.Terminate() 1117 if err != nil { 1118 logger.Error("close database error", zap.Error(err)) 1119 } 1120 closedCh <- struct{}{} 1121 }() 1122 1123 // Wait then close or hard close. 1124 closeTimeout := serverGracefulCloseTimeout 1125 select { 1126 case <-closedCh: 1127 logger.Info("server closed") 1128 case <-time.After(closeTimeout): 1129 logger.Error("server closed after timeout", zap.Duration("timeout", closeTimeout)) 1130 } 1131 } 1132 1133 func startDebugServer( 1134 debugWriter xdebug.ZipWriter, 1135 logger *zap.Logger, 1136 debugListenAddress string, 1137 mux *http.ServeMux, 1138 ) func() { 1139 xdebug.RegisterPProfHandlers(mux) 1140 server := http.Server{Addr: debugListenAddress, Handler: mux} 1141 1142 if debugWriter != nil { 1143 if err := debugWriter.RegisterHandler(xdebug.DebugURL, mux); err != nil { 1144 logger.Error("unable to register debug writer endpoint", zap.Error(err)) 1145 } 1146 } 1147 1148 go func() { 1149 if err := server.ListenAndServe(); !errors.Is(err, http.ErrServerClosed) { 1150 logger.Error("debug server could not listen", 1151 zap.String("address", debugListenAddress), zap.Error(err)) 1152 } 1153 }() 1154 1155 return func() { 1156 ctx, cancel := context.WithTimeout(context.Background(), debugServerGracefulCloseTimeout) 1157 defer cancel() 1158 if err := server.Shutdown(ctx); err != nil { 1159 logger.Warn("debug server failed to shutdown gracefully") 1160 } else { 1161 logger.Info("debug server closed") 1162 } 1163 } 1164 } 1165 1166 func bgValidateProcessLimits(logger *zap.Logger) { 1167 // If unable to validate process limits on the current configuration, 1168 // do not run background validator task. 1169 if canValidate, message := canValidateProcessLimits(); !canValidate { 1170 logger.Warn("cannot validate process limits: invalid configuration found", 1171 zap.String("message", message)) 1172 return 1173 } 1174 1175 start := time.Now() 1176 t := time.NewTicker(bgProcessLimitInterval) 1177 defer t.Stop() 1178 for { 1179 // only monitor for first `maxBgProcessLimitMonitorDuration` of process lifetime 1180 if time.Since(start) > maxBgProcessLimitMonitorDuration { 1181 return 1182 } 1183 1184 err := validateProcessLimits() 1185 if err == nil { 1186 return 1187 } 1188 1189 logger.Warn("invalid configuration found, refer to linked documentation for more information", 1190 zap.String("url", xdocs.Path("operational_guide/kernel_configuration")), 1191 zap.Error(err), 1192 ) 1193 1194 <-t.C 1195 } 1196 } 1197 1198 func kvWatchNewSeriesLimitPerShard( 1199 store kv.Store, 1200 logger *zap.Logger, 1201 topo topology.Topology, 1202 runtimeOptsMgr m3dbruntime.OptionsManager, 1203 defaultClusterNewSeriesLimit int, 1204 ) { 1205 var initClusterLimit int 1206 1207 value, err := store.Get(kvconfig.ClusterNewSeriesInsertLimitKey) 1208 if err == nil { 1209 protoValue := &commonpb.Int64Proto{} 1210 err = value.Unmarshal(protoValue) 1211 if err == nil { 1212 initClusterLimit = int(protoValue.Value) 1213 } 1214 } 1215 1216 if err != nil { 1217 if err != kv.ErrNotFound { 1218 logger.Warn("error resolving cluster new series insert limit", zap.Error(err)) 1219 } 1220 initClusterLimit = defaultClusterNewSeriesLimit 1221 } 1222 1223 err = setNewSeriesLimitPerShardOnChange(topo, runtimeOptsMgr, initClusterLimit) 1224 if err != nil { 1225 logger.Warn("unable to set cluster new series insert limit", zap.Error(err)) 1226 } 1227 1228 watch, err := store.Watch(kvconfig.ClusterNewSeriesInsertLimitKey) 1229 if err != nil { 1230 logger.Error("could not watch cluster new series insert limit", zap.Error(err)) 1231 return 1232 } 1233 1234 go func() { 1235 protoValue := &commonpb.Int64Proto{} 1236 for range watch.C() { 1237 value := defaultClusterNewSeriesLimit 1238 if newValue := watch.Get(); newValue != nil { 1239 if err := newValue.Unmarshal(protoValue); err != nil { 1240 logger.Warn("unable to parse new cluster new series insert limit", zap.Error(err)) 1241 continue 1242 } 1243 value = int(protoValue.Value) 1244 } 1245 1246 err = setNewSeriesLimitPerShardOnChange(topo, runtimeOptsMgr, value) 1247 if err != nil { 1248 logger.Warn("unable to set cluster new series insert limit", zap.Error(err)) 1249 continue 1250 } 1251 } 1252 }() 1253 } 1254 1255 func kvWatchEncodersPerBlockLimit( 1256 store kv.Store, 1257 logger *zap.Logger, 1258 runtimeOptsMgr m3dbruntime.OptionsManager, 1259 defaultEncodersPerBlockLimit int, 1260 ) { 1261 var initEncoderLimit int 1262 1263 value, err := store.Get(kvconfig.EncodersPerBlockLimitKey) 1264 if err == nil { 1265 protoValue := &commonpb.Int64Proto{} 1266 err = value.Unmarshal(protoValue) 1267 if err == nil { 1268 initEncoderLimit = int(protoValue.Value) 1269 } 1270 } 1271 1272 if err != nil { 1273 if err != kv.ErrNotFound { 1274 logger.Warn("error resolving encoder per block limit", zap.Error(err)) 1275 } 1276 initEncoderLimit = defaultEncodersPerBlockLimit 1277 } 1278 1279 err = setEncodersPerBlockLimitOnChange(runtimeOptsMgr, initEncoderLimit) 1280 if err != nil { 1281 logger.Warn("unable to set encoder per block limit", zap.Error(err)) 1282 } 1283 1284 watch, err := store.Watch(kvconfig.EncodersPerBlockLimitKey) 1285 if err != nil { 1286 logger.Error("could not watch encoder per block limit", zap.Error(err)) 1287 return 1288 } 1289 1290 go func() { 1291 protoValue := &commonpb.Int64Proto{} 1292 for range watch.C() { 1293 value := defaultEncodersPerBlockLimit 1294 if newValue := watch.Get(); newValue != nil { 1295 if err := newValue.Unmarshal(protoValue); err != nil { 1296 logger.Warn("unable to parse new encoder per block limit", zap.Error(err)) 1297 continue 1298 } 1299 value = int(protoValue.Value) 1300 } 1301 1302 err = setEncodersPerBlockLimitOnChange(runtimeOptsMgr, value) 1303 if err != nil { 1304 logger.Warn("unable to set encoder per block limit", zap.Error(err)) 1305 continue 1306 } 1307 } 1308 }() 1309 } 1310 1311 func kvWatchQueryLimit( 1312 store kv.Store, 1313 logger *zap.Logger, 1314 docsLimit limits.LookbackLimit, 1315 bytesReadLimit limits.LookbackLimit, 1316 diskSeriesReadLimit limits.LookbackLimit, 1317 aggregateDocsLimit limits.LookbackLimit, 1318 defaultOpts limits.Options, 1319 ) { 1320 value, err := store.Get(kvconfig.QueryLimits) 1321 if err == nil { 1322 dynamicLimits := &kvpb.QueryLimits{} 1323 err = value.Unmarshal(dynamicLimits) 1324 if err == nil { 1325 updateQueryLimits( 1326 logger, docsLimit, bytesReadLimit, diskSeriesReadLimit, 1327 aggregateDocsLimit, dynamicLimits, defaultOpts) 1328 } 1329 } else if !errors.Is(err, kv.ErrNotFound) { 1330 logger.Warn("error resolving query limit", zap.Error(err)) 1331 } 1332 1333 watch, err := store.Watch(kvconfig.QueryLimits) 1334 if err != nil { 1335 logger.Error("could not watch query limit", zap.Error(err)) 1336 return 1337 } 1338 1339 go func() { 1340 dynamicLimits := &kvpb.QueryLimits{} 1341 for range watch.C() { 1342 if newValue := watch.Get(); newValue != nil { 1343 if err := newValue.Unmarshal(dynamicLimits); err != nil { 1344 logger.Warn("unable to parse new query limits", zap.Error(err)) 1345 continue 1346 } 1347 updateQueryLimits( 1348 logger, docsLimit, bytesReadLimit, diskSeriesReadLimit, 1349 aggregateDocsLimit, dynamicLimits, defaultOpts) 1350 } 1351 } 1352 }() 1353 } 1354 1355 func updateQueryLimits( 1356 logger *zap.Logger, 1357 docsLimit limits.LookbackLimit, 1358 bytesReadLimit limits.LookbackLimit, 1359 diskSeriesReadLimit limits.LookbackLimit, 1360 aggregateDocsLimit limits.LookbackLimit, 1361 dynamicOpts *kvpb.QueryLimits, 1362 configOpts limits.Options, 1363 ) { 1364 var ( 1365 // Default to the config-based limits if unset in dynamic limits. 1366 // Otherwise, use the dynamic limit. 1367 docsLimitOpts = configOpts.DocsLimitOpts() 1368 bytesReadLimitOpts = configOpts.BytesReadLimitOpts() 1369 diskSeriesReadLimitOpts = configOpts.DiskSeriesReadLimitOpts() 1370 aggDocsLimitOpts = configOpts.AggregateDocsLimitOpts() 1371 ) 1372 if dynamicOpts != nil { 1373 if dynamicOpts.MaxRecentlyQueriedSeriesBlocks != nil { 1374 docsLimitOpts = dynamicLimitToLimitOpts(dynamicOpts.MaxRecentlyQueriedSeriesBlocks) 1375 } 1376 if dynamicOpts.MaxRecentlyQueriedSeriesDiskBytesRead != nil { 1377 bytesReadLimitOpts = dynamicLimitToLimitOpts(dynamicOpts.MaxRecentlyQueriedSeriesDiskBytesRead) 1378 } 1379 if dynamicOpts.MaxRecentlyQueriedSeriesDiskRead != nil { 1380 diskSeriesReadLimitOpts = dynamicLimitToLimitOpts(dynamicOpts.MaxRecentlyQueriedSeriesDiskRead) 1381 } 1382 if dynamicOpts.MaxRecentlyQueriedMetadataRead != nil { 1383 aggDocsLimitOpts = dynamicLimitToLimitOpts(dynamicOpts.MaxRecentlyQueriedMetadataRead) 1384 } 1385 } 1386 1387 if err := updateQueryLimit(docsLimit, docsLimitOpts); err != nil { 1388 logger.Error("error updating docs limit", zap.Error(err)) 1389 } 1390 1391 if err := updateQueryLimit(bytesReadLimit, bytesReadLimitOpts); err != nil { 1392 logger.Error("error updating bytes read limit", zap.Error(err)) 1393 } 1394 1395 if err := updateQueryLimit(diskSeriesReadLimit, diskSeriesReadLimitOpts); err != nil { 1396 logger.Error("error updating series read limit", zap.Error(err)) 1397 } 1398 1399 if err := updateQueryLimit(aggregateDocsLimit, aggDocsLimitOpts); err != nil { 1400 logger.Error("error updating metadata read limit", zap.Error(err)) 1401 } 1402 } 1403 1404 func updateQueryLimit( 1405 limit limits.LookbackLimit, 1406 newOpts limits.LookbackLimitOptions, 1407 ) error { 1408 old := limit.Options() 1409 if old.Equals(newOpts) { 1410 return nil 1411 } 1412 1413 return limit.Update(newOpts) 1414 } 1415 1416 func dynamicLimitToLimitOpts(dynamicLimit *kvpb.QueryLimit) limits.LookbackLimitOptions { 1417 return limits.LookbackLimitOptions{ 1418 Limit: dynamicLimit.Limit, 1419 Lookback: time.Duration(dynamicLimit.LookbackSeconds) * time.Second, 1420 ForceExceeded: dynamicLimit.ForceExceeded, 1421 ForceWaited: dynamicLimit.ForceWaited, 1422 } 1423 } 1424 1425 func kvWatchClientConsistencyLevels( 1426 store kv.Store, 1427 logger *zap.Logger, 1428 clientOpts client.AdminOptions, 1429 runtimeOptsMgr m3dbruntime.OptionsManager, 1430 ) { 1431 setReadConsistencyLevel := func( 1432 v string, 1433 applyFn func(topology.ReadConsistencyLevel, m3dbruntime.Options) m3dbruntime.Options, 1434 ) error { 1435 for _, level := range topology.ValidReadConsistencyLevels() { 1436 if level.String() == v { 1437 runtimeOpts := applyFn(level, runtimeOptsMgr.Get()) 1438 return runtimeOptsMgr.Update(runtimeOpts) 1439 } 1440 } 1441 return fmt.Errorf("invalid read consistency level set: %s", v) 1442 } 1443 1444 setConsistencyLevel := func( 1445 v string, 1446 applyFn func(topology.ConsistencyLevel, m3dbruntime.Options) m3dbruntime.Options, 1447 ) error { 1448 for _, level := range topology.ValidConsistencyLevels() { 1449 if level.String() == v { 1450 runtimeOpts := applyFn(level, runtimeOptsMgr.Get()) 1451 return runtimeOptsMgr.Update(runtimeOpts) 1452 } 1453 } 1454 return fmt.Errorf("invalid consistency level set: %s", v) 1455 } 1456 1457 kvWatchStringValue(store, logger, 1458 kvconfig.ClientBootstrapConsistencyLevel, 1459 func(value string) error { 1460 return setReadConsistencyLevel(value, 1461 func(level topology.ReadConsistencyLevel, opts m3dbruntime.Options) m3dbruntime.Options { 1462 return opts.SetClientBootstrapConsistencyLevel(level) 1463 }) 1464 }, 1465 func() error { 1466 return runtimeOptsMgr.Update(runtimeOptsMgr.Get(). 1467 SetClientBootstrapConsistencyLevel(clientOpts.BootstrapConsistencyLevel())) 1468 }) 1469 1470 kvWatchStringValue(store, logger, 1471 kvconfig.ClientReadConsistencyLevel, 1472 func(value string) error { 1473 return setReadConsistencyLevel(value, 1474 func(level topology.ReadConsistencyLevel, opts m3dbruntime.Options) m3dbruntime.Options { 1475 return opts.SetClientReadConsistencyLevel(level) 1476 }) 1477 }, 1478 func() error { 1479 return runtimeOptsMgr.Update(runtimeOptsMgr.Get(). 1480 SetClientReadConsistencyLevel(clientOpts.ReadConsistencyLevel())) 1481 }) 1482 1483 kvWatchStringValue(store, logger, 1484 kvconfig.ClientWriteConsistencyLevel, 1485 func(value string) error { 1486 return setConsistencyLevel(value, 1487 func(level topology.ConsistencyLevel, opts m3dbruntime.Options) m3dbruntime.Options { 1488 return opts.SetClientWriteConsistencyLevel(level) 1489 }) 1490 }, 1491 func() error { 1492 return runtimeOptsMgr.Update(runtimeOptsMgr.Get(). 1493 SetClientWriteConsistencyLevel(clientOpts.WriteConsistencyLevel())) 1494 }) 1495 } 1496 1497 func kvWatchStringValue( 1498 store kv.Store, 1499 logger *zap.Logger, 1500 key string, 1501 onValue func(value string) error, 1502 onDelete func() error, 1503 ) { 1504 protoValue := &commonpb.StringProto{} 1505 1506 // First try to eagerly set the value so it doesn't flap if the 1507 // watch returns but not immediately for an existing value 1508 value, err := store.Get(key) 1509 if err != nil && err != kv.ErrNotFound { 1510 logger.Error("could not resolve KV", zap.String("key", key), zap.Error(err)) 1511 } 1512 if err == nil { 1513 if err := value.Unmarshal(protoValue); err != nil { 1514 logger.Error("could not unmarshal KV key", zap.String("key", key), zap.Error(err)) 1515 } else if err := onValue(protoValue.Value); err != nil { 1516 logger.Error("could not process value of KV", zap.String("key", key), zap.Error(err)) 1517 } else { 1518 logger.Info("set KV key", zap.String("key", key), zap.Any("value", protoValue.Value)) 1519 } 1520 } 1521 1522 watch, err := store.Watch(key) 1523 if err != nil { 1524 logger.Error("could not watch KV key", zap.String("key", key), zap.Error(err)) 1525 return 1526 } 1527 1528 go func() { 1529 for range watch.C() { 1530 newValue := watch.Get() 1531 if newValue == nil { 1532 if err := onDelete(); err != nil { 1533 logger.Warn("could not set default for KV key", zap.String("key", key), zap.Error(err)) 1534 } 1535 continue 1536 } 1537 1538 err := newValue.Unmarshal(protoValue) 1539 if err != nil { 1540 logger.Warn("could not unmarshal KV key", zap.String("key", key), zap.Error(err)) 1541 continue 1542 } 1543 if err := onValue(protoValue.Value); err != nil { 1544 logger.Warn("could not process change for KV key", zap.String("key", key), zap.Error(err)) 1545 continue 1546 } 1547 logger.Info("set KV key", zap.String("key", key), zap.Any("value", protoValue.Value)) 1548 } 1549 }() 1550 } 1551 1552 func setNewSeriesLimitPerShardOnChange( 1553 topo topology.Topology, 1554 runtimeOptsMgr m3dbruntime.OptionsManager, 1555 clusterLimit int, 1556 ) error { 1557 perPlacedShardLimit := clusterLimitToPlacedShardLimit(topo, clusterLimit) 1558 runtimeOpts := runtimeOptsMgr.Get() 1559 if runtimeOpts.WriteNewSeriesLimitPerShardPerSecond() == perPlacedShardLimit { 1560 // Not changed, no need to set the value and trigger a runtime options update 1561 return nil 1562 } 1563 1564 newRuntimeOpts := runtimeOpts. 1565 SetWriteNewSeriesLimitPerShardPerSecond(perPlacedShardLimit) 1566 return runtimeOptsMgr.Update(newRuntimeOpts) 1567 } 1568 1569 func clusterLimitToPlacedShardLimit(topo topology.Topology, clusterLimit int) int { 1570 if clusterLimit < 1 { 1571 return 0 1572 } 1573 topoMap := topo.Get() 1574 numShards := len(topoMap.ShardSet().AllIDs()) 1575 numPlacedShards := numShards * topoMap.Replicas() 1576 if numPlacedShards < 1 { 1577 return 0 1578 } 1579 nodeLimit := int(math.Ceil( 1580 float64(clusterLimit) / float64(numPlacedShards))) 1581 return nodeLimit 1582 } 1583 1584 func setEncodersPerBlockLimitOnChange( 1585 runtimeOptsMgr m3dbruntime.OptionsManager, 1586 encoderLimit int, 1587 ) error { 1588 runtimeOpts := runtimeOptsMgr.Get() 1589 if runtimeOpts.EncodersPerBlockLimit() == encoderLimit { 1590 // Not changed, no need to set the value and trigger a runtime options update 1591 return nil 1592 } 1593 1594 newRuntimeOpts := runtimeOpts. 1595 SetEncodersPerBlockLimit(encoderLimit) 1596 return runtimeOptsMgr.Update(newRuntimeOpts) 1597 } 1598 1599 func withEncodingAndPoolingOptions( 1600 cfg config.DBConfiguration, 1601 logger *zap.Logger, 1602 opts storage.Options, 1603 policy config.PoolingPolicy, 1604 ) storage.Options { 1605 iOpts := opts.InstrumentOptions() 1606 scope := opts.InstrumentOptions().MetricsScope() 1607 1608 // Set the byte slice capacities for the thrift pooling. 1609 thriftBytesAllocSizes := policy.ThriftBytesPoolAllocSizesOrDefault() 1610 logger.Info("set thrift bytes pool slice sizes", 1611 zap.Ints("sizes", thriftBytesAllocSizes)) 1612 tbinarypool.SetMaxBytesPoolAlloc(thriftBytesAllocSizes...) 1613 1614 bytesPoolOpts := pool.NewObjectPoolOptions(). 1615 SetInstrumentOptions(iOpts.SetMetricsScope(scope.SubScope("bytes-pool"))) 1616 checkedBytesPoolOpts := bytesPoolOpts. 1617 SetInstrumentOptions(iOpts.SetMetricsScope(scope.SubScope("checked-bytes-pool"))) 1618 1619 buckets := make([]pool.Bucket, len(policy.BytesPool.Buckets)) 1620 for i, bucket := range policy.BytesPool.Buckets { 1621 var b pool.Bucket 1622 b.Capacity = bucket.CapacityOrDefault() 1623 b.Count = bucket.SizeOrDefault() 1624 b.Options = bytesPoolOpts. 1625 SetRefillLowWatermark(bucket.RefillLowWaterMarkOrDefault()). 1626 SetRefillHighWatermark(bucket.RefillHighWaterMarkOrDefault()) 1627 buckets[i] = b 1628 1629 logger.Info("bytes pool configured", 1630 zap.Int("capacity", bucket.CapacityOrDefault()), 1631 zap.Int("size", int(bucket.SizeOrDefault())), 1632 zap.Float64("refillLowWaterMark", bucket.RefillLowWaterMarkOrDefault()), 1633 zap.Float64("refillHighWaterMark", bucket.RefillHighWaterMarkOrDefault())) 1634 } 1635 1636 var bytesPool pool.CheckedBytesPool 1637 switch policy.TypeOrDefault() { 1638 case config.SimplePooling: 1639 bytesPool = pool.NewCheckedBytesPool( 1640 buckets, 1641 checkedBytesPoolOpts, 1642 func(s []pool.Bucket) pool.BytesPool { 1643 return pool.NewBytesPool(s, bytesPoolOpts) 1644 }) 1645 default: 1646 logger.Fatal("unrecognized pooling type", zap.Any("type", policy.Type)) 1647 } 1648 1649 { 1650 // Avoid polluting the rest of the function with `l` var 1651 l := logger 1652 if t := policy.Type; t != nil { 1653 l = l.With(zap.String("policy", string(*t))) 1654 } 1655 1656 l.Info("bytes pool init start") 1657 bytesPool.Init() 1658 l.Info("bytes pool init end") 1659 } 1660 1661 segmentReaderPool := xio.NewSegmentReaderPool( 1662 poolOptions( 1663 policy.SegmentReaderPool, 1664 scope.SubScope("segment-reader-pool"))) 1665 segmentReaderPool.Init() 1666 1667 encoderPool := encoding.NewEncoderPool( 1668 poolOptions( 1669 policy.EncoderPool, 1670 scope.SubScope("encoder-pool"))) 1671 1672 closersPoolOpts := poolOptions( 1673 policy.ClosersPool, 1674 scope.SubScope("closers-pool")) 1675 1676 contextPoolOpts := poolOptions( 1677 policy.ContextPool, 1678 scope.SubScope("context-pool")) 1679 1680 contextPool := xcontext.NewPool(xcontext.NewOptions(). 1681 SetContextPoolOptions(contextPoolOpts). 1682 SetFinalizerPoolOptions(closersPoolOpts)) 1683 1684 iteratorPool := encoding.NewReaderIteratorPool( 1685 poolOptions( 1686 policy.IteratorPool, 1687 scope.SubScope("iterator-pool"))) 1688 1689 multiIteratorPool := encoding.NewMultiReaderIteratorPool( 1690 poolOptions( 1691 policy.IteratorPool, 1692 scope.SubScope("multi-iterator-pool"))) 1693 1694 writeBatchPoolInitialBatchSize := 0 1695 if policy.WriteBatchPool.InitialBatchSize != nil { 1696 // Use config value if available. 1697 writeBatchPoolInitialBatchSize = *policy.WriteBatchPool.InitialBatchSize 1698 } 1699 1700 var writeBatchPoolMaxBatchSize *int 1701 if policy.WriteBatchPool.MaxBatchSize != nil { 1702 writeBatchPoolMaxBatchSize = policy.WriteBatchPool.MaxBatchSize 1703 } 1704 1705 var writeBatchPoolSize int 1706 if policy.WriteBatchPool.Size != nil { 1707 writeBatchPoolSize = *policy.WriteBatchPool.Size 1708 } else { 1709 // If no value set, calculate a reasonable value based on the commit log 1710 // queue size. We base it off the commitlog queue size because we will 1711 // want to be able to buffer at least one full commitlog queues worth of 1712 // writes without allocating because these objects are very expensive to 1713 // allocate. 1714 commitlogQueueSize := opts.CommitLogOptions().BacklogQueueSize() 1715 expectedBatchSize := writeBatchPoolInitialBatchSize 1716 if expectedBatchSize == 0 { 1717 expectedBatchSize = client.DefaultWriteBatchSize 1718 } 1719 writeBatchPoolSize = commitlogQueueSize / expectedBatchSize 1720 } 1721 1722 writeBatchPoolOpts := pool.NewObjectPoolOptions() 1723 writeBatchPoolOpts = writeBatchPoolOpts. 1724 SetSize(writeBatchPoolSize). 1725 // Set watermarks to zero because this pool is sized to be as large as we 1726 // ever need it to be, so background allocations are usually wasteful. 1727 SetRefillLowWatermark(0.0). 1728 SetRefillHighWatermark(0.0). 1729 SetInstrumentOptions( 1730 writeBatchPoolOpts. 1731 InstrumentOptions(). 1732 SetMetricsScope(scope.SubScope("write-batch-pool"))) 1733 1734 writeBatchPool := writes.NewWriteBatchPool( 1735 writeBatchPoolOpts, 1736 writeBatchPoolInitialBatchSize, 1737 writeBatchPoolMaxBatchSize) 1738 1739 tagPoolPolicy := policy.TagsPool 1740 identifierPool := ident.NewPool(bytesPool, ident.PoolOptions{ 1741 IDPoolOptions: poolOptions( 1742 policy.IdentifierPool, scope.SubScope("identifier-pool")), 1743 TagsPoolOptions: maxCapacityPoolOptions(tagPoolPolicy, scope.SubScope("tags-pool")), 1744 TagsCapacity: tagPoolPolicy.CapacityOrDefault(), 1745 TagsMaxCapacity: tagPoolPolicy.MaxCapacityOrDefault(), 1746 TagsIteratorPoolOptions: poolOptions( 1747 policy.TagsIteratorPool, 1748 scope.SubScope("tags-iterator-pool")), 1749 }) 1750 1751 fetchBlockMetadataResultsPoolPolicy := policy.FetchBlockMetadataResultsPool 1752 fetchBlockMetadataResultsPool := block.NewFetchBlockMetadataResultsPool( 1753 capacityPoolOptions( 1754 fetchBlockMetadataResultsPoolPolicy, 1755 scope.SubScope("fetch-block-metadata-results-pool")), 1756 fetchBlockMetadataResultsPoolPolicy.CapacityOrDefault()) 1757 1758 fetchBlocksMetadataResultsPoolPolicy := policy.FetchBlocksMetadataResultsPool 1759 fetchBlocksMetadataResultsPool := block.NewFetchBlocksMetadataResultsPool( 1760 capacityPoolOptions( 1761 fetchBlocksMetadataResultsPoolPolicy, 1762 scope.SubScope("fetch-blocks-metadata-results-pool")), 1763 fetchBlocksMetadataResultsPoolPolicy.CapacityOrDefault()) 1764 1765 bytesWrapperPoolOpts := poolOptions( 1766 policy.CheckedBytesWrapperPool, 1767 scope.SubScope("checked-bytes-wrapper-pool")) 1768 bytesWrapperPool := xpool.NewCheckedBytesWrapperPool( 1769 bytesWrapperPoolOpts) 1770 bytesWrapperPool.Init() 1771 1772 encodingOpts := encoding.NewOptions(). 1773 SetEncoderPool(encoderPool). 1774 SetReaderIteratorPool(iteratorPool). 1775 SetBytesPool(bytesPool). 1776 SetSegmentReaderPool(segmentReaderPool). 1777 SetCheckedBytesWrapperPool(bytesWrapperPool). 1778 SetMetrics(encoding.NewMetrics(scope)) 1779 1780 encoderPool.Init(func() encoding.Encoder { 1781 if cfg.Proto != nil && cfg.Proto.Enabled { 1782 enc := proto.NewEncoder(0, encodingOpts) 1783 return enc 1784 } 1785 1786 return m3tsz.NewEncoder(0, nil, m3tsz.DefaultIntOptimizationEnabled, encodingOpts) 1787 }) 1788 1789 iteratorPool.Init(func(r xio.Reader64, descr namespace.SchemaDescr) encoding.ReaderIterator { 1790 if cfg.Proto != nil && cfg.Proto.Enabled { 1791 return proto.NewIterator(r, descr, encodingOpts) 1792 } 1793 return m3tsz.NewReaderIterator(r, m3tsz.DefaultIntOptimizationEnabled, encodingOpts) 1794 }) 1795 1796 multiIteratorPool.Init(func(r xio.Reader64, descr namespace.SchemaDescr) encoding.ReaderIterator { 1797 iter := iteratorPool.Get() 1798 iter.Reset(r, descr) 1799 return iter 1800 }) 1801 1802 writeBatchPool.Init() 1803 1804 bucketPool := series.NewBufferBucketPool( 1805 poolOptions(policy.BufferBucketPool, scope.SubScope("buffer-bucket-pool"))) 1806 bucketVersionsPool := series.NewBufferBucketVersionsPool( 1807 poolOptions(policy.BufferBucketVersionsPool, scope.SubScope("buffer-bucket-versions-pool"))) 1808 1809 retrieveRequestPool := fs.NewRetrieveRequestPool(segmentReaderPool, 1810 poolOptions(policy.RetrieveRequestPool, scope.SubScope("retrieve-request-pool"))) 1811 retrieveRequestPool.Init() 1812 1813 opts = opts. 1814 SetBytesPool(bytesPool). 1815 SetContextPool(contextPool). 1816 SetEncoderPool(encoderPool). 1817 SetReaderIteratorPool(iteratorPool). 1818 SetMultiReaderIteratorPool(multiIteratorPool). 1819 SetIdentifierPool(identifierPool). 1820 SetFetchBlockMetadataResultsPool(fetchBlockMetadataResultsPool). 1821 SetFetchBlocksMetadataResultsPool(fetchBlocksMetadataResultsPool). 1822 SetWriteBatchPool(writeBatchPool). 1823 SetBufferBucketPool(bucketPool). 1824 SetBufferBucketVersionsPool(bucketVersionsPool). 1825 SetRetrieveRequestPool(retrieveRequestPool). 1826 SetCheckedBytesWrapperPool(bytesWrapperPool) 1827 1828 blockOpts := opts.DatabaseBlockOptions(). 1829 SetDatabaseBlockAllocSize(policy.BlockAllocSizeOrDefault()). 1830 SetContextPool(contextPool). 1831 SetEncoderPool(encoderPool). 1832 SetReaderIteratorPool(iteratorPool). 1833 SetMultiReaderIteratorPool(multiIteratorPool). 1834 SetSegmentReaderPool(segmentReaderPool). 1835 SetBytesPool(bytesPool) 1836 1837 if opts.SeriesCachePolicy() == series.CacheLRU { 1838 var ( 1839 runtimeOpts = opts.RuntimeOptionsManager() 1840 wiredListOpts = block.WiredListOptions{ 1841 RuntimeOptionsManager: runtimeOpts, 1842 InstrumentOptions: iOpts, 1843 ClockOptions: opts.ClockOptions(), 1844 } 1845 lruCfg = cfg.Cache.SeriesConfiguration().LRU 1846 ) 1847 1848 if lruCfg != nil && lruCfg.EventsChannelSize > 0 { 1849 wiredListOpts.EventsChannelSize = int(lruCfg.EventsChannelSize) 1850 } 1851 wiredList := block.NewWiredList(wiredListOpts) 1852 blockOpts = blockOpts.SetWiredList(wiredList) 1853 } 1854 blockPool := block.NewDatabaseBlockPool( 1855 poolOptions( 1856 policy.BlockPool, 1857 scope.SubScope("block-pool"))) 1858 blockPool.Init(func() block.DatabaseBlock { 1859 return block.NewDatabaseBlock(0, 0, ts.Segment{}, blockOpts, namespace.Context{}) 1860 }) 1861 blockOpts = blockOpts.SetDatabaseBlockPool(blockPool) 1862 opts = opts.SetDatabaseBlockOptions(blockOpts) 1863 1864 // NB(prateek): retention opts are overridden per namespace during series creation 1865 retentionOpts := retention.NewOptions() 1866 seriesOpts := storage.NewSeriesOptionsFromOptions(opts, retentionOpts). 1867 SetFetchBlockMetadataResultsPool(opts.FetchBlockMetadataResultsPool()) 1868 seriesPool := series.NewDatabaseSeriesPool( 1869 poolOptions( 1870 policy.SeriesPool, 1871 scope.SubScope("series-pool"))) 1872 1873 opts = opts. 1874 SetSeriesOptions(seriesOpts). 1875 SetDatabaseSeriesPool(seriesPool) 1876 opts = opts.SetCommitLogOptions(opts.CommitLogOptions(). 1877 SetBytesPool(bytesPool). 1878 SetIdentifierPool(identifierPool)) 1879 1880 postingsListOpts := poolOptions(policy.PostingsListPool, scope.SubScope("postingslist-pool")) 1881 postingsList := postings.NewPool(postingsListOpts, roaring.NewPostingsList) 1882 1883 queryResultsPool := index.NewQueryResultsPool( 1884 poolOptions(policy.IndexResultsPool, scope.SubScope("index-query-results-pool"))) 1885 aggregateQueryResultsPool := index.NewAggregateResultsPool( 1886 poolOptions(policy.IndexResultsPool, scope.SubScope("index-aggregate-results-pool"))) 1887 aggregateQueryValuesPool := index.NewAggregateValuesPool( 1888 poolOptions(policy.IndexResultsPool, scope.SubScope("index-aggregate-values-pool"))) 1889 1890 // Set value transformation options. 1891 opts = opts.SetTruncateType(cfg.Transforms.TruncateBy) 1892 forcedValue := cfg.Transforms.ForcedValue 1893 if forcedValue != nil { 1894 opts = opts.SetWriteTransformOptions(series.WriteTransformOptions{ 1895 ForceValueEnabled: true, 1896 ForceValue: *forcedValue, 1897 }) 1898 } 1899 1900 // Set index options. 1901 indexOpts := opts.IndexOptions(). 1902 SetInstrumentOptions(iOpts). 1903 SetMemSegmentOptions( 1904 opts.IndexOptions().MemSegmentOptions(). 1905 SetPostingsListPool(postingsList). 1906 SetInstrumentOptions(iOpts)). 1907 SetFSTSegmentOptions( 1908 opts.IndexOptions().FSTSegmentOptions(). 1909 SetPostingsListPool(postingsList). 1910 SetInstrumentOptions(iOpts). 1911 SetContextPool(opts.ContextPool())). 1912 SetSegmentBuilderOptions( 1913 opts.IndexOptions().SegmentBuilderOptions(). 1914 SetPostingsListPool(postingsList)). 1915 SetIdentifierPool(identifierPool). 1916 SetCheckedBytesPool(bytesPool). 1917 SetQueryResultsPool(queryResultsPool). 1918 SetAggregateResultsPool(aggregateQueryResultsPool). 1919 SetAggregateValuesPool(aggregateQueryValuesPool). 1920 SetForwardIndexProbability(cfg.Index.ForwardIndexProbability). 1921 SetForwardIndexThreshold(cfg.Index.ForwardIndexThreshold) 1922 1923 queryResultsPool.Init(func() index.QueryResults { 1924 // NB(r): Need to initialize after setting the index opts so 1925 // it sees the same reference of the options as is set for the DB. 1926 return index.NewQueryResults(nil, index.QueryResultsOptions{}, indexOpts) 1927 }) 1928 aggregateQueryResultsPool.Init(func() index.AggregateResults { 1929 // NB(r): Need to initialize after setting the index opts so 1930 // it sees the same reference of the options as is set for the DB. 1931 return index.NewAggregateResults(nil, index.AggregateResultsOptions{}, indexOpts) 1932 }) 1933 aggregateQueryValuesPool.Init(func() index.AggregateValues { 1934 // NB(r): Need to initialize after setting the index opts so 1935 // it sees the same reference of the options as is set for the DB. 1936 return index.NewAggregateValues(indexOpts) 1937 }) 1938 1939 return opts.SetIndexOptions(indexOpts) 1940 } 1941 1942 func newAdminClient( 1943 config client.Configuration, 1944 clockOpts clock.Options, 1945 iOpts instrument.Options, 1946 tchannelOpts *tchannel.ChannelOptions, 1947 topologyInitializer topology.Initializer, 1948 runtimeOptsMgr m3dbruntime.OptionsManager, 1949 origin topology.Host, 1950 protoEnabled bool, 1951 schemaRegistry namespace.SchemaRegistry, 1952 kvStore kv.Store, 1953 contextPool xcontext.Pool, 1954 checkedBytesPool pool.CheckedBytesPool, 1955 identifierPool ident.Pool, 1956 logger *zap.Logger, 1957 custom []client.CustomAdminOption, 1958 ) (client.AdminClient, error) { 1959 if config.EnvironmentConfig != nil { 1960 // If the user has provided an override for the dynamic client configuration 1961 // then we need to honor it by not passing our own topology initializer. 1962 topologyInitializer = nil 1963 } 1964 1965 // NB: append custom options coming from run options to existing options. 1966 options := []client.CustomAdminOption{ 1967 func(opts client.AdminOptions) client.AdminOptions { 1968 return opts.SetChannelOptions(tchannelOpts).(client.AdminOptions) 1969 }, 1970 func(opts client.AdminOptions) client.AdminOptions { 1971 return opts.SetRuntimeOptionsManager(runtimeOptsMgr).(client.AdminOptions) 1972 }, 1973 func(opts client.AdminOptions) client.AdminOptions { 1974 return opts.SetContextPool(contextPool).(client.AdminOptions) 1975 }, 1976 func(opts client.AdminOptions) client.AdminOptions { 1977 return opts.SetCheckedBytesPool(checkedBytesPool).(client.AdminOptions) 1978 }, 1979 func(opts client.AdminOptions) client.AdminOptions { 1980 return opts.SetIdentifierPool(identifierPool).(client.AdminOptions) 1981 }, 1982 func(opts client.AdminOptions) client.AdminOptions { 1983 return opts.SetOrigin(origin).(client.AdminOptions) 1984 }, 1985 func(opts client.AdminOptions) client.AdminOptions { 1986 if protoEnabled { 1987 return opts.SetEncodingProto(encoding.NewOptions()).(client.AdminOptions) 1988 } 1989 return opts 1990 }, 1991 func(opts client.AdminOptions) client.AdminOptions { 1992 return opts.SetSchemaRegistry(schemaRegistry).(client.AdminOptions) 1993 }, 1994 } 1995 1996 options = append(options, custom...) 1997 m3dbClient, err := config.NewAdminClient( 1998 client.ConfigurationParameters{ 1999 ClockOptions: clockOpts, 2000 InstrumentOptions: iOpts. 2001 SetMetricsScope(iOpts.MetricsScope().SubScope("m3dbclient")), 2002 TopologyInitializer: topologyInitializer, 2003 }, 2004 options..., 2005 ) 2006 if err != nil { 2007 return nil, err 2008 } 2009 2010 // Kick off runtime options manager KV watches. 2011 clientAdminOpts := m3dbClient.Options().(client.AdminOptions) 2012 kvWatchClientConsistencyLevels(kvStore, logger, 2013 clientAdminOpts, runtimeOptsMgr) 2014 return m3dbClient, nil 2015 } 2016 2017 func poolOptions( 2018 policy config.PoolPolicy, 2019 scope tally.Scope, 2020 ) pool.ObjectPoolOptions { 2021 var ( 2022 opts = pool.NewObjectPoolOptions() 2023 size = policy.SizeOrDefault() 2024 refillLowWaterMark = policy.RefillLowWaterMarkOrDefault() 2025 refillHighWaterMark = policy.RefillHighWaterMarkOrDefault() 2026 ) 2027 2028 if size > 0 { 2029 opts = opts.SetSize(int(size)) 2030 if refillLowWaterMark > 0 && 2031 refillHighWaterMark > 0 && 2032 refillHighWaterMark > refillLowWaterMark { 2033 opts = opts. 2034 SetRefillLowWatermark(refillLowWaterMark). 2035 SetRefillHighWatermark(refillHighWaterMark) 2036 } 2037 } 2038 opts = opts.SetDynamic(size.IsDynamic()) 2039 2040 if scope != nil { 2041 opts = opts.SetInstrumentOptions(opts.InstrumentOptions(). 2042 SetMetricsScope(scope)) 2043 } 2044 return opts 2045 } 2046 2047 func capacityPoolOptions( 2048 policy config.CapacityPoolPolicy, 2049 scope tally.Scope, 2050 ) pool.ObjectPoolOptions { 2051 var ( 2052 opts = pool.NewObjectPoolOptions() 2053 size = policy.SizeOrDefault() 2054 refillLowWaterMark = policy.RefillLowWaterMarkOrDefault() 2055 refillHighWaterMark = policy.RefillHighWaterMarkOrDefault() 2056 ) 2057 2058 if size > 0 { 2059 opts = opts.SetSize(int(size)) 2060 if refillLowWaterMark > 0 && 2061 refillHighWaterMark > 0 && 2062 refillHighWaterMark > refillLowWaterMark { 2063 opts = opts.SetRefillLowWatermark(refillLowWaterMark) 2064 opts = opts.SetRefillHighWatermark(refillHighWaterMark) 2065 } 2066 } 2067 opts = opts.SetDynamic(size.IsDynamic()) 2068 2069 if scope != nil { 2070 opts = opts.SetInstrumentOptions(opts.InstrumentOptions(). 2071 SetMetricsScope(scope)) 2072 } 2073 return opts 2074 } 2075 2076 func maxCapacityPoolOptions( 2077 policy config.MaxCapacityPoolPolicy, 2078 scope tally.Scope, 2079 ) pool.ObjectPoolOptions { 2080 var ( 2081 opts = pool.NewObjectPoolOptions() 2082 size = policy.SizeOrDefault() 2083 refillLowWaterMark = policy.RefillLowWaterMarkOrDefault() 2084 refillHighWaterMark = policy.RefillHighWaterMarkOrDefault() 2085 ) 2086 2087 if size > 0 { 2088 opts = opts.SetSize(int(size)) 2089 if refillLowWaterMark > 0 && 2090 refillHighWaterMark > 0 && 2091 refillHighWaterMark > refillLowWaterMark { 2092 opts = opts.SetRefillLowWatermark(refillLowWaterMark) 2093 opts = opts.SetRefillHighWatermark(refillHighWaterMark) 2094 } 2095 } 2096 opts = opts.SetDynamic(size.IsDynamic()) 2097 2098 if scope != nil { 2099 opts = opts.SetInstrumentOptions(opts.InstrumentOptions(). 2100 SetMetricsScope(scope)) 2101 } 2102 return opts 2103 } 2104 2105 func hostSupportsHugeTLB() (bool, error) { 2106 // Try and determine if the host supports HugeTLB in the first place 2107 withHugeTLB, err := mmap.Bytes(10, mmap.Options{ 2108 HugeTLB: mmap.HugeTLBOptions{ 2109 Enabled: true, 2110 Threshold: 0, 2111 }, 2112 }) 2113 if err != nil { 2114 return false, fmt.Errorf("could not mmap anonymous region: %v", err) 2115 } 2116 defer mmap.Munmap(withHugeTLB) 2117 2118 if withHugeTLB.Warning == nil { 2119 // If there was no warning, then the host didn't complain about 2120 // usa of huge TLB 2121 return true, nil 2122 } 2123 2124 // If we got a warning, try mmap'ing without HugeTLB 2125 withoutHugeTLB, err := mmap.Bytes(10, mmap.Options{}) 2126 if err != nil { 2127 return false, fmt.Errorf("could not mmap anonymous region: %v", err) 2128 } 2129 defer mmap.Munmap(withoutHugeTLB) 2130 if withoutHugeTLB.Warning == nil { 2131 // The machine doesn't support HugeTLB, proceed without it 2132 return false, nil 2133 } 2134 // The warning was probably caused by something else, proceed using HugeTLB 2135 return true, nil 2136 } 2137 2138 func newTopoMapProvider(t topology.Topology) *topoMapProvider { 2139 return &topoMapProvider{t} 2140 } 2141 2142 type topoMapProvider struct { 2143 t topology.Topology 2144 } 2145 2146 func (t *topoMapProvider) TopologyMap() (topology.Map, error) { 2147 if t.t == nil { 2148 return nil, errors.New("topology map provider has not be set yet") 2149 } 2150 2151 return t.t.Get(), nil 2152 } 2153 2154 // Ensure mmap reporter implements mmap.Reporter 2155 var _ mmap.Reporter = (*mmapReporter)(nil) 2156 2157 type mmapReporter struct { 2158 sync.Mutex 2159 scope tally.Scope 2160 entries map[string]*mmapReporterEntry 2161 } 2162 2163 type mmapReporterEntry struct { 2164 value int64 2165 gauge tally.Gauge 2166 } 2167 2168 func newMmapReporter(scope tally.Scope) *mmapReporter { 2169 return &mmapReporter{ 2170 scope: scope, 2171 entries: make(map[string]*mmapReporterEntry), 2172 } 2173 } 2174 2175 func (r *mmapReporter) Run(ctx context.Context) { 2176 ticker := time.NewTicker(30 * time.Second) 2177 defer ticker.Stop() 2178 2179 for { 2180 select { 2181 case <-ctx.Done(): 2182 return 2183 case <-ticker.C: 2184 r.Lock() 2185 for _, r := range r.entries { 2186 r.gauge.Update(float64(r.value)) 2187 } 2188 r.Unlock() 2189 } 2190 } 2191 } 2192 2193 func (r *mmapReporter) entryKeyAndTags(ctx mmap.Context) (string, map[string]string) { 2194 numTags := 1 2195 if ctx.Metadata != nil { 2196 numTags += len(ctx.Metadata) 2197 } 2198 2199 tags := make(map[string]string, numTags) 2200 tags[mmapReporterTagName] = ctx.Name 2201 if ctx.Metadata != nil { 2202 for k, v := range ctx.Metadata { 2203 tags[k] = v 2204 } 2205 } 2206 2207 entryKey := tally.KeyForStringMap(tags) 2208 return entryKey, tags 2209 } 2210 2211 func (r *mmapReporter) ReportMap(ctx mmap.Context) error { 2212 if ctx.Name == "" { 2213 return fmt.Errorf("report mmap map missing context name: %+v", ctx) 2214 } 2215 2216 entryKey, entryTags := r.entryKeyAndTags(ctx) 2217 2218 r.Lock() 2219 defer r.Unlock() 2220 2221 entry, ok := r.entries[entryKey] 2222 if !ok { 2223 entry = &mmapReporterEntry{ 2224 gauge: r.scope.Tagged(entryTags).Gauge(mmapReporterMetricName), 2225 } 2226 r.entries[entryKey] = entry 2227 } 2228 2229 entry.value += ctx.Size 2230 2231 return nil 2232 } 2233 2234 func (r *mmapReporter) ReportUnmap(ctx mmap.Context) error { 2235 if ctx.Name == "" { 2236 return fmt.Errorf("report mmap unmap missing context name: %+v", ctx) 2237 } 2238 2239 entryKey, _ := r.entryKeyAndTags(ctx) 2240 2241 r.Lock() 2242 defer r.Unlock() 2243 2244 entry, ok := r.entries[entryKey] 2245 if !ok { 2246 return fmt.Errorf("report mmap unmap missing entry for context: %+v", ctx) 2247 } 2248 2249 entry.value -= ctx.Size 2250 2251 if entry.value == 0 { 2252 // No more similar mmaps active for this context name, garbage collect 2253 delete(r.entries, entryKey) 2254 } 2255 2256 return nil 2257 }