github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/client/session.go (about) 1 // Copyright (c) 2016 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 client 22 23 import ( 24 "bytes" 25 gocontext "context" 26 "errors" 27 "fmt" 28 "math" 29 "sort" 30 "strings" 31 "sync" 32 "sync/atomic" 33 "time" 34 35 "github.com/m3db/m3/src/cluster/shard" 36 "github.com/m3db/m3/src/dbnode/digest" 37 "github.com/m3db/m3/src/dbnode/encoding" 38 "github.com/m3db/m3/src/dbnode/generated/thrift/rpc" 39 "github.com/m3db/m3/src/dbnode/namespace" 40 "github.com/m3db/m3/src/dbnode/network/server/tchannelthrift/convert" 41 "github.com/m3db/m3/src/dbnode/runtime" 42 "github.com/m3db/m3/src/dbnode/storage/block" 43 "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" 44 "github.com/m3db/m3/src/dbnode/storage/index" 45 idxconvert "github.com/m3db/m3/src/dbnode/storage/index/convert" 46 "github.com/m3db/m3/src/dbnode/topology" 47 "github.com/m3db/m3/src/dbnode/ts" 48 "github.com/m3db/m3/src/dbnode/x/xio" 49 "github.com/m3db/m3/src/dbnode/x/xpool" 50 "github.com/m3db/m3/src/x/checked" 51 "github.com/m3db/m3/src/x/clock" 52 "github.com/m3db/m3/src/x/context" 53 xerrors "github.com/m3db/m3/src/x/errors" 54 "github.com/m3db/m3/src/x/ident" 55 "github.com/m3db/m3/src/x/instrument" 56 "github.com/m3db/m3/src/x/pool" 57 xresource "github.com/m3db/m3/src/x/resource" 58 xretry "github.com/m3db/m3/src/x/retry" 59 "github.com/m3db/m3/src/x/sampler" 60 "github.com/m3db/m3/src/x/serialize" 61 xsync "github.com/m3db/m3/src/x/sync" 62 tbinarypool "github.com/m3db/m3/src/x/thrift" 63 xtime "github.com/m3db/m3/src/x/time" 64 65 "github.com/uber-go/tally" 66 "github.com/uber/tchannel-go/thrift" 67 "go.uber.org/zap" 68 "go.uber.org/zap/zapcore" 69 ) 70 71 const ( 72 clusterConnectWaitInterval = 10 * time.Millisecond 73 gaugeReportInterval = 500 * time.Millisecond 74 blockMetadataChBufSize = 65536 75 hostNotAvailableMinSleepInterval = 1 * time.Millisecond 76 hostNotAvailableMaxSleepInterval = 100 * time.Millisecond 77 ) 78 79 type resultTypeEnum string 80 81 const ( 82 resultTypeMetadata resultTypeEnum = "metadata" 83 resultTypeBootstrap = "bootstrap" 84 resultTypeRaw = "raw" 85 ) 86 87 var errUnknownWriteAttemptType = errors.New( 88 "unknown write attempt type specified, internal error") 89 90 var ( 91 // ErrClusterConnectTimeout is raised when connecting to the cluster and 92 // ensuring at least each partition has an up node with a connection to it 93 ErrClusterConnectTimeout = errors.New("timed out establishing min connections to cluster") 94 // errSessionStatusNotInitial is raised when trying to open a session and 95 // its not in the initial clean state 96 errSessionStatusNotInitial = errors.New("session not in initial state") 97 // ErrSessionStatusNotOpen is raised when operations are requested when the 98 // session is not in the open state 99 ErrSessionStatusNotOpen = errors.New("session not in open state") 100 // errSessionBadBlockResultFromPeer is raised when there is a bad block 101 // return from a peer when fetching blocks from peers 102 errSessionBadBlockResultFromPeer = errors.New("session fetched bad block result from peer") 103 // errSessionInvalidConnectClusterConnectConsistencyLevel is raised when 104 // the connect consistency level specified is not recognized 105 errSessionInvalidConnectClusterConnectConsistencyLevel = errors.New( 106 "session has invalid connect consistency level specified", 107 ) 108 // errSessionHasNoHostQueueForHost is raised when host queue requested for a missing host 109 errSessionHasNoHostQueueForHost = newHostNotAvailableError(errors.New("session has no host queue for host")) 110 // errUnableToEncodeTags is raised when the server is unable to encode provided tags 111 // to be sent over the wire. 112 errUnableToEncodeTags = errors.New("unable to include tags") 113 // errEnqueueChIsClosed is returned when attempting to use a closed enqueuCh. 114 errEnqueueChIsClosed = errors.New("error enqueueCh is cosed") 115 ) 116 117 // sessionState is volatile state that is protected by a 118 // read/write mutex 119 type sessionState struct { 120 sync.RWMutex 121 122 status status 123 124 writeLevel topology.ConsistencyLevel 125 readLevel topology.ReadConsistencyLevel 126 bootstrapLevel topology.ReadConsistencyLevel 127 128 queues []hostQueue 129 queuesByHostID map[string]hostQueue 130 topo topology.Topology 131 topoMap topology.Map 132 topoWatch topology.MapWatch 133 replicas int 134 majority int 135 } 136 137 func (s *sessionState) readConsistencyLevelWithRLock( 138 override *topology.ReadConsistencyLevel, 139 ) topology.ReadConsistencyLevel { 140 if override == nil { 141 return s.readLevel 142 } 143 return *override 144 } 145 146 type session struct { 147 state sessionState 148 opts Options 149 runtimeOptsListenerCloser xresource.SimpleCloser 150 scope tally.Scope 151 nowFn clock.NowFn 152 log *zap.Logger 153 logWriteErrorSampler *sampler.Sampler 154 logFetchErrorSampler *sampler.Sampler 155 logHostWriteErrorSampler *sampler.Sampler 156 logHostFetchErrorSampler *sampler.Sampler 157 newHostQueueFn newHostQueueFn 158 writeRetrier xretry.Retrier 159 fetchRetrier xretry.Retrier 160 streamBlocksRetrier xretry.Retrier 161 pools sessionPools 162 fetchBatchSize int 163 newPeerBlocksQueueFn newPeerBlocksQueueFn 164 reattemptStreamBlocksFromPeersFn reattemptStreamBlocksFromPeersFn 165 pickBestPeerFn pickBestPeerFn 166 healthCheckNewConnFn healthCheckFn 167 origin topology.Host 168 streamBlocksMaxBlockRetries int 169 streamBlocksWorkers xsync.WorkerPool 170 streamBlocksBatchSize int 171 streamBlocksMetadataBatchTimeout time.Duration 172 streamBlocksBatchTimeout time.Duration 173 writeShardsInitializing bool 174 shardsLeavingCountTowardsConsistency bool 175 shardsLeavingAndInitializingCountTowardsConsistency bool 176 metrics sessionMetrics 177 } 178 179 type shardMetricsKey struct { 180 shardID uint32 181 resultType resultTypeEnum 182 } 183 184 type sessionMetrics struct { 185 sync.RWMutex 186 writeSuccess tally.Counter 187 writeSuccessForCountLeavingAndInitializingAsPair tally.Counter 188 writeErrorsBadRequest tally.Counter 189 writeErrorsInternalError tally.Counter 190 writeLatencyHistogram tally.Histogram 191 writeNodesRespondingErrors []tally.Counter 192 writeNodesRespondingBadRequestErrors []tally.Counter 193 fetchSuccess tally.Counter 194 fetchErrorsBadRequest tally.Counter 195 fetchErrorsInternalError tally.Counter 196 fetchLatencyHistogram tally.Histogram 197 fetchNodesRespondingErrors []tally.Counter 198 fetchNodesRespondingBadRequestErrors []tally.Counter 199 topologyUpdatedSuccess tally.Counter 200 topologyUpdatedError tally.Counter 201 streamFromPeersMetrics map[shardMetricsKey]streamFromPeersMetrics 202 } 203 204 func newSessionMetrics(scope tally.Scope) sessionMetrics { 205 return sessionMetrics{ 206 writeSuccess: scope.Counter("write.success"), 207 writeSuccessForCountLeavingAndInitializingAsPair: scope.Tagged(map[string]string{ 208 "success_type": "leaving_initializing_as_pair", 209 }).Counter("write.success"), 210 writeErrorsBadRequest: scope.Tagged(map[string]string{ 211 "error_type": "bad_request", 212 }).Counter("write.errors"), 213 writeErrorsInternalError: scope.Tagged(map[string]string{ 214 "error_type": "internal_error", 215 }).Counter("write.errors"), 216 writeLatencyHistogram: histogramWithDurationBuckets(scope, "write.latency"), 217 fetchSuccess: scope.Counter("fetch.success"), 218 fetchErrorsBadRequest: scope.Tagged(map[string]string{ 219 "error_type": "bad_request", 220 }).Counter("fetch.errors"), 221 fetchErrorsInternalError: scope.Tagged(map[string]string{ 222 "error_type": "internal_error", 223 }).Counter("fetch.errors"), 224 fetchLatencyHistogram: histogramWithDurationBuckets(scope, "fetch.latency"), 225 topologyUpdatedSuccess: scope.Counter("topology.updated-success"), 226 topologyUpdatedError: scope.Counter("topology.updated-error"), 227 streamFromPeersMetrics: make(map[shardMetricsKey]streamFromPeersMetrics), 228 } 229 } 230 231 type streamFromPeersMetrics struct { 232 fetchBlocksFromPeers tally.Gauge 233 metadataFetches tally.Gauge 234 metadataFetchBatchCall tally.Counter 235 metadataFetchBatchSuccess tally.Counter 236 metadataFetchBatchError tally.Counter 237 metadataFetchBatchBlockErr tally.Counter 238 metadataReceived tally.Counter 239 metadataPeerRetry tally.Counter 240 fetchBlockSuccess tally.Counter 241 fetchBlockError tally.Counter 242 fetchBlockFullRetry tally.Counter 243 fetchBlockFinalError tally.Counter 244 fetchBlockRetriesReqError tally.Counter 245 fetchBlockRetriesRespError tally.Counter 246 fetchBlockRetriesConsistencyLevelNotAchievedError tally.Counter 247 blocksEnqueueChannel tally.Gauge 248 } 249 250 type hostQueueOpts struct { 251 writeBatchRawRequestPool writeBatchRawRequestPool 252 writeBatchRawV2RequestPool writeBatchRawV2RequestPool 253 writeBatchRawRequestElementArrayPool writeBatchRawRequestElementArrayPool 254 writeBatchRawV2RequestElementArrayPool writeBatchRawV2RequestElementArrayPool 255 writeTaggedBatchRawRequestPool writeTaggedBatchRawRequestPool 256 writeTaggedBatchRawV2RequestPool writeTaggedBatchRawV2RequestPool 257 writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawRequestElementArrayPool 258 writeTaggedBatchRawV2RequestElementArrayPool writeTaggedBatchRawV2RequestElementArrayPool 259 fetchBatchRawV2RequestPool fetchBatchRawV2RequestPool 260 fetchBatchRawV2RequestElementArrayPool fetchBatchRawV2RequestElementArrayPool 261 opts Options 262 } 263 264 type newHostQueueFn func( 265 host topology.Host, 266 hostQueueOpts hostQueueOpts, 267 ) (hostQueue, error) 268 269 func newSession(opts Options) (clientSession, error) { 270 topo, err := opts.TopologyInitializer().Init() 271 if err != nil { 272 return nil, err 273 } 274 275 logWriteErrorSampler, err := sampler.NewSampler(opts.LogErrorSampleRate()) 276 if err != nil { 277 return nil, err 278 } 279 280 logFetchErrorSampler, err := sampler.NewSampler(opts.LogErrorSampleRate()) 281 if err != nil { 282 return nil, err 283 } 284 285 logHostWriteErrorSampler, err := sampler.NewSampler(opts.LogHostWriteErrorSampleRate()) 286 if err != nil { 287 return nil, err 288 } 289 290 logHostFetchErrorSampler, err := sampler.NewSampler(opts.LogHostFetchErrorSampleRate()) 291 if err != nil { 292 return nil, err 293 } 294 295 scope := opts.InstrumentOptions().MetricsScope() 296 297 s := &session{ 298 state: sessionState{ 299 writeLevel: opts.WriteConsistencyLevel(), 300 readLevel: opts.ReadConsistencyLevel(), 301 queuesByHostID: make(map[string]hostQueue), 302 topo: topo, 303 }, 304 opts: opts, 305 scope: scope, 306 nowFn: opts.ClockOptions().NowFn(), 307 log: opts.InstrumentOptions().Logger(), 308 logWriteErrorSampler: logWriteErrorSampler, 309 logFetchErrorSampler: logFetchErrorSampler, 310 logHostWriteErrorSampler: logHostWriteErrorSampler, 311 logHostFetchErrorSampler: logHostFetchErrorSampler, 312 newHostQueueFn: newHostQueue, 313 fetchBatchSize: opts.FetchBatchSize(), 314 newPeerBlocksQueueFn: newPeerBlocksQueue, 315 healthCheckNewConnFn: healthCheck, 316 writeRetrier: opts.WriteRetrier(), 317 fetchRetrier: opts.FetchRetrier(), 318 pools: sessionPools{ 319 context: opts.ContextPool(), 320 checkedBytes: opts.CheckedBytesPool(), 321 id: opts.IdentifierPool(), 322 }, 323 writeShardsInitializing: opts.WriteShardsInitializing(), 324 shardsLeavingCountTowardsConsistency: opts.ShardsLeavingCountTowardsConsistency(), 325 shardsLeavingAndInitializingCountTowardsConsistency: opts.ShardsLeavingAndInitializingCountTowardsConsistency(), 326 metrics: newSessionMetrics(scope), 327 } 328 s.reattemptStreamBlocksFromPeersFn = s.streamBlocksReattemptFromPeers 329 s.pickBestPeerFn = s.streamBlocksPickBestPeer 330 writeAttemptPoolOpts := pool.NewObjectPoolOptions(). 331 SetDynamic(s.opts.WriteOpPoolSize().IsDynamic()). 332 SetSize(int(s.opts.WriteOpPoolSize())). 333 SetInstrumentOptions(opts.InstrumentOptions().SetMetricsScope( 334 scope.SubScope("write-attempt-pool"), 335 )) 336 s.pools.writeAttempt = newWriteAttemptPool(s, writeAttemptPoolOpts) 337 s.pools.writeAttempt.Init() 338 339 fetchAttemptPoolOpts := pool.NewObjectPoolOptions(). 340 SetDynamic(s.opts.FetchBatchOpPoolSize().IsDynamic()). 341 SetSize(int(s.opts.FetchBatchOpPoolSize())). 342 SetInstrumentOptions(opts.InstrumentOptions().SetMetricsScope( 343 scope.SubScope("fetch-attempt-pool"), 344 )) 345 s.pools.fetchAttempt = newFetchAttemptPool(s, fetchAttemptPoolOpts) 346 s.pools.fetchAttempt.Init() 347 348 fetchTaggedAttemptPoolImplOpts := pool.NewObjectPoolOptions(). 349 SetDynamic(s.opts.FetchBatchOpPoolSize().IsDynamic()). 350 SetSize(int(s.opts.FetchBatchOpPoolSize())). 351 SetInstrumentOptions(opts.InstrumentOptions().SetMetricsScope( 352 scope.SubScope("fetch-tagged-attempt-pool"), 353 )) 354 s.pools.fetchTaggedAttempt = newFetchTaggedAttemptPool(s, fetchTaggedAttemptPoolImplOpts) 355 s.pools.fetchTaggedAttempt.Init() 356 357 aggregateAttemptPoolImplOpts := pool.NewObjectPoolOptions(). 358 SetDynamic(s.opts.FetchBatchOpPoolSize().IsDynamic()). 359 SetSize(int(s.opts.FetchBatchOpPoolSize())). 360 SetInstrumentOptions(opts.InstrumentOptions().SetMetricsScope( 361 scope.SubScope("aggregate-attempt-pool"), 362 )) 363 s.pools.aggregateAttempt = newAggregateAttemptPool(s, aggregateAttemptPoolImplOpts) 364 s.pools.aggregateAttempt.Init() 365 366 tagEncoderPoolOpts := pool.NewObjectPoolOptions(). 367 SetDynamic(s.opts.TagEncoderPoolSize().IsDynamic()). 368 SetSize(int(s.opts.TagEncoderPoolSize())). 369 SetInstrumentOptions(opts.InstrumentOptions().SetMetricsScope( 370 scope.SubScope("tag-encoder-pool"), 371 )) 372 s.pools.tagEncoder = serialize.NewTagEncoderPool(opts.TagEncoderOptions(), tagEncoderPoolOpts) 373 s.pools.tagEncoder.Init() 374 375 tagDecoderPoolOpts := pool.NewObjectPoolOptions(). 376 SetDynamic(s.opts.TagDecoderPoolSize().IsDynamic()). 377 SetSize(int(s.opts.TagDecoderPoolSize())). 378 SetInstrumentOptions(opts.InstrumentOptions().SetMetricsScope( 379 scope.SubScope("tag-decoder-pool"), 380 )) 381 s.pools.tagDecoder = serialize.NewTagDecoderPool(opts.TagDecoderOptions(), tagDecoderPoolOpts) 382 s.pools.tagDecoder.Init() 383 384 wrapperPoolOpts := pool.NewObjectPoolOptions(). 385 SetDynamic(s.opts.CheckedBytesWrapperPoolSize().IsDynamic()). 386 SetSize(int(s.opts.CheckedBytesWrapperPoolSize())). 387 SetInstrumentOptions(opts.InstrumentOptions().SetMetricsScope( 388 scope.SubScope("client-checked-bytes-wrapper-pool"))) 389 s.pools.checkedBytesWrapper = xpool.NewCheckedBytesWrapperPool(wrapperPoolOpts) 390 s.pools.checkedBytesWrapper.Init() 391 392 if opts, ok := opts.(AdminOptions); ok { 393 s.state.bootstrapLevel = opts.BootstrapConsistencyLevel() 394 s.origin = opts.Origin() 395 s.streamBlocksMaxBlockRetries = opts.FetchSeriesBlocksMaxBlockRetries() 396 s.streamBlocksWorkers = xsync.NewWorkerPool(opts.FetchSeriesBlocksBatchConcurrency()) 397 s.streamBlocksWorkers.Init() 398 s.streamBlocksBatchSize = opts.FetchSeriesBlocksBatchSize() 399 s.streamBlocksMetadataBatchTimeout = opts.FetchSeriesBlocksMetadataBatchTimeout() 400 s.streamBlocksBatchTimeout = opts.FetchSeriesBlocksBatchTimeout() 401 s.streamBlocksRetrier = opts.StreamBlocksRetrier() 402 } 403 404 if runtimeOptsMgr := opts.RuntimeOptionsManager(); runtimeOptsMgr != nil { 405 runtimeOptsMgr.RegisterListener(s) 406 } 407 408 return s, nil 409 } 410 411 func (s *session) SetRuntimeOptions(value runtime.Options) { 412 s.state.Lock() 413 s.state.bootstrapLevel = value.ClientBootstrapConsistencyLevel() 414 s.state.readLevel = value.ClientReadConsistencyLevel() 415 s.state.writeLevel = value.ClientWriteConsistencyLevel() 416 s.state.Unlock() 417 } 418 419 func (s *session) ShardID(id ident.ID) (uint32, error) { 420 s.state.RLock() 421 if s.state.status != statusOpen { 422 s.state.RUnlock() 423 return 0, ErrSessionStatusNotOpen 424 } 425 value := s.state.topoMap.ShardSet().Lookup(id) 426 s.state.RUnlock() 427 return value, nil 428 } 429 430 // newPeerMetadataStreamingProgressMetrics returns a struct with an embedded 431 // list of fields that can be used to emit metrics about the current state of 432 // the peer metadata streaming process 433 func (s *session) newPeerMetadataStreamingProgressMetrics( 434 shard uint32, 435 resultType resultTypeEnum, 436 ) *streamFromPeersMetrics { 437 mKey := shardMetricsKey{shardID: shard, resultType: resultType} 438 s.metrics.RLock() 439 m, ok := s.metrics.streamFromPeersMetrics[mKey] 440 s.metrics.RUnlock() 441 442 if ok { 443 return &m 444 } 445 446 scope := s.opts.InstrumentOptions().MetricsScope() 447 448 s.metrics.Lock() 449 m, ok = s.metrics.streamFromPeersMetrics[mKey] 450 if ok { 451 s.metrics.Unlock() 452 return &m 453 } 454 scope = scope.SubScope("stream-from-peers").Tagged(map[string]string{ 455 "shard": fmt.Sprintf("%d", shard), 456 "resultType": string(resultType), 457 }) 458 m = streamFromPeersMetrics{ 459 fetchBlocksFromPeers: scope.Gauge("fetch-blocks-inprogress"), 460 metadataFetches: scope.Gauge("fetch-metadata-peers-inprogress"), 461 metadataFetchBatchCall: scope.Counter("fetch-metadata-peers-batch-call"), 462 metadataFetchBatchSuccess: scope.Counter("fetch-metadata-peers-batch-success"), 463 metadataFetchBatchError: scope.Counter("fetch-metadata-peers-batch-error"), 464 metadataFetchBatchBlockErr: scope.Counter("fetch-metadata-peers-batch-block-err"), 465 metadataReceived: scope.Counter("fetch-metadata-peers-received"), 466 metadataPeerRetry: scope.Counter("fetch-metadata-peers-peer-retry"), 467 fetchBlockSuccess: scope.Counter("fetch-block-success"), 468 fetchBlockError: scope.Counter("fetch-block-error"), 469 fetchBlockFinalError: scope.Counter("fetch-block-final-error"), 470 fetchBlockFullRetry: scope.Counter("fetch-block-full-retry"), 471 fetchBlockRetriesReqError: scope.Tagged(map[string]string{ 472 "reason": "request-error", 473 }).Counter("fetch-block-retries"), 474 fetchBlockRetriesRespError: scope.Tagged(map[string]string{ 475 "reason": "response-error", 476 }).Counter("fetch-block-retries"), 477 fetchBlockRetriesConsistencyLevelNotAchievedError: scope.Tagged(map[string]string{ 478 "reason": "consistency-level-not-achieved-error", 479 }).Counter("fetch-block-retries"), 480 blocksEnqueueChannel: scope.Gauge("fetch-blocks-enqueue-channel-length"), 481 } 482 s.metrics.streamFromPeersMetrics[mKey] = m 483 s.metrics.Unlock() 484 return &m 485 } 486 487 func (s *session) recordWriteMetrics(consistencyResultErr error, state *writeState, start time.Time) { 488 respErrs := int32(len(state.errors)) 489 if idx := s.nodesRespondingErrorsMetricIndex(respErrs); idx >= 0 { 490 if IsBadRequestError(consistencyResultErr) { 491 s.metrics.writeNodesRespondingBadRequestErrors[idx].Inc(1) 492 } else { 493 s.metrics.writeNodesRespondingErrors[idx].Inc(1) 494 } 495 } 496 if consistencyResultErr == nil { 497 498 if state.leavingAndInitializingPairCounted { 499 s.metrics.writeSuccessForCountLeavingAndInitializingAsPair.Inc(1) 500 } else { 501 s.metrics.writeSuccess.Inc(1) 502 } 503 } else if IsBadRequestError(consistencyResultErr) { 504 s.metrics.writeErrorsBadRequest.Inc(1) 505 } else { 506 s.metrics.writeErrorsInternalError.Inc(1) 507 } 508 s.metrics.writeLatencyHistogram.RecordDuration(s.nowFn().Sub(start)) 509 510 if consistencyResultErr != nil && s.logWriteErrorSampler.Sample() { 511 s.log.Error("m3db client write error occurred", 512 zap.Float64("sampleRateLog", s.logWriteErrorSampler.SampleRate().Value()), 513 zap.Error(consistencyResultErr)) 514 } 515 } 516 517 func (s *session) recordFetchMetrics(consistencyResultErr error, respErrs int32, start time.Time) { 518 if idx := s.nodesRespondingErrorsMetricIndex(respErrs); idx >= 0 { 519 if IsBadRequestError(consistencyResultErr) { 520 s.metrics.fetchNodesRespondingBadRequestErrors[idx].Inc(1) 521 } else { 522 s.metrics.fetchNodesRespondingErrors[idx].Inc(1) 523 } 524 } 525 if consistencyResultErr == nil { 526 s.metrics.fetchSuccess.Inc(1) 527 } else if IsBadRequestError(consistencyResultErr) { 528 s.metrics.fetchErrorsBadRequest.Inc(1) 529 } else { 530 s.metrics.fetchErrorsInternalError.Inc(1) 531 } 532 s.metrics.fetchLatencyHistogram.RecordDuration(s.nowFn().Sub(start)) 533 534 if consistencyResultErr != nil && s.logFetchErrorSampler.Sample() { 535 s.log.Error("m3db client fetch error occurred", 536 zap.Float64("sampleRateLog", s.logFetchErrorSampler.SampleRate().Value()), 537 zap.Error(consistencyResultErr)) 538 } 539 } 540 541 func (s *session) nodesRespondingErrorsMetricIndex(respErrs int32) int32 { 542 idx := respErrs - 1 543 replicas := int32(s.Replicas()) 544 if respErrs > replicas { 545 // Cap to the max replicas, we might get more errors 546 // when a node is initializing a shard causing replicas + 1 547 // nodes to respond to operations 548 idx = replicas - 1 549 } 550 return idx 551 } 552 553 func (s *session) Open() error { 554 s.state.Lock() 555 if s.state.status != statusNotOpen { 556 s.state.Unlock() 557 return errSessionStatusNotInitial 558 } 559 560 watch, err := s.state.topo.Watch() 561 if err != nil { 562 s.state.Unlock() 563 return err 564 } 565 566 // Wait for the topology to be available 567 <-watch.C() 568 569 topoMap := watch.Get() 570 571 queues, replicas, majority, err := s.hostQueues(topoMap, nil) 572 if err != nil { 573 s.state.Unlock() 574 return err 575 } 576 s.setTopologyWithLock(topoMap, queues, replicas, majority) 577 s.state.topoWatch = watch 578 579 // NB(r): Alloc pools that can take some time in Open, expectation 580 // is already that Open will take some time 581 writeOperationPoolOpts := pool.NewObjectPoolOptions(). 582 SetDynamic(s.opts.WriteOpPoolSize().IsDynamic()). 583 SetSize(int(s.opts.WriteOpPoolSize())). 584 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 585 s.scope.SubScope("write-op-pool"), 586 )) 587 s.pools.writeOperation = newWriteOperationPool(writeOperationPoolOpts) 588 s.pools.writeOperation.Init() 589 590 writeTaggedOperationPoolOpts := pool.NewObjectPoolOptions(). 591 SetDynamic(s.opts.WriteTaggedOpPoolSize().IsDynamic()). 592 SetSize(int(s.opts.WriteTaggedOpPoolSize())). 593 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 594 s.scope.SubScope("write-op-tagged-pool"), 595 )) 596 s.pools.writeTaggedOperation = newWriteTaggedOpPool(writeTaggedOperationPoolOpts) 597 s.pools.writeTaggedOperation.Init() 598 599 writeStatePoolOpts := pool.NewObjectPoolOptions(). 600 SetDynamic(s.opts.WriteOpPoolSize().IsDynamic()). 601 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 602 s.scope.SubScope("write-state-pool"), 603 )) 604 605 if !s.opts.WriteOpPoolSize().IsDynamic() { 606 writeStatePoolSize := s.opts.WriteOpPoolSize() 607 if !s.opts.WriteTaggedOpPoolSize().IsDynamic() && s.opts.WriteTaggedOpPoolSize() > writeStatePoolSize { 608 writeStatePoolSize = s.opts.WriteTaggedOpPoolSize() 609 } 610 writeStatePoolOpts = writeStatePoolOpts.SetSize(int(writeStatePoolSize)) 611 } 612 s.pools.writeState = newWriteStatePool(s.pools.tagEncoder, writeStatePoolOpts, s.log, 613 s.logHostWriteErrorSampler) 614 s.pools.writeState.Init() 615 616 fetchBatchOpPoolOpts := pool.NewObjectPoolOptions(). 617 SetDynamic(s.opts.FetchBatchOpPoolSize().IsDynamic()). 618 SetSize(int(s.opts.FetchBatchOpPoolSize())). 619 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 620 s.scope.SubScope("fetch-batch-op-pool"), 621 )) 622 s.pools.fetchBatchOp = newFetchBatchOpPool(fetchBatchOpPoolOpts, s.fetchBatchSize) 623 s.pools.fetchBatchOp.Init() 624 625 fetchTaggedOpPoolOpts := pool.NewObjectPoolOptions(). 626 SetDynamic(s.opts.FetchBatchOpPoolSize().IsDynamic()). 627 SetSize(int(s.opts.FetchBatchOpPoolSize())). 628 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 629 s.scope.SubScope("fetch-tagged-op-pool"), 630 )) 631 s.pools.fetchTaggedOp = newFetchTaggedOpPool(fetchTaggedOpPoolOpts) 632 s.pools.fetchTaggedOp.Init() 633 634 aggregateOpPoolOpts := pool.NewObjectPoolOptions(). 635 SetDynamic(s.opts.FetchBatchOpPoolSize().IsDynamic()). 636 SetSize(int(s.opts.FetchBatchOpPoolSize())). 637 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 638 s.scope.SubScope("aggregate-op-pool"), 639 )) 640 s.pools.aggregateOp = newAggregateOpPool(aggregateOpPoolOpts) 641 s.pools.aggregateOp.Init() 642 643 fetchStatePoolOpts := pool.NewObjectPoolOptions(). 644 SetDynamic(s.opts.FetchBatchOpPoolSize().IsDynamic()). 645 SetSize(int(s.opts.FetchBatchOpPoolSize())). 646 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 647 s.scope.SubScope("fetch-tagged-state-pool"), 648 )) 649 s.pools.fetchState = newFetchStatePool(fetchStatePoolOpts, s.log, 650 s.logHostFetchErrorSampler) 651 s.pools.fetchState.Init() 652 653 seriesIteratorPoolOpts := pool.NewObjectPoolOptions(). 654 SetDynamic(s.opts.SeriesIteratorPoolSize().IsDynamic()). 655 SetSize(int(s.opts.SeriesIteratorPoolSize())). 656 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 657 s.scope.SubScope("series-iterator-pool"), 658 )) 659 s.pools.seriesIterator = encoding.NewSeriesIteratorPool(seriesIteratorPoolOpts) 660 s.pools.seriesIterator.Init() 661 s.state.status = statusOpen 662 s.state.Unlock() 663 664 go func() { 665 for range watch.C() { 666 s.log.Info("received update for topology") 667 topoMap := watch.Get() 668 669 s.state.RLock() 670 existingQueues := s.state.queues 671 s.state.RUnlock() 672 673 queues, replicas, majority, err := s.hostQueues(topoMap, existingQueues) 674 if err != nil { 675 s.log.Error("could not update topology map", zap.Error(err)) 676 s.metrics.topologyUpdatedError.Inc(1) 677 continue 678 } 679 s.state.Lock() 680 s.setTopologyWithLock(topoMap, queues, replicas, majority) 681 s.state.Unlock() 682 s.metrics.topologyUpdatedSuccess.Inc(1) 683 } 684 }() 685 686 return nil 687 } 688 689 func (s *session) BorrowConnections( 690 shardID uint32, 691 fn WithBorrowConnectionFn, 692 opts BorrowConnectionOptions, 693 ) (BorrowConnectionsResult, error) { 694 var result BorrowConnectionsResult 695 s.state.RLock() 696 topoMap, err := s.topologyMapWithStateRLock() 697 s.state.RUnlock() 698 if err != nil { 699 return result, err 700 } 701 702 var ( 703 multiErr = xerrors.NewMultiError() 704 breakLoop bool 705 ) 706 err = topoMap.RouteShardForEach(shardID, func( 707 _ int, 708 shard shard.Shard, 709 host topology.Host, 710 ) { 711 if multiErr.NumErrors() > 0 || breakLoop { 712 // Error or has broken 713 return 714 } 715 if opts.ExcludeOrigin && s.origin != nil && s.origin.ID() == host.ID() { 716 // Skip origin host. 717 return 718 } 719 720 var ( 721 userResult WithBorrowConnectionResult 722 userErr error 723 ) 724 borrowErr := s.BorrowConnection(host.ID(), func( 725 client rpc.TChanNode, 726 channel Channel, 727 ) { 728 userResult, userErr = fn(shard, host, client, channel) 729 }) 730 if borrowErr != nil { 731 // Wasn't able to even borrow, skip if don't want to error 732 // on down hosts or return the borrow error. 733 if !opts.ContinueOnBorrowError { 734 multiErr = multiErr.Add(borrowErr) 735 } 736 return 737 } 738 739 // Track successful borrow. 740 result.Borrowed++ 741 742 // Track whether has broken loop. 743 breakLoop = userResult.Break 744 745 // Return whether user error occurred to break or not. 746 if userErr != nil { 747 multiErr = multiErr.Add(userErr) 748 } 749 }) 750 if err != nil { 751 // Route error. 752 return result, err 753 } 754 // Potentially a user error or borrow error, otherwise 755 // FinalError() will return nil. 756 return result, multiErr.FinalError() 757 } 758 759 func (s *session) BorrowConnection(hostID string, fn WithConnectionFn) error { 760 s.state.RLock() 761 unlocked := false 762 queue, ok := s.state.queuesByHostID[hostID] 763 if !ok { 764 s.state.RUnlock() 765 return errSessionHasNoHostQueueForHost 766 } 767 err := queue.BorrowConnection(func(client rpc.TChanNode, ch Channel) { 768 // Unlock early on success 769 s.state.RUnlock() 770 unlocked = true 771 772 // Execute function with borrowed connection 773 fn(client, ch) 774 }) 775 if !unlocked { 776 s.state.RUnlock() 777 } 778 return err 779 } 780 781 func (s *session) DedicatedConnection( 782 shardID uint32, 783 opts DedicatedConnectionOptions, 784 ) (rpc.TChanNode, Channel, error) { 785 s.state.RLock() 786 topoMap, err := s.topologyMapWithStateRLock() 787 s.state.RUnlock() 788 if err != nil { 789 return nil, nil, err 790 } 791 792 var ( 793 client rpc.TChanNode 794 channel Channel 795 succeeded bool 796 multiErr = xerrors.NewMultiError() 797 ) 798 err = topoMap.RouteShardForEach(shardID, func( 799 _ int, 800 targetShard shard.Shard, 801 host topology.Host, 802 ) { 803 stateFilter := opts.ShardStateFilter 804 if succeeded || !(stateFilter == shard.Unknown || targetShard.State() == stateFilter) { 805 return 806 } 807 808 if s.origin != nil && s.origin.ID() == host.ID() { 809 // Skip origin host. 810 return 811 } 812 813 newConnFn := s.opts.NewConnectionFn() 814 channel, client, err = newConnFn(channelName, host.Address(), s.opts) 815 if err != nil { 816 multiErr = multiErr.Add(err) 817 return 818 } 819 820 if err := s.healthCheckNewConnFn(client, s.opts, opts.BootstrappedNodesOnly); err != nil { 821 channel.Close() 822 multiErr = multiErr.Add(err) 823 return 824 } 825 826 succeeded = true 827 }) 828 if err != nil { 829 return nil, nil, err 830 } 831 832 if !succeeded { 833 multiErr = multiErr.Add( 834 fmt.Errorf("failed to create a dedicated connection for shard %d", shardID)) 835 return nil, nil, multiErr.FinalError() 836 } 837 838 return client, channel, nil 839 } 840 841 func (s *session) hostQueues( 842 topoMap topology.Map, 843 existing []hostQueue, 844 ) ([]hostQueue, int, int, error) { 845 // NB(r): we leave existing writes in the host queues to finish 846 // as they are already enroute to their destination. This is an edge case 847 // that might result in leaving nodes counting towards quorum, but fixing it 848 // would result in additional chatter. 849 850 start := s.nowFn() 851 852 existingByHostID := make(map[string]hostQueue, len(existing)) 853 for _, queue := range existing { 854 existingByHostID[queue.Host().ID()] = queue 855 } 856 857 hosts := topoMap.Hosts() 858 queues := make([]hostQueue, 0, len(hosts)) 859 newQueues := make([]hostQueue, 0, len(hosts)) 860 for _, host := range hosts { 861 if existingQueue, ok := existingByHostID[host.ID()]; ok { 862 queues = append(queues, existingQueue) 863 continue 864 } 865 newQueue, err := s.newHostQueue(host, topoMap) 866 if err != nil { 867 return nil, 0, 0, err 868 } 869 queues = append(queues, newQueue) 870 newQueues = append(newQueues, newQueue) 871 } 872 873 replicas := topoMap.Replicas() 874 majority := topoMap.MajorityReplicas() 875 876 firstConnectConsistencyLevel := s.opts.ClusterConnectConsistencyLevel() 877 if firstConnectConsistencyLevel == topology.ConnectConsistencyLevelNone { 878 // Return immediately if no connect consistency required 879 return queues, replicas, majority, nil 880 } 881 882 connectConsistencyLevel := firstConnectConsistencyLevel 883 if connectConsistencyLevel == topology.ConnectConsistencyLevelAny { 884 // If level any specified, first attempt all then proceed lowering requirement 885 connectConsistencyLevel = topology.ConnectConsistencyLevelAll 886 } 887 888 // Abort if we do not connect 889 connected := false 890 defer func() { 891 if !connected { 892 for _, queue := range newQueues { 893 queue.Close() 894 } 895 } 896 }() 897 898 for { 899 if now := s.nowFn(); now.Sub(start) >= s.opts.ClusterConnectTimeout() { 900 switch firstConnectConsistencyLevel { 901 case topology.ConnectConsistencyLevelAny: 902 // If connecting with connect any strategy then keep 903 // trying but lower consistency requirement 904 start = now 905 connectConsistencyLevel-- 906 if connectConsistencyLevel == topology.ConnectConsistencyLevelNone { 907 // Already tried to resolve all consistency requirements, just 908 // return successfully at this point 909 err := fmt.Errorf("timed out connecting, returning success") 910 s.log.Warn("cluster connect with consistency any", zap.Error(err)) 911 connected = true 912 return queues, replicas, majority, nil 913 } 914 default: 915 // Timed out connecting to a specific consistency requirement 916 return nil, 0, 0, ErrClusterConnectTimeout 917 } 918 } 919 920 var level topology.ConsistencyLevel 921 switch connectConsistencyLevel { 922 case topology.ConnectConsistencyLevelAll: 923 level = topology.ConsistencyLevelAll 924 case topology.ConnectConsistencyLevelMajority: 925 level = topology.ConsistencyLevelMajority 926 case topology.ConnectConsistencyLevelOne: 927 level = topology.ConsistencyLevelOne 928 default: 929 return nil, 0, 0, errSessionInvalidConnectClusterConnectConsistencyLevel 930 } 931 clusterAvailable, err := s.clusterAvailabilityWithQueuesAndMap(level, 932 queues, topoMap) 933 if err != nil { 934 return nil, 0, 0, err 935 } 936 if clusterAvailable { 937 // All done 938 break 939 } 940 time.Sleep(clusterConnectWaitInterval) 941 } 942 943 connected = true 944 return queues, replicas, majority, nil 945 } 946 947 func (s *session) WriteClusterAvailability() (bool, error) { 948 level := s.opts.WriteConsistencyLevel() 949 return s.clusterAvailability(level) 950 } 951 952 func (s *session) ReadClusterAvailability() (bool, error) { 953 var convertedConsistencyLevel topology.ConsistencyLevel 954 level := s.opts.ReadConsistencyLevel() 955 switch level { 956 case topology.ReadConsistencyLevelNone: 957 // Already ready. 958 return true, nil 959 case topology.ReadConsistencyLevelOne: 960 convertedConsistencyLevel = topology.ConsistencyLevelOne 961 case topology.ReadConsistencyLevelUnstrictMajority: 962 convertedConsistencyLevel = topology.ConsistencyLevelOne 963 case topology.ReadConsistencyLevelMajority: 964 convertedConsistencyLevel = topology.ConsistencyLevelMajority 965 case topology.ReadConsistencyLevelUnstrictAll: 966 convertedConsistencyLevel = topology.ConsistencyLevelOne 967 case topology.ReadConsistencyLevelAll: 968 convertedConsistencyLevel = topology.ConsistencyLevelAll 969 default: 970 return false, fmt.Errorf("unknown consistency level: %d", level) 971 } 972 return s.clusterAvailability(convertedConsistencyLevel) 973 } 974 975 func (s *session) clusterAvailability( 976 level topology.ConsistencyLevel, 977 ) (bool, error) { 978 s.state.RLock() 979 queues := s.state.queues 980 topoMap, err := s.topologyMapWithStateRLock() 981 s.state.RUnlock() 982 if err != nil { 983 return false, err 984 } 985 return s.clusterAvailabilityWithQueuesAndMap(level, queues, topoMap) 986 } 987 988 func (s *session) clusterAvailabilityWithQueuesAndMap( 989 level topology.ConsistencyLevel, 990 queues []hostQueue, 991 topoMap topology.Map, 992 ) (bool, error) { 993 shards := topoMap.ShardSet().AllIDs() 994 minConnectionCount := s.opts.MinConnectionCount() 995 replicas := topoMap.Replicas() 996 majority := topoMap.MajorityReplicas() 997 998 for _, shardID := range shards { 999 shardReplicasAvailable := 0 1000 routeErr := topoMap.RouteShardForEach(shardID, func(idx int, _ shard.Shard, _ topology.Host) { 1001 if queues[idx].ConnectionCount() >= minConnectionCount { 1002 shardReplicasAvailable++ 1003 } 1004 }) 1005 if routeErr != nil { 1006 return false, routeErr 1007 } 1008 var clusterAvailableForShard bool 1009 switch level { 1010 case topology.ConsistencyLevelAll: 1011 clusterAvailableForShard = shardReplicasAvailable == replicas 1012 case topology.ConsistencyLevelMajority: 1013 clusterAvailableForShard = shardReplicasAvailable >= majority 1014 case topology.ConsistencyLevelOne: 1015 clusterAvailableForShard = shardReplicasAvailable > 0 1016 default: 1017 return false, fmt.Errorf("unknown consistency level: %d", level) 1018 } 1019 if !clusterAvailableForShard { 1020 return false, nil 1021 } 1022 } 1023 1024 return true, nil 1025 } 1026 1027 func (s *session) setTopologyWithLock(topoMap topology.Map, queues []hostQueue, replicas, majority int) { 1028 prevQueues := s.state.queues 1029 1030 newQueuesByHostID := make(map[string]hostQueue, len(queues)) 1031 for _, queue := range queues { 1032 newQueuesByHostID[queue.Host().ID()] = queue 1033 } 1034 1035 s.state.queues = queues 1036 s.state.queuesByHostID = newQueuesByHostID 1037 1038 s.state.topoMap = topoMap 1039 1040 s.state.replicas = replicas 1041 s.state.majority = majority 1042 1043 // If the number of hostQueues has changed then we need to recreate the fetch 1044 // batch op array pool as it must be the exact length of the queues as we index 1045 // directly into the return array in fetch calls. 1046 if len(queues) != len(prevQueues) { 1047 poolOpts := pool.NewObjectPoolOptions(). 1048 SetSize(int(s.opts.FetchBatchOpPoolSize())). 1049 SetDynamic(s.opts.FetchBatchOpPoolSize().IsDynamic()). 1050 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1051 s.scope.SubScope("fetch-batch-op-array-array-pool"), 1052 )) 1053 s.pools.fetchBatchOpArrayArray = newFetchBatchOpArrayArrayPool( 1054 poolOpts, 1055 len(queues), 1056 int(s.opts.FetchBatchOpPoolSize())/len(queues)) 1057 s.pools.fetchBatchOpArrayArray.Init() 1058 } 1059 1060 if s.pools.multiReaderIteratorArray == nil { 1061 s.pools.multiReaderIteratorArray = encoding.NewMultiReaderIteratorArrayPool([]pool.Bucket{ 1062 { 1063 Capacity: replicas, 1064 Count: s.opts.SeriesIteratorPoolSize(), 1065 }, 1066 }) 1067 s.pools.multiReaderIteratorArray.Init() 1068 } 1069 if s.pools.readerSliceOfSlicesIterator == nil { 1070 size := int(s.opts.SeriesIteratorPoolSize()) 1071 if !s.opts.SeriesIteratorPoolSize().IsDynamic() { 1072 size = replicas * int(s.opts.SeriesIteratorPoolSize()) 1073 } 1074 poolOpts := pool.NewObjectPoolOptions(). 1075 SetSize(size). 1076 SetDynamic(s.opts.SeriesIteratorPoolSize().IsDynamic()). 1077 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1078 s.scope.SubScope("reader-slice-of-slices-iterator-pool"), 1079 )) 1080 s.pools.readerSliceOfSlicesIterator = newReaderSliceOfSlicesIteratorPool(poolOpts) 1081 s.pools.readerSliceOfSlicesIterator.Init() 1082 } 1083 if s.pools.multiReaderIterator == nil { 1084 size := int(s.opts.SeriesIteratorPoolSize()) 1085 if !s.opts.SeriesIteratorPoolSize().IsDynamic() { 1086 size = replicas * int(s.opts.SeriesIteratorPoolSize()) 1087 } 1088 poolOpts := pool.NewObjectPoolOptions(). 1089 SetSize(size). 1090 SetDynamic(s.opts.SeriesIteratorPoolSize().IsDynamic()). 1091 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1092 s.scope.SubScope("multi-reader-iterator-pool"), 1093 )) 1094 s.pools.multiReaderIterator = encoding.NewMultiReaderIteratorPool(poolOpts) 1095 s.pools.multiReaderIterator.Init(s.opts.ReaderIteratorAllocate()) 1096 } 1097 if replicas > len(s.metrics.writeNodesRespondingErrors) { 1098 curr := len(s.metrics.writeNodesRespondingErrors) 1099 for i := curr; i < replicas; i++ { 1100 tags := map[string]string{"nodes": fmt.Sprintf("%d", i+1)} 1101 name := "write.nodes-responding-error" 1102 serverErrsSubScope := s.scope.Tagged(tags).Tagged(map[string]string{ 1103 "error_type": "server_error", 1104 }) 1105 badRequestErrsSubScope := s.scope.Tagged(tags).Tagged(map[string]string{ 1106 "error_type": "bad_request_error", 1107 }) 1108 sc := serverErrsSubScope.Counter(name) 1109 s.metrics.writeNodesRespondingErrors = append(s.metrics.writeNodesRespondingErrors, sc) 1110 cc := badRequestErrsSubScope.Counter(name) 1111 s.metrics.writeNodesRespondingBadRequestErrors = append(s.metrics.writeNodesRespondingBadRequestErrors, cc) 1112 } 1113 } 1114 if replicas > len(s.metrics.fetchNodesRespondingErrors) { 1115 curr := len(s.metrics.fetchNodesRespondingErrors) 1116 for i := curr; i < replicas; i++ { 1117 tags := map[string]string{"nodes": fmt.Sprintf("%d", i+1)} 1118 name := "fetch.nodes-responding-error" 1119 serverErrsSubScope := s.scope.Tagged(tags).Tagged(map[string]string{ 1120 "error_type": "server_error", 1121 }) 1122 badRequestErrsSubScope := s.scope.Tagged(tags).Tagged(map[string]string{ 1123 "error_type": "bad_request_error", 1124 }) 1125 sc := serverErrsSubScope.Counter(name) 1126 s.metrics.fetchNodesRespondingErrors = append(s.metrics.fetchNodesRespondingErrors, sc) 1127 cc := badRequestErrsSubScope.Counter(name) 1128 s.metrics.fetchNodesRespondingBadRequestErrors = append(s.metrics.fetchNodesRespondingBadRequestErrors, cc) 1129 } 1130 } 1131 1132 // Asynchronously close the set of host queues no longer in use 1133 go func() { 1134 for _, queue := range prevQueues { 1135 newQueue, ok := newQueuesByHostID[queue.Host().ID()] 1136 if !ok || newQueue != queue { 1137 queue.Close() 1138 } 1139 } 1140 }() 1141 1142 s.log.Info("successfully updated topology", 1143 zap.Int("numHosts", topoMap.HostsLen()), 1144 zap.Int("numShards", len(topoMap.ShardSet().AllIDs()))) 1145 } 1146 1147 func (s *session) newHostQueue(host topology.Host, topoMap topology.Map) (hostQueue, error) { 1148 // NB(r): Due to hosts being replicas we have: 1149 // = replica * numWrites 1150 // = total writes to all hosts 1151 // We need to pool: 1152 // = replica * (numWrites / writeBatchSize) 1153 // = number of batch request structs to pool 1154 // For purposes of simplifying the options for pooling the write op pool size 1155 // represents the number of ops to pool not including replication, this is due 1156 // to the fact that the ops are shared between the different host queue replicas. 1157 writeOpPoolSize := s.opts.WriteOpPoolSize() 1158 if s.opts.WriteTaggedOpPoolSize() > writeOpPoolSize { 1159 writeOpPoolSize = s.opts.WriteTaggedOpPoolSize() 1160 } 1161 totalBatches := topoMap.Replicas() * 1162 int(math.Ceil(float64(writeOpPoolSize)/float64(s.opts.WriteBatchSize()))) 1163 hostBatches := int(math.Ceil(float64(totalBatches) / float64(topoMap.HostsLen()))) 1164 1165 writeBatchRequestPoolOpts := pool.NewObjectPoolOptions(). 1166 SetSize(hostBatches). 1167 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1168 s.scope.SubScope("write-batch-request-pool"), 1169 )) 1170 writeBatchRequestPool := newWriteBatchRawRequestPool(writeBatchRequestPoolOpts) 1171 writeBatchRequestPool.Init() 1172 writeBatchV2RequestPool := newWriteBatchRawV2RequestPool(writeBatchRequestPoolOpts) 1173 writeBatchV2RequestPool.Init() 1174 1175 writeTaggedBatchRequestPoolOpts := pool.NewObjectPoolOptions(). 1176 SetSize(hostBatches). 1177 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1178 s.scope.SubScope("write-tagged-batch-request-pool"), 1179 )) 1180 writeTaggedBatchRequestPool := newWriteTaggedBatchRawRequestPool(writeTaggedBatchRequestPoolOpts) 1181 writeTaggedBatchRequestPool.Init() 1182 writeTaggedBatchV2RequestPool := newWriteTaggedBatchRawV2RequestPool(writeBatchRequestPoolOpts) 1183 writeTaggedBatchV2RequestPool.Init() 1184 1185 writeBatchRawRequestElementArrayPoolOpts := pool.NewObjectPoolOptions(). 1186 SetSize(hostBatches). 1187 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1188 s.scope.SubScope("id-datapoint-array-pool"), 1189 )) 1190 writeBatchRawRequestElementArrayPool := newWriteBatchRawRequestElementArrayPool( 1191 writeBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) 1192 writeBatchRawRequestElementArrayPool.Init() 1193 writeBatchRawV2RequestElementArrayPool := newWriteBatchRawV2RequestElementArrayPool( 1194 writeBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) 1195 writeBatchRawV2RequestElementArrayPool.Init() 1196 1197 writeTaggedBatchRawRequestElementArrayPoolOpts := pool.NewObjectPoolOptions(). 1198 SetSize(hostBatches). 1199 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1200 s.scope.SubScope("id-tagged-datapoint-array-pool"), 1201 )) 1202 writeTaggedBatchRawRequestElementArrayPool := newWriteTaggedBatchRawRequestElementArrayPool( 1203 writeTaggedBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) 1204 writeTaggedBatchRawRequestElementArrayPool.Init() 1205 writeTaggedBatchRawV2RequestElementArrayPool := newWriteTaggedBatchRawV2RequestElementArrayPool( 1206 writeTaggedBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) 1207 writeTaggedBatchRawV2RequestElementArrayPool.Init() 1208 1209 fetchBatchRawV2RequestPoolOpts := pool.NewObjectPoolOptions(). 1210 SetSize(hostBatches). 1211 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1212 s.scope.SubScope("fetch-batch-request-pool"), 1213 )) 1214 fetchBatchRawV2RequestPool := newFetchBatchRawV2RequestPool(fetchBatchRawV2RequestPoolOpts) 1215 fetchBatchRawV2RequestPool.Init() 1216 1217 fetchBatchRawV2RequestElementArrayPoolOpts := pool.NewObjectPoolOptions(). 1218 SetSize(hostBatches). 1219 SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( 1220 s.scope.SubScope("fetch-batch-request-array-pool"), 1221 )) 1222 fetchBatchRawV2RequestElementArrPool := newFetchBatchRawV2RequestElementArrayPool( 1223 fetchBatchRawV2RequestElementArrayPoolOpts, s.opts.FetchBatchSize(), 1224 ) 1225 fetchBatchRawV2RequestElementArrPool.Init() 1226 1227 hostQueue, err := s.newHostQueueFn(host, hostQueueOpts{ 1228 writeBatchRawRequestPool: writeBatchRequestPool, 1229 writeBatchRawV2RequestPool: writeBatchV2RequestPool, 1230 writeBatchRawRequestElementArrayPool: writeBatchRawRequestElementArrayPool, 1231 writeBatchRawV2RequestElementArrayPool: writeBatchRawV2RequestElementArrayPool, 1232 writeTaggedBatchRawRequestPool: writeTaggedBatchRequestPool, 1233 writeTaggedBatchRawV2RequestPool: writeTaggedBatchV2RequestPool, 1234 writeTaggedBatchRawRequestElementArrayPool: writeTaggedBatchRawRequestElementArrayPool, 1235 writeTaggedBatchRawV2RequestElementArrayPool: writeTaggedBatchRawV2RequestElementArrayPool, 1236 fetchBatchRawV2RequestPool: fetchBatchRawV2RequestPool, 1237 fetchBatchRawV2RequestElementArrayPool: fetchBatchRawV2RequestElementArrPool, 1238 opts: s.opts, 1239 }) 1240 if err != nil { 1241 return nil, err 1242 } 1243 hostQueue.Open() 1244 return hostQueue, nil 1245 } 1246 1247 func (s *session) Write( 1248 nsID, id ident.ID, 1249 t xtime.UnixNano, 1250 value float64, 1251 unit xtime.Unit, 1252 annotation []byte, 1253 ) error { 1254 w := s.pools.writeAttempt.Get() 1255 w.args.attemptType = untaggedWriteAttemptType 1256 w.args.namespace, w.args.id = nsID, id 1257 w.args.tags = ident.EmptyTagIterator 1258 w.args.t = t 1259 w.args.value, w.args.unit, w.args.annotation = value, unit, annotation 1260 err := s.writeRetrier.Attempt(w.attemptFn) 1261 s.pools.writeAttempt.Put(w) 1262 return err 1263 } 1264 1265 func (s *session) WriteTagged( 1266 nsID, id ident.ID, 1267 tags ident.TagIterator, 1268 t xtime.UnixNano, 1269 value float64, 1270 unit xtime.Unit, 1271 annotation []byte, 1272 ) error { 1273 w := s.pools.writeAttempt.Get() 1274 w.args.attemptType = taggedWriteAttemptType 1275 w.args.namespace, w.args.id, w.args.tags = nsID, id, tags 1276 w.args.t = t 1277 w.args.value, w.args.unit, w.args.annotation = value, unit, annotation 1278 err := s.writeRetrier.Attempt(w.attemptFn) 1279 s.pools.writeAttempt.Put(w) 1280 return err 1281 } 1282 1283 func (s *session) writeAttempt( 1284 wType writeAttemptType, 1285 nsID, id ident.ID, 1286 inputTags ident.TagIterator, 1287 t xtime.UnixNano, 1288 value float64, 1289 unit xtime.Unit, 1290 annotation []byte, 1291 ) error { 1292 startWriteAttempt := s.nowFn() 1293 1294 timeType, timeTypeErr := convert.ToTimeType(unit) 1295 if timeTypeErr != nil { 1296 return timeTypeErr 1297 } 1298 1299 timestamp, timestampErr := convert.ToValue(t, timeType) 1300 if timestampErr != nil { 1301 return timestampErr 1302 } 1303 1304 s.state.RLock() 1305 if s.state.status != statusOpen { 1306 s.state.RUnlock() 1307 return ErrSessionStatusNotOpen 1308 } 1309 1310 state, majority, enqueued, err := s.writeAttemptWithRLock( 1311 wType, nsID, id, inputTags, timestamp, value, timeType, annotation) 1312 s.state.RUnlock() 1313 1314 if err != nil { 1315 return err 1316 } 1317 1318 // it's safe to Wait() here, as we still hold the lock on state, after it's 1319 // returned from writeAttemptWithRLock. 1320 state.Wait() 1321 1322 err = s.writeConsistencyResult(state.consistencyLevel, majority, enqueued, 1323 enqueued-state.pending, int32(len(state.errors)), state.errors) 1324 1325 s.recordWriteMetrics(err, state, startWriteAttempt) 1326 // must Unlock before decRef'ing, as the latter releases the writeState back into a 1327 // pool if ref count == 0. 1328 state.Unlock() 1329 state.decRef() 1330 1331 return err 1332 } 1333 1334 // NB(prateek): the returned writeState, if valid, still holds the lock. Its ownership 1335 // is transferred to the calling function, and is expected to manage the lifecycle of 1336 // of the object (including releasing the lock/decRef'ing it). 1337 func (s *session) writeAttemptWithRLock( 1338 wType writeAttemptType, 1339 namespace, id ident.ID, 1340 inputTags ident.TagIterator, 1341 timestamp int64, 1342 value float64, 1343 timeType rpc.TimeType, 1344 annotation []byte, 1345 ) (*writeState, int32, int32, error) { 1346 var ( 1347 majority = int32(s.state.majority) 1348 enqueued int32 1349 ) 1350 1351 // NB(prateek): We retain an individual copy of the namespace, ID per 1352 // writeState, as each writeState tracks the lifecycle of it's resources in 1353 // use in the various queues. Tracking per writeAttempt isn't sufficient as 1354 // we may enqueue multiple writeStates concurrently depending on retries 1355 // and consistency level checks. 1356 var tagEncoder serialize.TagEncoder 1357 if wType == taggedWriteAttemptType { 1358 tagEncoder = s.pools.tagEncoder.Get() 1359 if err := tagEncoder.Encode(inputTags); err != nil { 1360 tagEncoder.Finalize() 1361 return nil, 0, 0, err 1362 } 1363 } 1364 nsID := s.cloneFinalizable(namespace) 1365 tsID := s.cloneFinalizable(id) 1366 1367 var ( 1368 clonedAnnotation checked.Bytes 1369 clonedAnnotationBytes []byte 1370 ) 1371 if len(annotation) > 0 { 1372 clonedAnnotation = s.pools.checkedBytes.Get(len(annotation)) 1373 clonedAnnotation.IncRef() 1374 clonedAnnotation.AppendAll(annotation) 1375 clonedAnnotationBytes = clonedAnnotation.Bytes() 1376 } 1377 1378 var op writeOp 1379 switch wType { 1380 case untaggedWriteAttemptType: 1381 wop := s.pools.writeOperation.Get() 1382 wop.namespace = nsID 1383 wop.shardID = s.state.topoMap.ShardSet().Lookup(tsID) 1384 wop.request.ID = tsID.Bytes() 1385 wop.request.Datapoint.Value = value 1386 wop.request.Datapoint.Timestamp = timestamp 1387 wop.request.Datapoint.TimestampTimeType = timeType 1388 wop.request.Datapoint.Annotation = clonedAnnotationBytes 1389 wop.requestV2.ID = wop.request.ID 1390 wop.requestV2.Datapoint = wop.request.Datapoint 1391 op = wop 1392 case taggedWriteAttemptType: 1393 wop := s.pools.writeTaggedOperation.Get() 1394 wop.namespace = nsID 1395 wop.shardID = s.state.topoMap.ShardSet().Lookup(tsID) 1396 wop.request.ID = tsID.Bytes() 1397 encodedTagBytes, ok := tagEncoder.Data() 1398 if !ok { 1399 return nil, 0, 0, errUnableToEncodeTags 1400 } 1401 wop.request.EncodedTags = encodedTagBytes.Bytes() 1402 wop.request.Datapoint.Value = value 1403 wop.request.Datapoint.Timestamp = timestamp 1404 wop.request.Datapoint.TimestampTimeType = timeType 1405 wop.request.Datapoint.Annotation = clonedAnnotationBytes 1406 wop.requestV2.ID = wop.request.ID 1407 wop.requestV2.EncodedTags = wop.request.EncodedTags 1408 wop.requestV2.Datapoint = wop.request.Datapoint 1409 op = wop 1410 default: 1411 // should never happen 1412 return nil, 0, 0, errUnknownWriteAttemptType 1413 } 1414 1415 state := s.pools.writeState.Get() 1416 state.consistencyLevel = s.state.writeLevel 1417 state.shardsLeavingCountTowardsConsistency = s.shardsLeavingCountTowardsConsistency 1418 state.shardsLeavingAndInitializingCountTowardsConsistency = s.shardsLeavingAndInitializingCountTowardsConsistency 1419 state.leavingAndInitializingPairCounted = false 1420 state.topoMap = s.state.topoMap 1421 state.lastResetTime = time.Now() 1422 state.incRef() 1423 // todo@bl: Can we combine the writeOpPool and the writeStatePool? 1424 state.op, state.majority = op, majority 1425 state.nsID, state.tsID, state.tagEncoder, state.annotation = nsID, tsID, tagEncoder, clonedAnnotation 1426 op.SetCompletionFn(state.completionFn) 1427 1428 if err := s.state.topoMap.RouteForEach(tsID, func( 1429 idx int, 1430 hostShard shard.Shard, 1431 host topology.Host, 1432 ) { 1433 if !s.writeShardsInitializing && hostShard.State() == shard.Initializing { 1434 // NB(r): Do not write to this node as the shard is initializing 1435 // and writing to intialized shards is not enabled (also 1436 // depending on your config initializing shards won't count 1437 // towards quorum, current defaults, so this is ok consistency wise). 1438 return 1439 } 1440 1441 // Count pending write requests before we enqueue the completion fns, 1442 // which rely on the count when executing 1443 state.pending++ 1444 state.queues = append(state.queues, s.state.queues[idx]) 1445 }); err != nil { 1446 state.decRef() 1447 return nil, 0, 0, err 1448 } 1449 1450 state.Lock() 1451 for i := range state.queues { 1452 state.incRef() 1453 if err := state.queues[i].Enqueue(state.op); err != nil { 1454 state.Unlock() 1455 state.decRef() 1456 1457 // NB(r): if this happens we have a bug, once we are in the read 1458 // lock the current queues should never be closed 1459 s.log.Error("[invariant violated] failed to enqueue write", zap.Error(err)) 1460 return nil, 0, 0, err 1461 } 1462 enqueued++ 1463 } 1464 1465 // NB(prateek): the current go-routine still holds a lock on the 1466 // returned writeState object. 1467 return state, majority, enqueued, nil 1468 } 1469 1470 func (s *session) Fetch( 1471 nsID ident.ID, 1472 id ident.ID, 1473 startInclusive, endExclusive xtime.UnixNano, 1474 ) (encoding.SeriesIterator, error) { 1475 tsIDs := ident.NewIDsIterator(id) 1476 results, err := s.FetchIDs(nsID, tsIDs, startInclusive, endExclusive) 1477 if err != nil { 1478 return nil, err 1479 } 1480 mutableResults := results.(encoding.MutableSeriesIterators) 1481 iters := mutableResults.Iters() 1482 iter := iters[0] 1483 // Reset to zero so that when we close this results set the iter doesn't get closed 1484 mutableResults.Reset(0) 1485 mutableResults.Close() 1486 return iter, nil 1487 } 1488 1489 func (s *session) FetchIDs( 1490 nsID ident.ID, 1491 ids ident.Iterator, 1492 startInclusive, endExclusive xtime.UnixNano, 1493 ) (encoding.SeriesIterators, error) { 1494 f := s.pools.fetchAttempt.Get() 1495 f.args.namespace, f.args.ids = nsID, ids 1496 f.args.start = startInclusive 1497 f.args.end = endExclusive 1498 err := s.fetchRetrier.Attempt(f.attemptFn) 1499 result := f.result 1500 s.pools.fetchAttempt.Put(f) 1501 return result, err 1502 } 1503 1504 func (s *session) Aggregate( 1505 ctx gocontext.Context, 1506 ns ident.ID, 1507 q index.Query, 1508 opts index.AggregationOptions, 1509 ) (AggregatedTagsIterator, FetchResponseMetadata, error) { 1510 f := s.pools.aggregateAttempt.Get() 1511 f.args.ctx = ctx 1512 f.args.ns = ns 1513 f.args.query = q 1514 f.args.opts = opts 1515 err := s.fetchRetrier.Attempt(f.attemptFn) 1516 iter, metadata := f.resultIter, f.resultMetadata 1517 s.pools.aggregateAttempt.Put(f) 1518 return iter, metadata, err 1519 } 1520 1521 func (s *session) aggregateAttempt( 1522 ctx gocontext.Context, 1523 ns ident.ID, 1524 q index.Query, 1525 opts index.AggregationOptions, 1526 ) (AggregatedTagsIterator, FetchResponseMetadata, error) { 1527 s.state.RLock() 1528 if s.state.status != statusOpen { 1529 s.state.RUnlock() 1530 return nil, FetchResponseMetadata{}, ErrSessionStatusNotOpen 1531 } 1532 1533 // NB(prateek): we have to clone the namespace, as we cannot guarantee the lifecycle 1534 // of the hostQueues responding is less than the lifecycle of the current method. 1535 nsClone := s.pools.id.Clone(ns) 1536 1537 req, err := convert.ToRPCAggregateQueryRawRequest(nsClone, q, opts) 1538 if err != nil { 1539 s.state.RUnlock() 1540 nsClone.Finalize() 1541 return nil, FetchResponseMetadata{}, xerrors.NewNonRetryableError(err) 1542 } 1543 if req.SeriesLimit != nil && opts.InstanceMultiple > 0 { 1544 topo := s.state.topoMap 1545 iPerReplica := int64(len(topo.Hosts()) / topo.Replicas()) 1546 iSeriesLimit := int64(float32(opts.SeriesLimit)*opts.InstanceMultiple) / iPerReplica 1547 if iSeriesLimit < *req.SeriesLimit { 1548 req.SeriesLimit = &iSeriesLimit 1549 } 1550 } 1551 1552 fetchState, err := s.newFetchStateWithRLock(ctx, nsClone, newFetchStateOpts{ 1553 stateType: aggregateFetchState, 1554 aggregateRequest: req, 1555 startInclusive: opts.StartInclusive, 1556 endExclusive: opts.EndExclusive, 1557 readConsistencyLevel: opts.ReadConsistencyLevel, 1558 }) 1559 s.state.RUnlock() 1560 1561 if err != nil { 1562 return nil, FetchResponseMetadata{}, err 1563 } 1564 1565 // it's safe to Wait() here, as we still hold the lock on fetchState, after it's 1566 // returned from newFetchStateWithRLock. 1567 fetchState.Wait() 1568 1569 // must Unlock before calling `asEncodingSeriesIterators` as the latter needs to acquire 1570 // the fetchState Lock 1571 fetchState.Unlock() 1572 iters, meta, err := fetchState.asAggregatedTagsIterator(s.pools, opts.SeriesLimit) 1573 1574 // must Unlock() before decRef'ing, as the latter releases the fetchState back into a 1575 // pool if ref count == 0. 1576 fetchState.decRef() 1577 1578 return iters, meta, err 1579 } 1580 1581 func (s *session) FetchTagged( 1582 ctx gocontext.Context, 1583 ns ident.ID, 1584 q index.Query, 1585 opts index.QueryOptions, 1586 ) (encoding.SeriesIterators, FetchResponseMetadata, error) { 1587 f := s.pools.fetchTaggedAttempt.Get() 1588 f.args.ctx = ctx 1589 f.args.ns = ns 1590 f.args.query = q 1591 f.args.opts = opts 1592 err := s.fetchRetrier.Attempt(f.dataAttemptFn) 1593 iters, metadata := f.dataResultIters, f.dataResultMetadata 1594 s.pools.fetchTaggedAttempt.Put(f) 1595 return iters, metadata, err 1596 } 1597 1598 func (s *session) FetchTaggedIDs( 1599 ctx gocontext.Context, 1600 ns ident.ID, 1601 q index.Query, 1602 opts index.QueryOptions, 1603 ) (TaggedIDsIterator, FetchResponseMetadata, error) { 1604 f := s.pools.fetchTaggedAttempt.Get() 1605 f.args.ctx = ctx 1606 f.args.ns = ns 1607 f.args.query = q 1608 f.args.opts = opts 1609 err := s.fetchRetrier.Attempt(f.idsAttemptFn) 1610 iter, metadata := f.idsResultIter, f.idsResultMetadata 1611 s.pools.fetchTaggedAttempt.Put(f) 1612 return iter, metadata, err 1613 } 1614 1615 func (s *session) fetchTaggedAttempt( 1616 ctx gocontext.Context, 1617 ns ident.ID, 1618 q index.Query, 1619 opts index.QueryOptions, 1620 ) (encoding.SeriesIterators, FetchResponseMetadata, error) { 1621 nsCtx, err := s.nsCtxFor(ns) 1622 if err != nil { 1623 return nil, FetchResponseMetadata{}, err 1624 } 1625 s.state.RLock() 1626 if s.state.status != statusOpen { 1627 s.state.RUnlock() 1628 return nil, FetchResponseMetadata{}, ErrSessionStatusNotOpen 1629 } 1630 1631 // NB(prateek): we have to clone the namespace, as we cannot guarantee the lifecycle 1632 // of the hostQueues responding is less than the lifecycle of the current method. 1633 nsClone := s.pools.id.Clone(ns) 1634 1635 // FOLLOWUP(prateek): currently both `index.Query` and the returned request depend on 1636 // native, un-pooled types; so we do not Clone() either. We will start doing so 1637 // once https://github.com/m3db/m3ninx/issues/42 lands. Including transferring ownership 1638 // of the Clone()'d value to the `fetchState`. 1639 const fetchData = true 1640 req, err := convert.ToRPCFetchTaggedRequest(nsClone, q, opts, fetchData) 1641 if err != nil { 1642 s.state.RUnlock() 1643 nsClone.Finalize() 1644 return nil, FetchResponseMetadata{}, xerrors.NewNonRetryableError(err) 1645 } 1646 if req.SeriesLimit != nil && opts.InstanceMultiple > 0 { 1647 topo := s.state.topoMap 1648 iPerReplica := int64(len(topo.Hosts()) / topo.Replicas()) 1649 iSeriesLimit := int64(float32(opts.SeriesLimit)*opts.InstanceMultiple) / iPerReplica 1650 if iSeriesLimit < *req.SeriesLimit { 1651 req.SeriesLimit = &iSeriesLimit 1652 } 1653 } 1654 1655 fetchState, err := s.newFetchStateWithRLock(ctx, nsClone, newFetchStateOpts{ 1656 stateType: fetchTaggedFetchState, 1657 fetchTaggedRequest: req, 1658 startInclusive: opts.StartInclusive, 1659 endExclusive: opts.EndExclusive, 1660 readConsistencyLevel: opts.ReadConsistencyLevel, 1661 }) 1662 s.state.RUnlock() 1663 1664 if err != nil { 1665 return nil, FetchResponseMetadata{}, err 1666 } 1667 1668 // it's safe to Wait() here, as we still hold the lock on fetchState, after it's 1669 // returned from newFetchStateWithRLock. 1670 fetchState.Wait() 1671 1672 // must Unlock before calling `asEncodingSeriesIterators` as the latter needs to acquire 1673 // the fetchState Lock 1674 fetchState.Unlock() 1675 1676 iterOpts := s.opts.IterationOptions() 1677 if opts.IterateEqualTimestampStrategy != nil { 1678 iterOpts.IterateEqualTimestampStrategy = *opts.IterateEqualTimestampStrategy 1679 } 1680 1681 iters, metadata, err := fetchState.asEncodingSeriesIterators( 1682 s.pools, nsCtx.Schema, iterOpts, opts.SeriesLimit) 1683 1684 // must Unlock() before decRef'ing, as the latter releases the fetchState back into a 1685 // pool if ref count == 0. 1686 fetchState.decRef() 1687 1688 return iters, metadata, err 1689 } 1690 1691 func (s *session) fetchTaggedIDsAttempt( 1692 ctx gocontext.Context, 1693 ns ident.ID, 1694 q index.Query, 1695 opts index.QueryOptions, 1696 ) (TaggedIDsIterator, FetchResponseMetadata, error) { 1697 s.state.RLock() 1698 if s.state.status != statusOpen { 1699 s.state.RUnlock() 1700 return nil, FetchResponseMetadata{}, ErrSessionStatusNotOpen 1701 } 1702 1703 // NB(prateek): we have to clone the namespace, as we cannot guarantee the lifecycle 1704 // of the hostQueues responding is less than the lifecycle of the current method. 1705 nsClone := s.pools.id.Clone(ns) 1706 1707 // FOLLOWUP(prateek): currently both `index.Query` and the returned request depend on 1708 // native, un-pooled types; so we do not Clone() either. We will start doing so 1709 // once https://github.com/m3db/m3ninx/issues/42 lands. Including transferring ownership 1710 // of the Clone()'d value to the `fetchState`. 1711 const fetchData = false 1712 req, err := convert.ToRPCFetchTaggedRequest(nsClone, q, opts, fetchData) 1713 if err != nil { 1714 s.state.RUnlock() 1715 nsClone.Finalize() 1716 return nil, FetchResponseMetadata{}, xerrors.NewNonRetryableError(err) 1717 } 1718 if req.SeriesLimit != nil && opts.InstanceMultiple > 0 { 1719 topo := s.state.topoMap 1720 iPerReplica := int64(len(topo.Hosts()) / topo.Replicas()) 1721 iSeriesLimit := int64(float32(opts.SeriesLimit)*opts.InstanceMultiple) / iPerReplica 1722 if iSeriesLimit < *req.SeriesLimit { 1723 req.SeriesLimit = &iSeriesLimit 1724 } 1725 } 1726 1727 fetchState, err := s.newFetchStateWithRLock(ctx, nsClone, newFetchStateOpts{ 1728 stateType: fetchTaggedFetchState, 1729 fetchTaggedRequest: req, 1730 startInclusive: opts.StartInclusive, 1731 endExclusive: opts.EndExclusive, 1732 readConsistencyLevel: opts.ReadConsistencyLevel, 1733 }) 1734 s.state.RUnlock() 1735 1736 if err != nil { 1737 return nil, FetchResponseMetadata{}, err 1738 } 1739 1740 // it's safe to Wait() here, as we still hold the lock on fetchState, after it's 1741 // returned from newFetchStateWithRLock. 1742 fetchState.Wait() 1743 1744 // must Unlock before calling `asTaggedIDsIterator` as the latter needs to acquire 1745 // the fetchState Lock 1746 fetchState.Unlock() 1747 iter, metadata, err := fetchState.asTaggedIDsIterator(s.pools, opts.SeriesLimit) 1748 1749 // must Unlock() before decRef'ing, as the latter releases the fetchState back into a 1750 // pool if ref count == 0. 1751 fetchState.decRef() 1752 1753 return iter, metadata, err 1754 } 1755 1756 type newFetchStateOpts struct { 1757 stateType fetchStateType 1758 startInclusive xtime.UnixNano 1759 endExclusive xtime.UnixNano 1760 readConsistencyLevel *topology.ReadConsistencyLevel 1761 1762 // only valid if stateType == fetchTaggedFetchState 1763 fetchTaggedRequest rpc.FetchTaggedRequest 1764 1765 // only valid if stateType == aggregateFetchState 1766 aggregateRequest rpc.AggregateQueryRawRequest 1767 } 1768 1769 // NB(prateek): the returned fetchState, if valid, still holds the lock. Its ownership 1770 // is transferred to the calling function, and is expected to manage the lifecycle of 1771 // of the object (including releasing the lock/decRef'ing it). 1772 // NB: ownership of ns is transferred to the returned fetchState object. 1773 func (s *session) newFetchStateWithRLock( 1774 ctx gocontext.Context, 1775 ns ident.ID, 1776 opts newFetchStateOpts, 1777 ) (*fetchState, error) { 1778 var ( 1779 topoMap = s.state.topoMap 1780 fetchState = s.pools.fetchState.Get() 1781 ) 1782 fetchState.nsID = ns // transfer ownership to `fetchState` 1783 fetchState.incRef() // indicate current go-routine has a reference to the fetchState 1784 1785 readLevel := s.state.readConsistencyLevelWithRLock(opts.readConsistencyLevel) 1786 1787 // wire up the operation based on the opts specified 1788 var ( 1789 op op 1790 closer func() 1791 ) 1792 switch opts.stateType { 1793 case fetchTaggedFetchState: 1794 fetchOp := s.pools.fetchTaggedOp.Get() 1795 fetchOp.incRef() // indicate current go-routine has a reference to the op 1796 closer = fetchOp.decRef // release the ref for the current go-routine 1797 fetchOp.update(ctx, opts.fetchTaggedRequest, fetchState.completionFn) 1798 fetchState.ResetFetchTagged(opts.startInclusive, opts.endExclusive, 1799 fetchOp, topoMap, s.state.majority, readLevel) 1800 op = fetchOp 1801 1802 case aggregateFetchState: 1803 aggOp := s.pools.aggregateOp.Get() 1804 aggOp.incRef() // indicate current go-routine has a reference to the op 1805 closer = aggOp.decRef // release the ref for the current go-routine 1806 aggOp.update(ctx, opts.aggregateRequest, fetchState.completionFn) 1807 fetchState.ResetAggregate(opts.startInclusive, opts.endExclusive, 1808 aggOp, topoMap, s.state.majority, readLevel) 1809 op = aggOp 1810 1811 default: 1812 fetchState.decRef() // release fetchState 1813 instrument.EmitInvariantViolation(s.opts.InstrumentOptions()) 1814 return nil, xerrors.NewNonRetryableError(instrument.InvariantErrorf( 1815 "unknown fetchState type: %v", opts.stateType)) 1816 } 1817 1818 fetchState.Lock() 1819 for _, hq := range s.state.queues { 1820 // inc to indicate the hostQueue has a reference to `op` which has a ref to the fetchState 1821 fetchState.incRef() 1822 if err := hq.Enqueue(op); err != nil { 1823 fetchState.Unlock() 1824 closer() // release the ref for the current go-routine 1825 fetchState.decRef() // release the ref for the hostQueue 1826 fetchState.decRef() // release the ref for the current go-routine 1827 1828 // NB: if this happens we have a bug, once we are in the read 1829 // lock the current queues should never be closed 1830 wrappedErr := xerrors.NewNonRetryableError(fmt.Errorf("failed to enqueue in fetchState: %v", err)) 1831 instrument.EmitAndLogInvariantViolation(s.opts.InstrumentOptions(), func(l *zap.Logger) { 1832 l.Error(wrappedErr.Error()) 1833 }) 1834 return nil, wrappedErr 1835 } 1836 } 1837 1838 closer() // release the ref for the current go-routine 1839 1840 // NB(prateek): the calling go-routine still holds the lock and a ref 1841 // on the returned fetchState object. 1842 return fetchState, nil 1843 } 1844 1845 func (s *session) fetchIDsAttempt( 1846 inputNamespace ident.ID, 1847 inputIDs ident.Iterator, 1848 startInclusive, endExclusive xtime.UnixNano, 1849 ) (encoding.SeriesIterators, error) { 1850 nsCtx, err := s.nsCtxFor(inputNamespace) 1851 if err != nil { 1852 return nil, err 1853 } 1854 1855 var ( 1856 wg sync.WaitGroup 1857 allPending int32 1858 routeErr error 1859 enqueueErr error 1860 resultErrLock sync.RWMutex 1861 resultErr error 1862 resultErrs int32 1863 majority int32 1864 numReplicas int32 1865 readLevel topology.ReadConsistencyLevel 1866 fetchBatchOpsByHostIdx [][]*fetchBatchOp 1867 success = false 1868 startFetchAttempt = s.nowFn() 1869 ) 1870 1871 // NB(prateek): need to make a copy of inputNamespace and inputIDs to control 1872 // their life-cycle within this function. 1873 namespace := s.pools.id.Clone(inputNamespace) 1874 // First, we duplicate the iterator (only the struct referencing the underlying slice, 1875 // not the slice itself). Need this to be able to iterate the original iterator 1876 // multiple times in case of retries. 1877 ids := inputIDs.Duplicate() 1878 1879 rangeStart, tsErr := convert.ToValue(startInclusive, rpc.TimeType_UNIX_NANOSECONDS) 1880 if tsErr != nil { 1881 return nil, tsErr 1882 } 1883 1884 rangeEnd, tsErr := convert.ToValue(endExclusive, rpc.TimeType_UNIX_NANOSECONDS) 1885 if tsErr != nil { 1886 return nil, tsErr 1887 } 1888 1889 s.state.RLock() 1890 if s.state.status != statusOpen { 1891 s.state.RUnlock() 1892 return nil, ErrSessionStatusNotOpen 1893 } 1894 1895 iters := encoding.NewSizedSeriesIterators(ids.Remaining()) 1896 1897 defer func() { 1898 // NB(r): Ensure we cover all edge cases and close the iters in any case 1899 // of an error being returned 1900 if !success { 1901 iters.Close() 1902 } 1903 }() 1904 1905 // NB(r): We must take and return pooled items in the session read lock for the 1906 // pools that change during a topology update. 1907 // This is due to when a queue is re-initialized it enqueues a fixed number 1908 // of entries into the backing channel for the pool and will forever stall 1909 // on the last few puts if any unexpected entries find their way there 1910 // while it is filling. 1911 fetchBatchOpsByHostIdx = s.pools.fetchBatchOpArrayArray.Get() 1912 1913 readLevel = s.state.readLevel 1914 majority = int32(s.state.majority) 1915 numReplicas = int32(s.state.replicas) 1916 1917 // NB(prateek): namespaceAccessors tracks the number of pending accessors for nsID. 1918 // It is set to incremented by `replica` for each requested ID during fetch enqueuing, 1919 // and once by initial request, and is decremented for each replica retrieved, inside 1920 // completionFn, and once by the allCompletionFn. So know we can Finalize `namespace` 1921 // once it's value reaches 0. 1922 namespaceAccessors := int32(0) 1923 1924 for idx := 0; ids.Next(); idx++ { 1925 var ( 1926 idx = idx // capture loop variable 1927 tsID = s.pools.id.Clone(ids.Current()) 1928 1929 wgIsDone int32 1930 // NB(xichen): resultsAccessors and idAccessors get initialized to number of replicas + 1 1931 // before enqueuing (incremented when iterating over the replicas for this ID), and gets 1932 // decremented for each replica as well as inside the allCompletionFn so we know when 1933 // resultsAccessors is 0, results are no longer accessed and it's safe to return results 1934 // to the pool. 1935 resultsAccessors int32 = 1 1936 idAccessors int32 = 1 1937 resultsLock sync.RWMutex 1938 results []encoding.MultiReaderIterator 1939 enqueued int32 1940 pending int32 1941 success int32 1942 errors []error 1943 errs int32 1944 ) 1945 1946 // increment namespaceAccesors by 1 to indicate it still needs to be handled by the 1947 // allCompletionFn for tsID. 1948 atomic.AddInt32(&namespaceAccessors, 1) 1949 1950 wg.Add(1) 1951 allCompletionFn := func() { 1952 var reportErrors []error 1953 errsLen := atomic.LoadInt32(&errs) 1954 if errsLen > 0 { 1955 resultErrLock.RLock() 1956 reportErrors = errors[:] 1957 resultErrLock.RUnlock() 1958 } 1959 responded := enqueued - atomic.LoadInt32(&pending) 1960 err := s.readConsistencyResult(readLevel, majority, enqueued, 1961 responded, errsLen, reportErrors) 1962 s.recordFetchMetrics(err, errsLen, startFetchAttempt) 1963 if err != nil { 1964 resultErrLock.Lock() 1965 if resultErr == nil { 1966 resultErr = err 1967 } 1968 resultErrs++ 1969 resultErrLock.Unlock() 1970 } else { 1971 resultsLock.RLock() 1972 numItersToInclude := int(success) 1973 numDesired := topology.NumDesiredForReadConsistency(readLevel, int(numReplicas), int(majority)) 1974 if numDesired < numItersToInclude { 1975 // Avoid decoding more data than is required to satisfy the consistency guarantees. 1976 numItersToInclude = numDesired 1977 } 1978 1979 itersToInclude := results[:numItersToInclude] 1980 resultsLock.RUnlock() 1981 1982 iter := s.pools.seriesIterator.Get() 1983 // NB(prateek): we need to allocate a copy of ident.ID to allow the seriesIterator 1984 // to have control over the lifecycle of ID. We cannot allow seriesIterator 1985 // to control the lifecycle of the original ident.ID, as it might still be in use 1986 // due to a pending request in queue. 1987 seriesID := s.pools.id.Clone(tsID) 1988 namespaceID := s.pools.id.Clone(namespace) 1989 consolidator := s.opts.IterationOptions().SeriesIteratorConsolidator 1990 iter.Reset(encoding.SeriesIteratorOptions{ 1991 ID: seriesID, 1992 Namespace: namespaceID, 1993 StartInclusive: startInclusive, 1994 EndExclusive: endExclusive, 1995 Replicas: itersToInclude, 1996 SeriesIteratorConsolidator: consolidator, 1997 }) 1998 iters.SetAt(idx, iter) 1999 } 2000 if atomic.AddInt32(&resultsAccessors, -1) == 0 { 2001 s.pools.multiReaderIteratorArray.Put(results) 2002 } 2003 if atomic.AddInt32(&idAccessors, -1) == 0 { 2004 tsID.Finalize() 2005 } 2006 if atomic.AddInt32(&namespaceAccessors, -1) == 0 { 2007 namespace.Finalize() 2008 } 2009 wg.Done() 2010 } 2011 completionFn := func(result interface{}, err error) { 2012 var snapshotSuccess int32 2013 if err != nil { 2014 if IsBadRequestError(err) { 2015 // Wrap with invalid params and non-retryable so it is 2016 // not retried. 2017 err = xerrors.NewInvalidParamsError(err) 2018 err = xerrors.NewNonRetryableError(err) 2019 } 2020 atomic.AddInt32(&errs, 1) 2021 // NB(r): reuse the error lock here as we do not want to create 2022 // a whole lot of locks for every single ID fetched due to size 2023 // of mutex being non-trivial and likely to cause more stack growth 2024 // or GC pressure if ends up on heap which is likely due to naive 2025 // escape analysis. 2026 resultErrLock.Lock() 2027 errors = append(errors, err) 2028 resultErrLock.Unlock() 2029 } else { 2030 slicesIter := s.pools.readerSliceOfSlicesIterator.Get() 2031 slicesIter.Reset(result.([]*rpc.Segments)) 2032 multiIter := s.pools.multiReaderIterator.Get() 2033 multiIter.ResetSliceOfSlices(slicesIter, nsCtx.Schema) 2034 // Results is pre-allocated after creating fetch ops for this ID below 2035 resultsLock.Lock() 2036 results[success] = multiIter 2037 success++ 2038 snapshotSuccess = success 2039 resultsLock.Unlock() 2040 } 2041 // NB(xichen): decrementing pending and checking remaining against zero must 2042 // come after incrementing success, otherwise we might end up passing results[:success] 2043 // to iter.Reset down below before setting the iterator in the results array, 2044 // which would cause a nil pointer exception. 2045 remaining := atomic.AddInt32(&pending, -1) 2046 shouldTerminate := topology.ReadConsistencyTermination( 2047 readLevel, majority, remaining, snapshotSuccess, 2048 ) 2049 if shouldTerminate && atomic.CompareAndSwapInt32(&wgIsDone, 0, 1) { 2050 allCompletionFn() 2051 } 2052 2053 if atomic.AddInt32(&resultsAccessors, -1) == 0 { 2054 s.pools.multiReaderIteratorArray.Put(results) 2055 } 2056 if atomic.AddInt32(&idAccessors, -1) == 0 { 2057 tsID.Finalize() 2058 } 2059 if atomic.AddInt32(&namespaceAccessors, -1) == 0 { 2060 namespace.Finalize() 2061 } 2062 } 2063 2064 if err := s.state.topoMap.RouteForEach(tsID, func( 2065 hostIdx int, 2066 hostShard shard.Shard, 2067 host topology.Host, 2068 ) { 2069 // Inc safely as this for each is sequential 2070 enqueued++ 2071 pending++ 2072 allPending++ 2073 resultsAccessors++ 2074 namespaceAccessors++ 2075 idAccessors++ 2076 2077 ops := fetchBatchOpsByHostIdx[hostIdx] 2078 2079 var f *fetchBatchOp 2080 if len(ops) > 0 { 2081 // Find the last and potentially current fetch op for this host 2082 f = ops[len(ops)-1] 2083 } 2084 if f == nil || f.Size() >= s.fetchBatchSize { 2085 // If no current fetch op or existing one is at batch capacity add one 2086 // NB(r): Note that we defer to the host queue to take ownership 2087 // of these ops and for returning the ops to the pool when done as 2088 // they know when their use is complete. 2089 f = s.pools.fetchBatchOp.Get() 2090 f.IncRef() 2091 fetchBatchOpsByHostIdx[hostIdx] = append(fetchBatchOpsByHostIdx[hostIdx], f) 2092 f.request.RangeStart = rangeStart 2093 f.request.RangeEnd = rangeEnd 2094 f.request.RangeTimeType = rpc.TimeType_UNIX_NANOSECONDS 2095 } 2096 2097 // Append IDWithNamespace to this request 2098 f.append(namespace.Bytes(), tsID.Bytes(), completionFn) 2099 }); err != nil { 2100 routeErr = err 2101 break 2102 } 2103 2104 // Once we've enqueued we know how many to expect so retrieve and set length 2105 results = s.pools.multiReaderIteratorArray.Get(int(enqueued)) 2106 results = results[:enqueued] 2107 } 2108 2109 if routeErr != nil { 2110 s.state.RUnlock() 2111 return nil, routeErr 2112 } 2113 2114 // Enqueue fetch ops 2115 for idx := range fetchBatchOpsByHostIdx { 2116 for _, f := range fetchBatchOpsByHostIdx[idx] { 2117 // Passing ownership of the op itself to the host queue 2118 f.DecRef() 2119 if err := s.state.queues[idx].Enqueue(f); err != nil && enqueueErr == nil { 2120 enqueueErr = err 2121 break 2122 } 2123 } 2124 if enqueueErr != nil { 2125 break 2126 } 2127 } 2128 s.pools.fetchBatchOpArrayArray.Put(fetchBatchOpsByHostIdx) 2129 s.state.RUnlock() 2130 2131 if enqueueErr != nil { 2132 s.log.Error("failed to enqueue fetch", zap.Error(enqueueErr)) 2133 return nil, enqueueErr 2134 } 2135 2136 wg.Wait() 2137 2138 resultErrLock.RLock() 2139 retErr := resultErr 2140 resultErrLock.RUnlock() 2141 if retErr != nil { 2142 return nil, retErr 2143 } 2144 success = true 2145 return iters, nil 2146 } 2147 2148 func (s *session) writeConsistencyResult( 2149 level topology.ConsistencyLevel, 2150 majority, enqueued, responded, resultErrs int32, 2151 errs []error, 2152 ) error { 2153 // Check consistency level satisfied 2154 success := enqueued - resultErrs 2155 if !topology.WriteConsistencyAchieved(level, int(majority), int(enqueued), int(success)) { 2156 return newConsistencyResultError(level, int(enqueued), int(responded), errs) 2157 } 2158 return nil 2159 } 2160 2161 func (s *session) readConsistencyResult( 2162 level topology.ReadConsistencyLevel, 2163 majority, enqueued, responded, resultErrs int32, 2164 errs []error, 2165 ) error { 2166 // Check consistency level satisfied 2167 success := enqueued - resultErrs 2168 if !topology.ReadConsistencyAchieved(level, int(majority), int(enqueued), int(success)) { 2169 return newConsistencyResultError(level, int(enqueued), int(responded), errs) 2170 } 2171 return nil 2172 } 2173 2174 func (s *session) IteratorPools() (encoding.IteratorPools, error) { 2175 s.state.RLock() 2176 defer s.state.RUnlock() 2177 if s.state.status != statusOpen { 2178 return nil, ErrSessionStatusNotOpen 2179 } 2180 return s.pools, nil 2181 } 2182 2183 func (s *session) Close() error { 2184 s.state.Lock() 2185 if s.state.status != statusOpen { 2186 s.state.Unlock() 2187 return ErrSessionStatusNotOpen 2188 } 2189 s.state.status = statusClosed 2190 queues := s.state.queues 2191 topoWatch := s.state.topoWatch 2192 topo := s.state.topo 2193 s.state.Unlock() 2194 2195 for _, q := range queues { 2196 q.Close() 2197 } 2198 2199 topoWatch.Close() 2200 topo.Close() 2201 2202 if closer := s.runtimeOptsListenerCloser; closer != nil { 2203 closer.Close() 2204 } 2205 2206 return nil 2207 } 2208 2209 func (s *session) Origin() topology.Host { 2210 return s.origin 2211 } 2212 2213 func (s *session) Replicas() int { 2214 s.state.RLock() 2215 v := s.state.replicas 2216 s.state.RUnlock() 2217 return v 2218 } 2219 2220 func (s *session) TopologyMap() (topology.Map, error) { 2221 s.state.RLock() 2222 topoMap, err := s.topologyMapWithStateRLock() 2223 s.state.RUnlock() 2224 return topoMap, err 2225 } 2226 2227 func (s *session) topologyMapWithStateRLock() (topology.Map, error) { 2228 status := s.state.status 2229 topoMap := s.state.topoMap 2230 2231 // Make sure the session is open, as thats what sets the initial topology. 2232 if status != statusOpen { 2233 return nil, ErrSessionStatusNotOpen 2234 } 2235 if topoMap == nil { 2236 // Should never happen. 2237 return nil, instrument.InvariantErrorf("session does not have a topology map") 2238 } 2239 2240 return topoMap, nil 2241 } 2242 2243 func (s *session) Truncate(namespace ident.ID) (int64, error) { 2244 var ( 2245 wg sync.WaitGroup 2246 enqueueErr xerrors.MultiError 2247 resultErrLock sync.Mutex 2248 resultErr xerrors.MultiError 2249 truncated int64 2250 ) 2251 2252 t := &truncateOp{} 2253 t.request.NameSpace = namespace.Bytes() 2254 t.completionFn = func(result interface{}, err error) { 2255 if err != nil { 2256 resultErrLock.Lock() 2257 resultErr = resultErr.Add(err) 2258 resultErrLock.Unlock() 2259 } else { 2260 res := result.(*rpc.TruncateResult_) 2261 atomic.AddInt64(&truncated, res.NumSeries) 2262 } 2263 wg.Done() 2264 } 2265 2266 s.state.RLock() 2267 for idx := range s.state.queues { 2268 wg.Add(1) 2269 if err := s.state.queues[idx].Enqueue(t); err != nil { 2270 wg.Done() 2271 enqueueErr = enqueueErr.Add(err) 2272 } 2273 } 2274 s.state.RUnlock() 2275 2276 if err := enqueueErr.FinalError(); err != nil { 2277 s.log.Error("failed to enqueue request", zap.Error(err)) 2278 return 0, err 2279 } 2280 2281 // Wait for namespace to be truncated on all replicas 2282 wg.Wait() 2283 2284 return truncated, resultErr.FinalError() 2285 } 2286 2287 // NB(r): Excluding maligned struct check here as we can 2288 // live with a few extra bytes since this struct is only 2289 // ever passed by stack, its much more readable not optimized 2290 // nolint: maligned 2291 type peers struct { 2292 peers []peer 2293 shard uint32 2294 majorityReplicas int 2295 selfExcluded bool 2296 selfHostShardSet topology.HostShardSet 2297 } 2298 2299 func (p peers) selfExcludedAndSelfHasShardAvailable() bool { 2300 if !p.selfExcluded { 2301 return false 2302 } 2303 state, err := p.selfHostShardSet.ShardSet().LookupStateByID(p.shard) 2304 if err != nil { 2305 return false 2306 } 2307 return state == shard.Available 2308 } 2309 2310 func (s *session) peersForShard(shardID uint32) (peers, error) { 2311 s.state.RLock() 2312 var ( 2313 lookupErr error 2314 result = peers{ 2315 peers: make([]peer, 0, s.state.topoMap.Replicas()), 2316 shard: shardID, 2317 majorityReplicas: s.state.topoMap.MajorityReplicas(), 2318 } 2319 ) 2320 err := s.state.topoMap.RouteShardForEach(shardID, func( 2321 idx int, 2322 _ shard.Shard, 2323 host topology.Host, 2324 ) { 2325 if s.origin != nil && s.origin.ID() == host.ID() { 2326 // Don't include the origin host 2327 result.selfExcluded = true 2328 // Include the origin host shard set for help determining quorum 2329 hostShardSet, ok := s.state.topoMap.LookupHostShardSet(host.ID()) 2330 if !ok { 2331 lookupErr = fmt.Errorf("could not find shard set for host ID: %s", host.ID()) 2332 } 2333 result.selfHostShardSet = hostShardSet 2334 return 2335 } 2336 result.peers = append(result.peers, newPeer(s, host)) 2337 }) 2338 s.state.RUnlock() 2339 if resultErr := xerrors.FirstError(err, lookupErr); resultErr != nil { 2340 return peers{}, resultErr 2341 } 2342 return result, nil 2343 } 2344 2345 func (s *session) FetchBootstrapBlocksMetadataFromPeers( 2346 namespace ident.ID, 2347 shard uint32, 2348 start, end xtime.UnixNano, 2349 resultOpts result.Options, 2350 ) (PeerBlockMetadataIter, error) { 2351 level := newSessionBootstrapRuntimeReadConsistencyLevel(s) 2352 return s.fetchBlocksMetadataFromPeers(namespace, 2353 shard, start, end, level, resultOpts) 2354 } 2355 2356 func (s *session) FetchBlocksMetadataFromPeers( 2357 namespace ident.ID, 2358 shard uint32, 2359 start, end xtime.UnixNano, 2360 consistencyLevel topology.ReadConsistencyLevel, 2361 resultOpts result.Options, 2362 ) (PeerBlockMetadataIter, error) { 2363 level := newStaticRuntimeReadConsistencyLevel(consistencyLevel) 2364 return s.fetchBlocksMetadataFromPeers(namespace, 2365 shard, start, end, level, resultOpts) 2366 } 2367 2368 func (s *session) fetchBlocksMetadataFromPeers( 2369 namespace ident.ID, 2370 shard uint32, 2371 start, end xtime.UnixNano, 2372 level runtimeReadConsistencyLevel, 2373 resultOpts result.Options, 2374 ) (PeerBlockMetadataIter, error) { 2375 peers, err := s.peersForShard(shard) 2376 if err != nil { 2377 return nil, err 2378 } 2379 2380 var ( 2381 metadataCh = make(chan receivedBlockMetadata, 2382 blockMetadataChBufSize) 2383 errCh = make(chan error, 1) 2384 meta = resultTypeMetadata 2385 m = s.newPeerMetadataStreamingProgressMetrics(shard, meta) 2386 ) 2387 go func() { 2388 errCh <- s.streamBlocksMetadataFromPeers(namespace, shard, 2389 peers, start, end, level, metadataCh, resultOpts, m) 2390 close(metadataCh) 2391 close(errCh) 2392 }() 2393 2394 iter := newMetadataIter(metadataCh, errCh, 2395 s.pools.tagDecoder, s.pools.id) 2396 return iter, nil 2397 } 2398 2399 // FetchBootstrapBlocksFromPeers will fetch the specified blocks from peers for 2400 // bootstrapping purposes. Refer to peer_bootstrapping.md for more details. 2401 func (s *session) FetchBootstrapBlocksFromPeers( 2402 nsMetadata namespace.Metadata, 2403 shard uint32, 2404 start, end xtime.UnixNano, 2405 opts result.Options, 2406 ) (result.ShardResult, error) { 2407 nsCtx, err := s.nsCtxFromMetadata(nsMetadata) 2408 if err != nil { 2409 return nil, err 2410 } 2411 var ( 2412 result = newBulkBlocksResult(nsCtx, s.opts, opts, 2413 s.pools.tagDecoder, s.pools.id) 2414 doneCh = make(chan struct{}) 2415 progress = s.newPeerMetadataStreamingProgressMetrics(shard, 2416 resultTypeBootstrap) 2417 level = newSessionBootstrapRuntimeReadConsistencyLevel(s) 2418 ) 2419 2420 // Determine which peers own the specified shard 2421 peers, err := s.peersForShard(shard) 2422 if err != nil { 2423 return nil, err 2424 } 2425 2426 // Emit a gauge indicating whether we're done or not 2427 go func() { 2428 for { 2429 select { 2430 case <-doneCh: 2431 progress.fetchBlocksFromPeers.Update(0) 2432 return 2433 default: 2434 progress.fetchBlocksFromPeers.Update(1) 2435 time.Sleep(gaugeReportInterval) 2436 } 2437 } 2438 }() 2439 defer close(doneCh) 2440 2441 // Begin pulling metadata, if one or multiple peers fail no error will 2442 // be returned from this routine as long as one peer succeeds completely 2443 metadataCh := make(chan receivedBlockMetadata, blockMetadataChBufSize) 2444 // Spin up a background goroutine which will begin streaming metadata from 2445 // all the peers and pushing them into the metadatach 2446 errCh := make(chan error, 1) 2447 go func() { 2448 errCh <- s.streamBlocksMetadataFromPeers(nsMetadata.ID(), shard, 2449 peers, start, end, level, metadataCh, opts, progress) 2450 close(metadataCh) 2451 }() 2452 2453 // Begin consuming metadata and making requests. This will block until all 2454 // data has been streamed (or failed to stream). Note that while this function 2455 // does return an error, an error will only be returned in a select few cases. 2456 // There are some scenarios in which if something goes wrong here we won't report it to 2457 // the caller, but metrics and logs are emitted internally. Also note that the 2458 // streamAndGroupCollectedBlocksMetadata function is injected. 2459 err = s.streamBlocksFromPeers(nsMetadata, shard, peers, metadataCh, opts, 2460 level, result, progress, s.streamAndGroupCollectedBlocksMetadata) 2461 if err != nil { 2462 return nil, err 2463 } 2464 2465 // Check if an error occurred during the metadata streaming 2466 if err = <-errCh; err != nil { 2467 return nil, err 2468 } 2469 2470 return result.result, nil 2471 } 2472 2473 func (s *session) FetchBlocksFromPeers( 2474 nsMetadata namespace.Metadata, 2475 shard uint32, 2476 consistencyLevel topology.ReadConsistencyLevel, 2477 metadatas []block.ReplicaMetadata, 2478 opts result.Options, 2479 ) (PeerBlocksIter, error) { 2480 nsCtx, err := s.nsCtxFromMetadata(nsMetadata) 2481 if err != nil { 2482 return nil, err 2483 } 2484 var ( 2485 logger = opts.InstrumentOptions().Logger() 2486 level = newStaticRuntimeReadConsistencyLevel(consistencyLevel) 2487 complete = int64(0) 2488 doneCh = make(chan error, 1) 2489 outputCh = make(chan peerBlocksDatapoint, 4096) 2490 result = newStreamBlocksResult(nsCtx, s.opts, opts, outputCh, 2491 s.pools.tagDecoder, s.pools.id) 2492 onDone = func(err error) { 2493 atomic.StoreInt64(&complete, 1) 2494 select { 2495 case doneCh <- err: 2496 default: 2497 } 2498 } 2499 progress = s.newPeerMetadataStreamingProgressMetrics(shard, resultTypeRaw) 2500 ) 2501 2502 peers, err := s.peersForShard(shard) 2503 if err != nil { 2504 return nil, err 2505 } 2506 2507 peersByHost := make(map[string]peer, len(peers.peers)) 2508 for _, peer := range peers.peers { 2509 peersByHost[peer.Host().ID()] = peer 2510 } 2511 2512 // If any metadata has tags then encode them up front so can 2513 // return an error on tag encoding rather than logging error that would 2514 // possibly get missed. 2515 var ( 2516 metadatasEncodedTags []checked.Bytes 2517 anyTags bool 2518 ) 2519 for _, meta := range metadatas { 2520 if len(meta.Tags.Values()) > 0 { 2521 anyTags = true 2522 break 2523 } 2524 } 2525 if anyTags { 2526 // NB(r): Allocate exact length so nil is used and each index 2527 // references same index as the incoming metadatas being fetched. 2528 metadatasEncodedTags = make([]checked.Bytes, len(metadatas)) 2529 tagsIter := ident.NewTagsIterator(ident.Tags{}) 2530 for idx, meta := range metadatas { 2531 if len(meta.Tags.Values()) == 0 { 2532 continue 2533 } 2534 2535 tagsIter.Reset(meta.Tags) 2536 tagsEncoder := s.pools.tagEncoder.Get() 2537 if err := tagsEncoder.Encode(tagsIter); err != nil { 2538 return nil, err 2539 } 2540 2541 encodedTagsCheckedBytes, ok := tagsEncoder.Data() 2542 if !ok { 2543 return nil, fmt.Errorf("could not encode tags: id=%s", meta.ID.String()) 2544 } 2545 2546 metadatasEncodedTags[idx] = encodedTagsCheckedBytes 2547 } 2548 } 2549 2550 go func() { 2551 for atomic.LoadInt64(&complete) == 0 { 2552 progress.fetchBlocksFromPeers.Update(1) 2553 time.Sleep(gaugeReportInterval) 2554 } 2555 progress.fetchBlocksFromPeers.Update(0) 2556 }() 2557 2558 metadataCh := make(chan receivedBlockMetadata, blockMetadataChBufSize) 2559 go func() { 2560 for idx, rb := range metadatas { 2561 peer, ok := peersByHost[rb.Host.ID()] 2562 if !ok { 2563 logger.Warn("replica requested from unknown peer, skipping", 2564 zap.Stringer("peer", rb.Host), 2565 zap.Stringer("id", rb.ID), 2566 zap.Time("start", rb.Start.ToTime()), 2567 ) 2568 continue 2569 } 2570 2571 // Attach encoded tags if present. 2572 var encodedTags checked.Bytes 2573 if idx < len(metadatasEncodedTags) { 2574 // Note: could still be nil if had no tags, but the slice 2575 // was built so need to take ref to encoded tags if 2576 // was encoded. 2577 encodedTags = metadatasEncodedTags[idx] 2578 } 2579 2580 metadataCh <- receivedBlockMetadata{ 2581 id: rb.Metadata.ID, 2582 encodedTags: encodedTags, 2583 peer: peer, 2584 block: blockMetadata{ 2585 start: rb.Start, 2586 size: rb.Size, 2587 checksum: rb.Checksum, 2588 lastRead: rb.LastRead, 2589 }, 2590 } 2591 } 2592 close(metadataCh) 2593 }() 2594 2595 // Begin consuming metadata and making requests. 2596 go func() { 2597 err := s.streamBlocksFromPeers(nsMetadata, shard, peers, metadataCh, 2598 opts, level, result, progress, s.passThroughBlocksMetadata) 2599 close(outputCh) 2600 onDone(err) 2601 }() 2602 2603 pbi := newPeerBlocksIter(outputCh, doneCh) 2604 return pbi, nil 2605 } 2606 2607 func (s *session) streamBlocksMetadataFromPeers( 2608 namespace ident.ID, 2609 shardID uint32, 2610 peers peers, 2611 start, end xtime.UnixNano, 2612 level runtimeReadConsistencyLevel, 2613 metadataCh chan<- receivedBlockMetadata, 2614 resultOpts result.Options, 2615 progress *streamFromPeersMetrics, 2616 ) error { 2617 var ( 2618 wg sync.WaitGroup 2619 errs = newSyncAbortableErrorsMap() 2620 pending = int64(len(peers.peers)) 2621 majority = int32(peers.majorityReplicas) 2622 enqueued = int32(len(peers.peers)) 2623 responded int32 2624 success int32 2625 ) 2626 if peers.selfExcludedAndSelfHasShardAvailable() { 2627 // If we excluded ourselves from fetching, we basically treat ourselves 2628 // as a successful peer response since we can bootstrap from ourselves 2629 // just fine 2630 enqueued++ 2631 success++ 2632 } 2633 2634 progress.metadataFetches.Update(float64(pending)) 2635 for idx, peer := range peers.peers { 2636 idx := idx 2637 peer := peer 2638 2639 wg.Add(1) 2640 go func() { 2641 defer func() { 2642 // Success or error counts towards a response 2643 atomic.AddInt32(&responded, 1) 2644 2645 // Decrement pending 2646 progress.metadataFetches.Update(float64(atomic.AddInt64(&pending, -1))) 2647 2648 // Mark done 2649 wg.Done() 2650 }() 2651 2652 var ( 2653 firstAttempt = true 2654 // NB(r): currPageToken keeps the position into the pagination of the 2655 // metadata from this peer, it begins as nil but if an error is 2656 // returned it will likely not be nil, this lets us restart fetching 2657 // if we need to (if consistency has not been achieved yet) without 2658 // losing place in the pagination. 2659 currPageToken pageToken 2660 currHostNotAvailableSleepInterval = hostNotAvailableMinSleepInterval 2661 ) 2662 condition := func() bool { 2663 if firstAttempt { 2664 // Always attempt at least once 2665 firstAttempt = false 2666 return true 2667 } 2668 2669 var ( 2670 currLevel = level.value() 2671 majority = int(majority) 2672 enqueued = int(enqueued) 2673 success = int(atomic.LoadInt32(&success)) 2674 ) 2675 metReadConsistency := topology.ReadConsistencyAchieved( 2676 currLevel, majority, enqueued, success) 2677 doRetry := !metReadConsistency && errs.getAbortError() == nil 2678 2679 if doRetry { 2680 // Track that we are reattempting the fetch metadata 2681 // pagination from a peer 2682 progress.metadataPeerRetry.Inc(1) 2683 } 2684 return doRetry 2685 } 2686 for condition() { 2687 var err error 2688 currPageToken, err = s.streamBlocksMetadataFromPeer(namespace, shardID, 2689 peer, start, end, currPageToken, metadataCh, resultOpts, progress) 2690 // Set error or success if err is nil 2691 errs.setError(idx, err) 2692 2693 // hostNotAvailable is a NonRetryableError for the purposes of short-circuiting 2694 // the automatic retry functionality, but in this case the client should avoid 2695 // aborting and continue retrying at this level until consistency can be reached. 2696 if isHostNotAvailableError(err) { 2697 // Prevent the loop from spinning too aggressively in the short-circuiting case. 2698 time.Sleep(currHostNotAvailableSleepInterval) 2699 currHostNotAvailableSleepInterval = minDuration( 2700 currHostNotAvailableSleepInterval*2, 2701 hostNotAvailableMaxSleepInterval, 2702 ) 2703 continue 2704 } 2705 2706 if err != nil && xerrors.IsNonRetryableError(err) { 2707 errs.setAbortError(err) 2708 return // Cannot recover from this error, so we break from the loop 2709 } 2710 2711 if err == nil { 2712 atomic.AddInt32(&success, 1) 2713 return 2714 } 2715 2716 // There was a retryable error, continue looping. 2717 } 2718 }() 2719 } 2720 2721 wg.Wait() 2722 2723 if err := errs.getAbortError(); err != nil { 2724 return err 2725 } 2726 2727 errors := errs.getErrors() 2728 return s.readConsistencyResult(level.value(), majority, enqueued, 2729 atomic.LoadInt32(&responded), int32(len(errors)), errors) 2730 } 2731 2732 type pageToken []byte 2733 2734 // streamBlocksMetadataFromPeer has several heap allocated anonymous 2735 // function, however, they're only allocated once per peer/shard combination 2736 // for the entire peer bootstrapping process so performance is acceptable 2737 func (s *session) streamBlocksMetadataFromPeer( 2738 namespace ident.ID, 2739 shard uint32, 2740 peer peer, 2741 start, end xtime.UnixNano, 2742 startPageToken pageToken, 2743 metadataCh chan<- receivedBlockMetadata, 2744 resultOpts result.Options, 2745 progress *streamFromPeersMetrics, 2746 ) (pageToken, error) { 2747 var ( 2748 optionIncludeSizes = true 2749 optionIncludeChecksums = true 2750 optionIncludeLastRead = true 2751 moreResults = true 2752 idPool = s.pools.id 2753 bytesPool = resultOpts.DatabaseBlockOptions().BytesPool() 2754 2755 // Only used for logs 2756 peerStr = peer.Host().ID() 2757 metadataCountByBlock = map[xtime.UnixNano]int64{} 2758 ) 2759 defer func() { 2760 for block, numMetadata := range metadataCountByBlock { 2761 s.log.Debug("finished streaming blocks metadata from peer", 2762 zap.Uint32("shard", shard), 2763 zap.String("peer", peerStr), 2764 zap.Int64("numMetadata", numMetadata), 2765 zap.Time("block", block.ToTime()), 2766 ) 2767 } 2768 }() 2769 2770 // Declare before loop to avoid redeclaring each iteration 2771 attemptFn := func(client rpc.TChanNode) error { 2772 tctx, _ := thrift.NewContext(s.streamBlocksMetadataBatchTimeout) 2773 req := rpc.NewFetchBlocksMetadataRawV2Request() 2774 req.NameSpace = namespace.Bytes() 2775 req.Shard = int32(shard) 2776 req.RangeStart = int64(start) 2777 req.RangeEnd = int64(end) 2778 req.Limit = int64(s.streamBlocksBatchSize) 2779 req.PageToken = startPageToken 2780 req.IncludeSizes = &optionIncludeSizes 2781 req.IncludeChecksums = &optionIncludeChecksums 2782 req.IncludeLastRead = &optionIncludeLastRead 2783 2784 progress.metadataFetchBatchCall.Inc(1) 2785 result, err := client.FetchBlocksMetadataRawV2(tctx, req) 2786 if err != nil { 2787 progress.metadataFetchBatchError.Inc(1) 2788 return err 2789 } 2790 2791 progress.metadataFetchBatchSuccess.Inc(1) 2792 progress.metadataReceived.Inc(int64(len(result.Elements))) 2793 2794 if result.NextPageToken != nil { 2795 // Reset pageToken + copy new pageToken into previously allocated memory, 2796 // extending as necessary 2797 startPageToken = append(startPageToken[:0], result.NextPageToken...) 2798 } else { 2799 // No further results 2800 moreResults = false 2801 } 2802 2803 for _, elem := range result.Elements { 2804 blockStart := xtime.UnixNano(elem.Start) 2805 2806 data := bytesPool.Get(len(elem.ID)) 2807 data.IncRef() 2808 data.AppendAll(elem.ID) 2809 data.DecRef() 2810 clonedID := idPool.BinaryID(data) 2811 // Return thrift bytes to pool once the ID has been copied. 2812 tbinarypool.BytesPoolPut(elem.ID) 2813 2814 var encodedTags checked.Bytes 2815 if tagBytes := elem.EncodedTags; len(tagBytes) != 0 { 2816 encodedTags = bytesPool.Get(len(tagBytes)) 2817 encodedTags.IncRef() 2818 encodedTags.AppendAll(tagBytes) 2819 encodedTags.DecRef() 2820 // Return thrift bytes to pool once the tags have been copied. 2821 tbinarypool.BytesPoolPut(tagBytes) 2822 } 2823 2824 // Error occurred retrieving block metadata, use default values 2825 if err := elem.Err; err != nil { 2826 progress.metadataFetchBatchBlockErr.Inc(1) 2827 s.log.Error("error occurred retrieving block metadata", 2828 zap.Uint32("shard", shard), 2829 zap.String("peer", peerStr), 2830 zap.Time("block", blockStart.ToTime()), 2831 zap.Error(err), 2832 ) 2833 // Enqueue with a zeroed checksum which triggers a fanout fetch 2834 metadataCh <- receivedBlockMetadata{ 2835 peer: peer, 2836 id: clonedID, 2837 encodedTags: encodedTags, 2838 block: blockMetadata{ 2839 start: blockStart, 2840 }, 2841 } 2842 continue 2843 } 2844 2845 var size int64 2846 if elem.Size != nil { 2847 size = *elem.Size 2848 } 2849 2850 var pChecksum *uint32 2851 if elem.Checksum != nil { 2852 value := uint32(*elem.Checksum) 2853 pChecksum = &value 2854 } 2855 2856 var lastRead xtime.UnixNano 2857 if elem.LastRead != nil { 2858 value, err := convert.ToTime(*elem.LastRead, elem.LastReadTimeType) 2859 if err == nil { 2860 lastRead = value 2861 } 2862 } 2863 2864 metadataCh <- receivedBlockMetadata{ 2865 peer: peer, 2866 id: clonedID, 2867 encodedTags: encodedTags, 2868 block: blockMetadata{ 2869 start: blockStart, 2870 size: size, 2871 checksum: pChecksum, 2872 lastRead: lastRead, 2873 }, 2874 } 2875 // Only used for logs 2876 metadataCountByBlock[blockStart]++ 2877 } 2878 return nil 2879 } 2880 2881 var attemptErr error 2882 checkedAttemptFn := func(client rpc.TChanNode, _ Channel) { 2883 attemptErr = attemptFn(client) 2884 } 2885 2886 fetchFn := func() error { 2887 borrowErr := peer.BorrowConnection(checkedAttemptFn) 2888 return xerrors.FirstError(borrowErr, attemptErr) 2889 } 2890 2891 for moreResults { 2892 if err := s.streamBlocksRetrier.Attempt(fetchFn); err != nil { 2893 return startPageToken, err 2894 } 2895 } 2896 return nil, nil 2897 } 2898 2899 func (s *session) streamBlocksFromPeers( 2900 nsMetadata namespace.Metadata, 2901 shard uint32, 2902 peers peers, 2903 metadataCh <-chan receivedBlockMetadata, 2904 opts result.Options, 2905 consistencyLevel runtimeReadConsistencyLevel, 2906 result blocksResult, 2907 progress *streamFromPeersMetrics, 2908 streamMetadataFn streamBlocksMetadataFn, 2909 ) error { 2910 var ( 2911 enqueueCh = newEnqueueChannel(progress) 2912 peerBlocksBatchSize = s.streamBlocksBatchSize 2913 numPeers = len(peers.peers) 2914 uncheckedBytesPool = opts.DatabaseBlockOptions().BytesPool().BytesPool() 2915 ) 2916 2917 // Consume the incoming metadata and enqueue to the ready channel 2918 // Spin up background goroutine to consume 2919 go func() { 2920 streamMetadataFn(numPeers, metadataCh, enqueueCh, uncheckedBytesPool) 2921 // Begin assessing the queue and how much is processed, once queue 2922 // is entirely processed then we can close the enqueue channel 2923 enqueueCh.closeOnAllProcessed() 2924 }() 2925 2926 // Fetch blocks from peers as results become ready 2927 peerQueues := make(peerBlocksQueues, 0, numPeers) 2928 for _, peer := range peers.peers { 2929 peer := peer 2930 size := peerBlocksBatchSize 2931 workers := s.streamBlocksWorkers 2932 drainEvery := 100 * time.Millisecond 2933 queue := s.newPeerBlocksQueueFn(peer, size, drainEvery, workers, 2934 func(batch []receivedBlockMetadata) { 2935 s.streamBlocksBatchFromPeer(nsMetadata, shard, peer, batch, opts, 2936 result, enqueueCh, s.streamBlocksRetrier, progress) 2937 }) 2938 peerQueues = append(peerQueues, queue) 2939 } 2940 2941 var ( 2942 selected []receivedBlockMetadata 2943 pooled selectPeersFromPerPeerBlockMetadatasPooledResources 2944 onQueueItemProcessed = func() { 2945 enqueueCh.trackProcessed(1) 2946 } 2947 ) 2948 for perPeerBlocksMetadata := range enqueueCh.read() { 2949 // Filter and select which blocks to retrieve from which peers 2950 selected, pooled = s.selectPeersFromPerPeerBlockMetadatas( 2951 perPeerBlocksMetadata, peerQueues, enqueueCh, consistencyLevel, peers, 2952 pooled, progress) 2953 2954 if len(selected) == 0 { 2955 onQueueItemProcessed() 2956 continue 2957 } 2958 2959 if len(selected) == 1 { 2960 queue := peerQueues.findQueue(selected[0].peer) 2961 queue.enqueue(selected[0], onQueueItemProcessed) 2962 continue 2963 } 2964 2965 // Need to fan out, only track this as processed once all peer 2966 // queues have completed their fetches, so account for the extra 2967 // items assigned to be fetched 2968 enqueueCh.trackPending(len(selected) - 1) 2969 for _, receivedBlockMetadata := range selected { 2970 queue := peerQueues.findQueue(receivedBlockMetadata.peer) 2971 queue.enqueue(receivedBlockMetadata, onQueueItemProcessed) 2972 } 2973 } 2974 2975 // Close all queues 2976 peerQueues.closeAll() 2977 2978 return nil 2979 } 2980 2981 type streamBlocksMetadataFn func( 2982 peersLen int, 2983 ch <-chan receivedBlockMetadata, 2984 enqueueCh enqueueChannel, 2985 pool pool.BytesPool, 2986 ) 2987 2988 func (s *session) passThroughBlocksMetadata( 2989 peersLen int, 2990 ch <-chan receivedBlockMetadata, 2991 enqueueCh enqueueChannel, 2992 _ pool.BytesPool, 2993 ) { 2994 // Receive off of metadata channel 2995 for { 2996 m, ok := <-ch 2997 if !ok { 2998 break 2999 } 3000 res := []receivedBlockMetadata{m} 3001 enqueueCh.enqueue(res) 3002 } 3003 } 3004 3005 func (s *session) streamAndGroupCollectedBlocksMetadata( 3006 peersLen int, 3007 metadataCh <-chan receivedBlockMetadata, 3008 enqueueCh enqueueChannel, 3009 pool pool.BytesPool, 3010 ) { 3011 metadata := newReceivedBlocksMap(pool) 3012 defer metadata.Reset() // Delete all the keys and return slices to pools 3013 3014 for { 3015 m, ok := <-metadataCh 3016 if !ok { 3017 break 3018 } 3019 3020 key := idAndBlockStart{ 3021 id: m.id, 3022 blockStart: int64(m.block.start), 3023 } 3024 received, ok := metadata.Get(key) 3025 if !ok { 3026 received = receivedBlocks{ 3027 results: make([]receivedBlockMetadata, 0, peersLen), 3028 } 3029 } 3030 3031 // The entry has already been enqueued which means the metadata we just 3032 // received is a duplicate. Discard it and move on. 3033 if received.enqueued { 3034 s.emitDuplicateMetadataLog(received, m) 3035 continue 3036 } 3037 3038 // Determine if the incoming metadata is a duplicate by checking if we've 3039 // already received metadata from this peer. 3040 existingIndex := -1 3041 for i, existingMetadata := range received.results { 3042 if existingMetadata.peer.Host().ID() == m.peer.Host().ID() { 3043 existingIndex = i 3044 break 3045 } 3046 } 3047 3048 if existingIndex != -1 { 3049 // If it is a duplicate, then overwrite it (always keep the most recent 3050 // duplicate) 3051 received.results[existingIndex] = m 3052 } else { 3053 // Otherwise it's not a duplicate, so its safe to append. 3054 received.results = append(received.results, m) 3055 } 3056 3057 // Since we always perform an overwrite instead of an append for duplicates 3058 // from the same peer, once len(received.results == peersLen) then we know 3059 // that we've received at least one metadata from every peer and its safe 3060 // to enqueue the entry. 3061 if len(received.results) == peersLen { 3062 enqueueCh.enqueue(received.results) 3063 received.enqueued = true 3064 } 3065 3066 // Ensure tracking enqueued by setting modified result back to map 3067 metadata.Set(key, received) 3068 } 3069 3070 // Enqueue all unenqueued received metadata. Note that these entries will have 3071 // metadata from only a subset of their peers. 3072 for _, entry := range metadata.Iter() { 3073 received := entry.Value() 3074 if received.enqueued { 3075 continue 3076 } 3077 enqueueCh.enqueue(received.results) 3078 } 3079 } 3080 3081 // emitDuplicateMetadataLog emits a log with the details of the duplicate metadata 3082 // event. Note: We're able to log the blocks themselves because the slice is no longer 3083 // mutated downstream after enqueuing into the enqueue channel, it's copied before 3084 // mutated or operated on. 3085 func (s *session) emitDuplicateMetadataLog( 3086 received receivedBlocks, 3087 metadata receivedBlockMetadata, 3088 ) { 3089 // Debug-level because this is a common enough occurrence that logging it by 3090 // default would be noisy. 3091 // This is due to peers sending the most recent data 3092 // to the oldest data in that order, hence sometimes its possible to resend 3093 // data for a block already sent over the wire if it just moved from being 3094 // mutable in memory to immutable on disk. 3095 if !s.log.Core().Enabled(zapcore.DebugLevel) { 3096 return 3097 } 3098 3099 var checksum uint32 3100 if v := metadata.block.checksum; v != nil { 3101 checksum = *v 3102 } 3103 3104 fields := make([]zapcore.Field, 0, len(received.results)+1) 3105 fields = append(fields, zap.String("incoming-metadata", fmt.Sprintf( 3106 "id=%s, peer=%s, start=%s, size=%v, checksum=%v", 3107 metadata.id.String(), 3108 metadata.peer.Host().String(), 3109 metadata.block.start.String(), 3110 metadata.block.size, 3111 checksum))) 3112 3113 for i, existing := range received.results { 3114 checksum = 0 3115 if v := existing.block.checksum; v != nil { 3116 checksum = *v 3117 } 3118 3119 fields = append(fields, zap.String( 3120 fmt.Sprintf("existing-metadata-%d", i), 3121 fmt.Sprintf( 3122 "id=%s, peer=%s, start=%s, size=%v, checksum=%v", 3123 existing.id.String(), 3124 existing.peer.Host().String(), 3125 existing.block.start.String(), 3126 existing.block.size, 3127 checksum))) 3128 } 3129 3130 s.log.Debug("received metadata, but peer metadata has already been submitted", fields...) 3131 } 3132 3133 type pickBestPeerFn func( 3134 perPeerBlockMetadata []receivedBlockMetadata, 3135 peerQueues peerBlocksQueues, 3136 resources pickBestPeerPooledResources, 3137 ) (index int, pooled pickBestPeerPooledResources) 3138 3139 type pickBestPeerPooledResources struct { 3140 ranking []receivedBlockMetadataQueue 3141 } 3142 3143 func (s *session) streamBlocksPickBestPeer( 3144 perPeerBlockMetadata []receivedBlockMetadata, 3145 peerQueues peerBlocksQueues, 3146 pooled pickBestPeerPooledResources, 3147 ) (int, pickBestPeerPooledResources) { 3148 // Order by least attempts then by least outstanding blocks being fetched 3149 pooled.ranking = pooled.ranking[:0] 3150 for i := range perPeerBlockMetadata { 3151 elem := receivedBlockMetadataQueue{ 3152 blockMetadata: perPeerBlockMetadata[i], 3153 queue: peerQueues.findQueue(perPeerBlockMetadata[i].peer), 3154 } 3155 pooled.ranking = append(pooled.ranking, elem) 3156 } 3157 elems := receivedBlockMetadataQueuesByAttemptsAscOutstandingAsc(pooled.ranking) 3158 sort.Stable(elems) 3159 3160 // Return index of the best peer 3161 var ( 3162 bestPeer = pooled.ranking[0].queue.peer 3163 idx int 3164 ) 3165 for i := range perPeerBlockMetadata { 3166 if bestPeer == perPeerBlockMetadata[i].peer { 3167 idx = i 3168 break 3169 } 3170 } 3171 return idx, pooled 3172 } 3173 3174 type selectPeersFromPerPeerBlockMetadatasPooledResources struct { 3175 currEligible []receivedBlockMetadata 3176 pickBestPeerPooledResources pickBestPeerPooledResources 3177 } 3178 3179 func (s *session) selectPeersFromPerPeerBlockMetadatas( 3180 perPeerBlocksMetadata []receivedBlockMetadata, 3181 peerQueues peerBlocksQueues, 3182 reEnqueueCh enqueueChannel, 3183 consistencyLevel runtimeReadConsistencyLevel, 3184 peers peers, 3185 pooled selectPeersFromPerPeerBlockMetadatasPooledResources, 3186 m *streamFromPeersMetrics, 3187 ) ([]receivedBlockMetadata, selectPeersFromPerPeerBlockMetadatasPooledResources) { 3188 // Copy into pooled array so we don't mutate existing slice passed 3189 pooled.currEligible = pooled.currEligible[:0] 3190 pooled.currEligible = append(pooled.currEligible, perPeerBlocksMetadata...) 3191 3192 currEligible := pooled.currEligible[:] 3193 3194 // Sort the per peer metadatas by peer ID for consistent results 3195 sort.Sort(peerBlockMetadataByID(currEligible)) 3196 3197 // Only select from peers not already attempted 3198 curr := currEligible[0] 3199 currID := curr.id 3200 currBlock := curr.block 3201 for i := len(currEligible) - 1; i >= 0; i-- { 3202 if currEligible[i].block.reattempt.attempt == 0 { 3203 // Not attempted yet 3204 continue 3205 } 3206 3207 // Check if eligible 3208 n := s.streamBlocksMaxBlockRetries 3209 if currEligible[i].block.reattempt.peerAttempts(currEligible[i].peer) >= n { 3210 // Swap current entry to tail 3211 receivedBlockMetadatas(currEligible).swap(i, len(currEligible)-1) 3212 // Trim newly last entry 3213 currEligible = currEligible[:len(currEligible)-1] 3214 continue 3215 } 3216 } 3217 3218 if len(currEligible) == 0 { 3219 // No current eligible peers to select from 3220 majority := peers.majorityReplicas 3221 enqueued := len(peers.peers) 3222 success := 0 3223 if peers.selfExcludedAndSelfHasShardAvailable() { 3224 // If we excluded ourselves from fetching, we basically treat ourselves 3225 // as a successful peer response since our copy counts towards quorum 3226 enqueued++ 3227 success++ 3228 } 3229 3230 errMsg := "all retries failed for streaming blocks from peers" 3231 fanoutFetchState := currBlock.reattempt.fanoutFetchState 3232 if fanoutFetchState != nil { 3233 if fanoutFetchState.decrementAndReturnPending() > 0 { 3234 // This block was fanned out to fetch from all peers and we haven't 3235 // received all the results yet, so don't retry it just yet 3236 return nil, pooled 3237 } 3238 3239 // NB(r): This was enqueued after a failed fetch and all other fanout 3240 // fetches have completed, check if the consistency level was achieved, 3241 // if not then re-enqueue to continue to retry otherwise do not 3242 // re-enqueue and see if we need mark this as an error. 3243 success = fanoutFetchState.success() 3244 } 3245 3246 level := consistencyLevel.value() 3247 achievedConsistencyLevel := topology.ReadConsistencyAchieved(level, majority, enqueued, success) 3248 if achievedConsistencyLevel { 3249 if success > 0 { 3250 // Some level of success met, no need to log an error 3251 return nil, pooled 3252 } 3253 3254 // No success, inform operator that although consistency level achieved 3255 // there were no successful fetches. This can happen if consistency 3256 // level is set to None. 3257 m.fetchBlockFinalError.Inc(1) 3258 s.log.Error(errMsg, 3259 zap.Stringer("id", currID), 3260 zap.Time("start", currBlock.start.ToTime()), 3261 zap.Int("attempted", currBlock.reattempt.attempt), 3262 zap.String("attemptErrs", xerrors.Errors(currBlock.reattempt.errs).Error()), 3263 zap.Stringer("consistencyLevel", level), 3264 ) 3265 3266 return nil, pooled 3267 } 3268 3269 // Retry again by re-enqueuing, have not met consistency level yet 3270 m.fetchBlockFullRetry.Inc(1) 3271 3272 err := fmt.Errorf(errMsg+": attempts=%d", curr.block.reattempt.attempt) 3273 reattemptReason := consistencyLevelNotAchievedErrReason 3274 reattemptType := fullRetryReattemptType 3275 reattemptBlocks := []receivedBlockMetadata{curr} 3276 s.reattemptStreamBlocksFromPeersFn(reattemptBlocks, reEnqueueCh, 3277 err, reattemptReason, reattemptType, m) 3278 3279 return nil, pooled 3280 } 3281 3282 var ( 3283 singlePeer = len(currEligible) == 1 3284 sameNonNilChecksum = true 3285 curChecksum *uint32 3286 ) 3287 for i := range currEligible { 3288 // If any peer has a nil checksum, this might be the most recent block 3289 // and therefore not sealed so we want to merge from all peers 3290 if currEligible[i].block.checksum == nil { 3291 sameNonNilChecksum = false 3292 break 3293 } 3294 if curChecksum == nil { 3295 curChecksum = currEligible[i].block.checksum 3296 } else if *curChecksum != *currEligible[i].block.checksum { 3297 sameNonNilChecksum = false 3298 break 3299 } 3300 } 3301 3302 // If all the peers have the same non-nil checksum, we pick the peer with the 3303 // fewest attempts and fewest outstanding requests 3304 if singlePeer || sameNonNilChecksum { 3305 var idx int 3306 if singlePeer { 3307 idx = 0 3308 } else { 3309 pooledResources := pooled.pickBestPeerPooledResources 3310 idx, pooledResources = s.pickBestPeerFn(currEligible, peerQueues, 3311 pooledResources) 3312 pooled.pickBestPeerPooledResources = pooledResources 3313 } 3314 3315 // Set the reattempt metadata 3316 selected := currEligible[idx] 3317 selected.block.reattempt.attempt++ 3318 selected.block.reattempt.attempted = append(selected.block.reattempt.attempted, selected.peer) 3319 selected.block.reattempt.fanoutFetchState = nil 3320 selected.block.reattempt.retryPeersMetadata = perPeerBlocksMetadata 3321 selected.block.reattempt.fetchedPeersMetadata = perPeerBlocksMetadata 3322 3323 // Return just the single peer we selected 3324 currEligible = currEligible[:1] 3325 currEligible[0] = selected 3326 } else { 3327 fanoutFetchState := newBlockFanoutFetchState(len(currEligible)) 3328 for i := range currEligible { 3329 // Set the reattempt metadata 3330 // NB(xichen): each block will only be retried on the same peer because we 3331 // already fan out the request to all peers. This means we merge data on 3332 // a best-effort basis and only fail if we failed to reach the desired 3333 // consistency level when reading data from all peers. 3334 var retryFrom []receivedBlockMetadata 3335 for j := range perPeerBlocksMetadata { 3336 if currEligible[i].peer == perPeerBlocksMetadata[j].peer { 3337 // NB(r): Take a ref to a subslice from the originally passed 3338 // slice as that is not mutated, whereas currEligible is reused 3339 retryFrom = perPeerBlocksMetadata[j : j+1] 3340 } 3341 } 3342 currEligible[i].block.reattempt.attempt++ 3343 currEligible[i].block.reattempt.attempted = append(currEligible[i].block.reattempt.attempted, currEligible[i].peer) 3344 currEligible[i].block.reattempt.fanoutFetchState = fanoutFetchState 3345 currEligible[i].block.reattempt.retryPeersMetadata = retryFrom 3346 currEligible[i].block.reattempt.fetchedPeersMetadata = perPeerBlocksMetadata 3347 } 3348 } 3349 3350 return currEligible, pooled 3351 } 3352 3353 func (s *session) streamBlocksBatchFromPeer( 3354 namespaceMetadata namespace.Metadata, 3355 shard uint32, 3356 peer peer, 3357 batch []receivedBlockMetadata, 3358 opts result.Options, 3359 blocksResult blocksResult, 3360 enqueueCh enqueueChannel, 3361 retrier xretry.Retrier, 3362 m *streamFromPeersMetrics, 3363 ) { 3364 // Prepare request 3365 var ( 3366 req = rpc.NewFetchBlocksRawRequest() 3367 result *rpc.FetchBlocksRawResult_ 3368 reqBlocksLen uint 3369 3370 nowFn = opts.ClockOptions().NowFn() 3371 ropts = namespaceMetadata.Options().RetentionOptions() 3372 retention = ropts.RetentionPeriod() 3373 earliestBlockStart = xtime.ToUnixNano(nowFn()). 3374 Add(-retention). 3375 Truncate(ropts.BlockSize()) 3376 ) 3377 req.NameSpace = namespaceMetadata.ID().Bytes() 3378 req.Shard = int32(shard) 3379 req.Elements = make([]*rpc.FetchBlocksRawRequestElement, 0, len(batch)) 3380 for i := range batch { 3381 blockStart := batch[i].block.start 3382 if blockStart.Before(earliestBlockStart) { 3383 continue // Fell out of retention while we were streaming blocks 3384 } 3385 req.Elements = append(req.Elements, &rpc.FetchBlocksRawRequestElement{ 3386 ID: batch[i].id.Bytes(), 3387 Starts: []int64{int64(blockStart)}, 3388 }) 3389 reqBlocksLen++ 3390 } 3391 if reqBlocksLen == 0 { 3392 // All blocks fell out of retention while streaming 3393 return 3394 } 3395 3396 // Attempt request 3397 if err := retrier.Attempt(func() error { 3398 var attemptErr error 3399 borrowErr := peer.BorrowConnection(func(client rpc.TChanNode, _ Channel) { 3400 tctx, _ := thrift.NewContext(s.streamBlocksBatchTimeout) 3401 result, attemptErr = client.FetchBlocksRaw(tctx, req) 3402 }) 3403 err := xerrors.FirstError(borrowErr, attemptErr) 3404 return err 3405 }); err != nil { 3406 blocksErr := fmt.Errorf( 3407 "stream blocks request error: error=%s, peer=%s", 3408 err.Error(), peer.Host().String(), 3409 ) 3410 s.reattemptStreamBlocksFromPeersFn(batch, enqueueCh, blocksErr, 3411 reqErrReason, nextRetryReattemptType, m) 3412 m.fetchBlockError.Inc(int64(reqBlocksLen)) 3413 s.log.Debug(blocksErr.Error()) 3414 return 3415 } 3416 3417 // Parse and act on result 3418 tooManyIDsLogged := false 3419 for i := range result.Elements { 3420 if i >= len(batch) { 3421 m.fetchBlockError.Inc(int64(len(req.Elements[i].Starts))) 3422 m.fetchBlockFinalError.Inc(int64(len(req.Elements[i].Starts))) 3423 if !tooManyIDsLogged { 3424 tooManyIDsLogged = true 3425 s.log.Error("stream blocks more IDs than expected", 3426 zap.Stringer("peer", peer.Host()), 3427 ) 3428 } 3429 continue 3430 } 3431 3432 id := batch[i].id 3433 if !bytes.Equal(id.Bytes(), result.Elements[i].ID) { 3434 blocksErr := fmt.Errorf( 3435 "stream blocks mismatched ID: expectedID=%s, actualID=%s, indexID=%d, peer=%s", 3436 batch[i].id.String(), id.String(), i, peer.Host().String(), 3437 ) 3438 failed := []receivedBlockMetadata{batch[i]} 3439 s.reattemptStreamBlocksFromPeersFn(failed, enqueueCh, blocksErr, 3440 respErrReason, nextRetryReattemptType, m) 3441 m.fetchBlockError.Inc(int64(len(req.Elements[i].Starts))) 3442 s.log.Debug(blocksErr.Error()) 3443 continue 3444 } 3445 3446 if len(result.Elements[i].Blocks) == 0 { 3447 // If fell out of retention during request this is healthy, otherwise 3448 // missing blocks will be repaired during an active repair 3449 continue 3450 } 3451 3452 // We only ever fetch a single block for a series 3453 if len(result.Elements[i].Blocks) != 1 { 3454 errMsg := "stream blocks returned more blocks than expected" 3455 blocksErr := fmt.Errorf(errMsg+": expected=%d, actual=%d", 3456 1, len(result.Elements[i].Blocks)) 3457 failed := []receivedBlockMetadata{batch[i]} 3458 s.reattemptStreamBlocksFromPeersFn(failed, enqueueCh, blocksErr, 3459 respErrReason, nextRetryReattemptType, m) 3460 m.fetchBlockError.Inc(int64(len(req.Elements[i].Starts))) 3461 s.log.Error(errMsg, 3462 zap.Stringer("id", id), 3463 zap.Times("expectedStarts", newTimesByUnixNanos(req.Elements[i].Starts)), 3464 zap.Times("actualStarts", newTimesByRPCBlocks(result.Elements[i].Blocks)), 3465 zap.Stringer("peer", peer.Host()), 3466 ) 3467 continue 3468 } 3469 3470 for j, block := range result.Elements[i].Blocks { 3471 if block.Start != int64(batch[i].block.start) { 3472 errMsg := "stream blocks returned different blocks than expected" 3473 blocksErr := fmt.Errorf(errMsg+": expected=%s, actual=%d", 3474 batch[i].block.start.String(), time.Unix(0, block.Start).String()) 3475 failed := []receivedBlockMetadata{batch[i]} 3476 s.reattemptStreamBlocksFromPeersFn(failed, enqueueCh, blocksErr, 3477 respErrReason, nextRetryReattemptType, m) 3478 m.fetchBlockError.Inc(int64(len(req.Elements[i].Starts))) 3479 s.log.Error(errMsg, 3480 zap.Stringer("id", id), 3481 zap.Times("expectedStarts", newTimesByUnixNanos(req.Elements[i].Starts)), 3482 zap.Times("actualStarts", newTimesByRPCBlocks(result.Elements[i].Blocks)), 3483 zap.Stringer("peer", peer.Host()), 3484 ) 3485 continue 3486 } 3487 3488 // Verify and if verify succeeds add the block from the peer 3489 err := s.verifyFetchedBlock(block) 3490 if err == nil { 3491 err = blocksResult.addBlockFromPeer(id, batch[i].encodedTags, 3492 peer.Host(), block) 3493 } 3494 if err != nil { 3495 failed := []receivedBlockMetadata{batch[i]} 3496 blocksErr := fmt.Errorf( 3497 "stream blocks bad block: id=%s, start=%d, error=%s, indexID=%d, indexBlock=%d, peer=%s", 3498 id.String(), block.Start, err.Error(), i, j, peer.Host().String()) 3499 s.reattemptStreamBlocksFromPeersFn(failed, enqueueCh, blocksErr, 3500 respErrReason, nextRetryReattemptType, m) 3501 m.fetchBlockError.Inc(1) 3502 s.log.Debug(blocksErr.Error()) 3503 continue 3504 } 3505 3506 // NB(r): Track a fanned out block fetch success if added block 3507 fanout := batch[i].block.reattempt.fanoutFetchState 3508 if fanout != nil { 3509 fanout.incrementSuccess() 3510 } 3511 3512 m.fetchBlockSuccess.Inc(1) 3513 } 3514 } 3515 } 3516 3517 func (s *session) verifyFetchedBlock(block *rpc.Block) error { 3518 if block.Err != nil { 3519 return fmt.Errorf("block error from peer: %s %s", block.Err.Type.String(), block.Err.Message) 3520 } 3521 if block.Segments == nil { 3522 return fmt.Errorf("block segments is bad: segments is nil") 3523 } 3524 if block.Segments.Merged == nil && len(block.Segments.Unmerged) == 0 { 3525 return fmt.Errorf("block segments is bad: merged and unmerged not set") 3526 } 3527 3528 if checksum := block.Checksum; checksum != nil { 3529 var ( 3530 d = digest.NewDigest() 3531 expected = uint32(*checksum) 3532 ) 3533 if merged := block.Segments.Merged; merged != nil { 3534 d = d.Update(merged.Head).Update(merged.Tail) 3535 } else { 3536 for _, s := range block.Segments.Unmerged { 3537 d = d.Update(s.Head).Update(s.Tail) 3538 } 3539 } 3540 if actual := d.Sum32(); actual != expected { 3541 return fmt.Errorf("block checksum is bad: expected=%d, actual=%d", expected, actual) 3542 } 3543 } 3544 3545 return nil 3546 } 3547 3548 func (s *session) cloneFinalizable(id ident.ID) ident.ID { 3549 if id.IsNoFinalize() { 3550 return id 3551 } 3552 return s.pools.id.Clone(id) 3553 } 3554 3555 func (s *session) nsCtxFromMetadata(nsMeta namespace.Metadata) (namespace.Context, error) { 3556 nsCtx := namespace.NewContextFrom(nsMeta) 3557 if s.opts.IsSetEncodingProto() && nsCtx.Schema == nil { 3558 return nsCtx, fmt.Errorf("no protobuf schema found for namespace: %s", nsMeta.ID().String()) 3559 } 3560 return nsCtx, nil 3561 } 3562 3563 func (s *session) nsCtxFor(ns ident.ID) (namespace.Context, error) { 3564 nsCtx := namespace.NewContextFor(ns, s.opts.SchemaRegistry()) 3565 if s.opts.IsSetEncodingProto() && nsCtx.Schema == nil { 3566 return nsCtx, fmt.Errorf("no protobuf schema found for namespace: %s", ns.String()) 3567 } 3568 return nsCtx, nil 3569 } 3570 3571 type reason int 3572 3573 const ( 3574 reqErrReason reason = iota 3575 respErrReason 3576 consistencyLevelNotAchievedErrReason 3577 ) 3578 3579 type reattemptType int 3580 3581 const ( 3582 nextRetryReattemptType reattemptType = iota 3583 fullRetryReattemptType 3584 ) 3585 3586 type reattemptStreamBlocksFromPeersFn func( 3587 []receivedBlockMetadata, 3588 enqueueChannel, 3589 error, 3590 reason, 3591 reattemptType, 3592 *streamFromPeersMetrics, 3593 ) error 3594 3595 func (s *session) streamBlocksReattemptFromPeers( 3596 blocks []receivedBlockMetadata, 3597 enqueueCh enqueueChannel, 3598 attemptErr error, 3599 reason reason, 3600 reattemptType reattemptType, 3601 m *streamFromPeersMetrics, 3602 ) error { 3603 switch reason { 3604 case reqErrReason: 3605 m.fetchBlockRetriesReqError.Inc(int64(len(blocks))) 3606 case respErrReason: 3607 m.fetchBlockRetriesRespError.Inc(int64(len(blocks))) 3608 case consistencyLevelNotAchievedErrReason: 3609 m.fetchBlockRetriesConsistencyLevelNotAchievedError.Inc(int64(len(blocks))) 3610 } 3611 3612 // Must do this asynchronously or else could get into a deadlock scenario 3613 // where cannot enqueue into the reattempt channel because no more work is 3614 // getting done because new attempts are blocked on existing attempts completing 3615 // and existing attempts are trying to enqueue into a full reattempt channel 3616 enqueue, done, err := enqueueCh.enqueueDelayed(len(blocks)) 3617 if err != nil { 3618 return err 3619 } 3620 go s.streamBlocksReattemptFromPeersEnqueue(blocks, attemptErr, reattemptType, 3621 enqueue, done) 3622 return nil 3623 } 3624 3625 func (s *session) streamBlocksReattemptFromPeersEnqueue( 3626 blocks []receivedBlockMetadata, 3627 attemptErr error, 3628 reattemptType reattemptType, 3629 enqueueFn enqueueDelayedFn, 3630 enqueueDoneFn enqueueDelayedDoneFn, 3631 ) { 3632 // NB(r): Notify the delayed enqueue is done. 3633 defer enqueueDoneFn() 3634 3635 for i := range blocks { 3636 var reattemptPeersMetadata []receivedBlockMetadata 3637 switch reattemptType { 3638 case nextRetryReattemptType: 3639 reattemptPeersMetadata = blocks[i].block.reattempt.retryPeersMetadata 3640 case fullRetryReattemptType: 3641 reattemptPeersMetadata = blocks[i].block.reattempt.fetchedPeersMetadata 3642 } 3643 if len(reattemptPeersMetadata) == 0 { 3644 continue 3645 } 3646 3647 // Reconstruct peers metadata for reattempt 3648 reattemptBlocksMetadata := make([]receivedBlockMetadata, len(reattemptPeersMetadata)) 3649 for j := range reattemptPeersMetadata { 3650 var reattempt blockMetadataReattempt 3651 if reattemptType == nextRetryReattemptType { 3652 // Only if a default type of retry do we want to actually want 3653 // to set all the retry metadata, otherwise this re-enqueued metadata 3654 // should start fresh 3655 reattempt = blocks[i].block.reattempt 3656 3657 // Copy the errors for every peer so they don't shard the same error 3658 // slice and therefore are not subject to race conditions when the 3659 // error slice is modified 3660 reattemptErrs := make([]error, len(reattempt.errs)+1) 3661 n := copy(reattemptErrs, reattempt.errs) 3662 reattemptErrs[n] = attemptErr 3663 reattempt.errs = reattemptErrs 3664 } 3665 3666 reattemptBlocksMetadata[j] = receivedBlockMetadata{ 3667 peer: reattemptPeersMetadata[j].peer, 3668 id: blocks[i].id, 3669 block: blockMetadata{ 3670 start: reattemptPeersMetadata[j].block.start, 3671 size: reattemptPeersMetadata[j].block.size, 3672 checksum: reattemptPeersMetadata[j].block.checksum, 3673 reattempt: reattempt, 3674 }, 3675 } 3676 } 3677 3678 // Re-enqueue the block to be fetched from all peers requested 3679 // to reattempt from 3680 enqueueFn(reattemptBlocksMetadata) 3681 } 3682 } 3683 3684 type blocksResult interface { 3685 addBlockFromPeer( 3686 id ident.ID, 3687 encodedTags checked.Bytes, 3688 peer topology.Host, 3689 block *rpc.Block, 3690 ) error 3691 } 3692 3693 type baseBlocksResult struct { 3694 nsCtx namespace.Context 3695 blockOpts block.Options 3696 blockAllocSize int 3697 contextPool context.Pool 3698 encoderPool encoding.EncoderPool 3699 multiReaderIteratorPool encoding.MultiReaderIteratorPool 3700 } 3701 3702 func newBaseBlocksResult( 3703 nsCtx namespace.Context, 3704 opts Options, 3705 resultOpts result.Options, 3706 ) baseBlocksResult { 3707 blockOpts := resultOpts.DatabaseBlockOptions() 3708 return baseBlocksResult{ 3709 nsCtx: nsCtx, 3710 blockOpts: blockOpts, 3711 blockAllocSize: blockOpts.DatabaseBlockAllocSize(), 3712 contextPool: opts.ContextPool(), 3713 encoderPool: blockOpts.EncoderPool(), 3714 multiReaderIteratorPool: blockOpts.MultiReaderIteratorPool(), 3715 } 3716 } 3717 3718 func (b *baseBlocksResult) segmentForBlock(seg *rpc.Segment) ts.Segment { 3719 var ( 3720 bytesPool = b.blockOpts.BytesPool() 3721 head, tail checked.Bytes 3722 ) 3723 if len(seg.Head) > 0 { 3724 head = bytesPool.Get(len(seg.Head)) 3725 head.IncRef() 3726 head.AppendAll(seg.Head) 3727 head.DecRef() 3728 } 3729 if len(seg.Tail) > 0 { 3730 tail = bytesPool.Get(len(seg.Tail)) 3731 tail.IncRef() 3732 tail.AppendAll(seg.Tail) 3733 tail.DecRef() 3734 } 3735 var checksum uint32 3736 if seg.Checksum != nil { 3737 checksum = uint32(*seg.Checksum) 3738 } 3739 3740 return ts.NewSegment(head, tail, checksum, ts.FinalizeHead&ts.FinalizeTail) 3741 } 3742 3743 func (b *baseBlocksResult) mergeReaders( 3744 start xtime.UnixNano, blockSize time.Duration, readers []xio.SegmentReader, 3745 ) (encoding.Encoder, error) { 3746 iter := b.multiReaderIteratorPool.Get() 3747 iter.Reset(readers, start, blockSize, b.nsCtx.Schema) 3748 defer iter.Close() 3749 3750 encoder := b.encoderPool.Get() 3751 encoder.Reset(start, b.blockAllocSize, b.nsCtx.Schema) 3752 3753 for iter.Next() { 3754 dp, unit, annotation := iter.Current() 3755 if err := encoder.Encode(dp, unit, annotation); err != nil { 3756 encoder.Close() 3757 return nil, err 3758 } 3759 } 3760 if err := iter.Err(); err != nil { 3761 encoder.Close() 3762 return nil, err 3763 } 3764 3765 return encoder, nil 3766 } 3767 3768 func (b *baseBlocksResult) newDatabaseBlock(block *rpc.Block) (block.DatabaseBlock, error) { 3769 var ( 3770 start = xtime.UnixNano(block.Start) 3771 segments = block.Segments 3772 result = b.blockOpts.DatabaseBlockPool().Get() 3773 ) 3774 3775 if segments == nil { 3776 result.Close() // return block to pool 3777 return nil, errSessionBadBlockResultFromPeer 3778 } 3779 3780 switch { 3781 case segments.Merged != nil: 3782 // Unmerged, can insert directly into a single block 3783 mergedBlock := segments.Merged 3784 result.Reset( 3785 start, 3786 durationConvert(mergedBlock.BlockSize), 3787 b.segmentForBlock(mergedBlock), 3788 b.nsCtx, 3789 ) 3790 3791 case segments.Unmerged != nil: 3792 // Must merge to provide a single block 3793 segmentReaderPool := b.blockOpts.SegmentReaderPool() 3794 readers := make([]xio.SegmentReader, len(segments.Unmerged)) 3795 3796 blockSize := time.Duration(0) 3797 for i, seg := range segments.Unmerged { 3798 segmentReader := segmentReaderPool.Get() 3799 segmentReader.Reset(b.segmentForBlock(seg)) 3800 readers[i] = segmentReader 3801 3802 bs := durationConvert(seg.BlockSize) 3803 if bs > blockSize { 3804 blockSize = bs 3805 } 3806 } 3807 encoder, err := b.mergeReaders(start, blockSize, readers) 3808 for _, reader := range readers { 3809 // Close each reader 3810 reader.Finalize() 3811 } 3812 3813 if err != nil { 3814 // mergeReaders(...) already calls encoder.Close() upon error 3815 result.Close() // return block to pool 3816 return nil, err 3817 } 3818 3819 // Set the block data 3820 result.Reset(start, blockSize, encoder.Discard(), b.nsCtx) 3821 3822 default: 3823 result.Close() // return block to pool 3824 return nil, errSessionBadBlockResultFromPeer 3825 } 3826 3827 return result, nil 3828 } 3829 3830 // Ensure streamBlocksResult implements blocksResult 3831 var _ blocksResult = (*streamBlocksResult)(nil) 3832 3833 type streamBlocksResult struct { 3834 baseBlocksResult 3835 outputCh chan<- peerBlocksDatapoint 3836 tagDecoderPool serialize.TagDecoderPool 3837 idPool ident.Pool 3838 nsCtx namespace.Context 3839 } 3840 3841 func newStreamBlocksResult( 3842 nsCtx namespace.Context, 3843 opts Options, 3844 resultOpts result.Options, 3845 outputCh chan<- peerBlocksDatapoint, 3846 tagDecoderPool serialize.TagDecoderPool, 3847 idPool ident.Pool, 3848 ) *streamBlocksResult { 3849 return &streamBlocksResult{ 3850 nsCtx: nsCtx, 3851 baseBlocksResult: newBaseBlocksResult(nsCtx, opts, resultOpts), 3852 outputCh: outputCh, 3853 tagDecoderPool: tagDecoderPool, 3854 idPool: idPool, 3855 } 3856 } 3857 3858 type peerBlocksDatapoint struct { 3859 id ident.ID 3860 tags ident.Tags 3861 peer topology.Host 3862 block block.DatabaseBlock 3863 } 3864 3865 func (s *streamBlocksResult) addBlockFromPeer( 3866 id ident.ID, 3867 encodedTags checked.Bytes, 3868 peer topology.Host, 3869 block *rpc.Block, 3870 ) error { 3871 result, err := s.newDatabaseBlock(block) 3872 if err != nil { 3873 return err 3874 } 3875 tags, err := newTagsFromEncodedTags(id, encodedTags, 3876 s.tagDecoderPool, s.idPool) 3877 if err != nil { 3878 return err 3879 } 3880 s.outputCh <- peerBlocksDatapoint{ 3881 id: id, 3882 tags: tags, 3883 peer: peer, 3884 block: result, 3885 } 3886 return nil 3887 } 3888 3889 type peerBlocksIter struct { 3890 inputCh <-chan peerBlocksDatapoint 3891 errCh <-chan error 3892 current peerBlocksDatapoint 3893 err error 3894 done bool 3895 } 3896 3897 func newPeerBlocksIter( 3898 inputC <-chan peerBlocksDatapoint, 3899 errC <-chan error, 3900 ) *peerBlocksIter { 3901 return &peerBlocksIter{ 3902 inputCh: inputC, 3903 errCh: errC, 3904 } 3905 } 3906 3907 func (it *peerBlocksIter) Current() (topology.Host, ident.ID, ident.Tags, block.DatabaseBlock) { 3908 return it.current.peer, it.current.id, it.current.tags, it.current.block 3909 } 3910 3911 func (it *peerBlocksIter) Err() error { 3912 return it.err 3913 } 3914 3915 func (it *peerBlocksIter) Next() bool { 3916 if it.done || it.err != nil { 3917 return false 3918 } 3919 m, more := <-it.inputCh 3920 3921 if !more { 3922 it.err = <-it.errCh 3923 it.done = true 3924 return false 3925 } 3926 3927 it.current = m 3928 return true 3929 } 3930 3931 // Ensure streamBlocksResult implements blocksResult 3932 var _ blocksResult = (*bulkBlocksResult)(nil) 3933 3934 type bulkBlocksResult struct { 3935 sync.RWMutex 3936 baseBlocksResult 3937 result result.ShardResult 3938 tagDecoderPool serialize.TagDecoderPool 3939 idPool ident.Pool 3940 nsCtx namespace.Context 3941 } 3942 3943 func newBulkBlocksResult( 3944 nsCtx namespace.Context, 3945 opts Options, 3946 resultOpts result.Options, 3947 tagDecoderPool serialize.TagDecoderPool, 3948 idPool ident.Pool, 3949 ) *bulkBlocksResult { 3950 return &bulkBlocksResult{ 3951 nsCtx: nsCtx, 3952 baseBlocksResult: newBaseBlocksResult(nsCtx, opts, resultOpts), 3953 result: result.NewShardResult(resultOpts), 3954 tagDecoderPool: tagDecoderPool, 3955 idPool: idPool, 3956 } 3957 } 3958 3959 func (r *bulkBlocksResult) addBlockFromPeer( 3960 id ident.ID, 3961 encodedTags checked.Bytes, 3962 peer topology.Host, 3963 block *rpc.Block, 3964 ) error { 3965 start := xtime.UnixNano(block.Start) 3966 result, err := r.newDatabaseBlock(block) 3967 if err != nil { 3968 return err 3969 } 3970 3971 var ( 3972 tags ident.Tags 3973 attemptedDecodeTags bool 3974 ) 3975 for { 3976 r.Lock() 3977 currBlock, exists := r.result.BlockAt(id, start) 3978 if !exists { 3979 if encodedTags == nil || attemptedDecodeTags { 3980 r.result.AddBlock(id, tags, result) 3981 r.Unlock() 3982 break 3983 } 3984 r.Unlock() 3985 3986 // Tags not decoded yet, attempt decoded and then reinsert 3987 attemptedDecodeTags = true 3988 tags, err = newTagsFromEncodedTags(id, encodedTags, 3989 r.tagDecoderPool, r.idPool) 3990 if err != nil { 3991 return err 3992 } 3993 continue 3994 } 3995 3996 // Remove the existing block from the result so it doesn't get 3997 // merged again 3998 r.result.RemoveBlockAt(id, start) 3999 r.Unlock() 4000 4001 // If we've already received data for this block, merge them 4002 // with the new block if possible 4003 tmpCtx := r.contextPool.Get() 4004 currReader, err := currBlock.Stream(tmpCtx) 4005 if err != nil { 4006 return err 4007 } 4008 4009 // If there are no data in the current block, there is no 4010 // need to merge 4011 if currReader.IsEmpty() { 4012 continue 4013 } 4014 4015 resultReader, err := result.Stream(tmpCtx) 4016 if err != nil { 4017 return err 4018 } 4019 if resultReader.IsEmpty() { 4020 return nil 4021 } 4022 4023 readers := []xio.SegmentReader{currReader.SegmentReader, resultReader.SegmentReader} 4024 blockSize := currReader.BlockSize 4025 4026 encoder, err := r.mergeReaders(start, blockSize, readers) 4027 if err != nil { 4028 return err 4029 } 4030 4031 result.Close() 4032 4033 result = r.blockOpts.DatabaseBlockPool().Get() 4034 result.Reset(start, blockSize, encoder.Discard(), r.nsCtx) 4035 4036 tmpCtx.Close() 4037 } 4038 4039 return nil 4040 } 4041 4042 type enqueueCh struct { 4043 sync.Mutex 4044 sending int 4045 enqueued int 4046 processed int 4047 peersMetadataCh chan []receivedBlockMetadata 4048 closed bool 4049 enqueueDelayedFn enqueueDelayedFn 4050 enqueueDelayedDoneFn enqueueDelayedDoneFn 4051 metrics *streamFromPeersMetrics 4052 } 4053 4054 // enqueueChannelDefaultLen is the queue length for processing series ready to 4055 // be fetched from other peers. 4056 // It was reduced from 32k to 512 since each struct in the queue is quite large 4057 // and with 32k capacity was using significant memory with high shard 4058 // concurrency. 4059 const enqueueChannelDefaultLen = 512 4060 4061 func newEnqueueChannel(m *streamFromPeersMetrics) enqueueChannel { 4062 c := &enqueueCh{ 4063 peersMetadataCh: make(chan []receivedBlockMetadata, enqueueChannelDefaultLen), 4064 metrics: m, 4065 } 4066 4067 // Allocate the enqueue delayed fn just once 4068 c.enqueueDelayedFn = func(peersMetadata []receivedBlockMetadata) { 4069 c.peersMetadataCh <- peersMetadata 4070 } 4071 c.enqueueDelayedDoneFn = func() { 4072 c.Lock() 4073 c.sending-- 4074 c.Unlock() 4075 } 4076 4077 go func() { 4078 for { 4079 c.Lock() 4080 closed := c.closed 4081 numEnqueued := float64(len(c.peersMetadataCh)) 4082 c.Unlock() 4083 if closed { 4084 return 4085 } 4086 m.blocksEnqueueChannel.Update(numEnqueued) 4087 time.Sleep(gaugeReportInterval) 4088 } 4089 }() 4090 return c 4091 } 4092 4093 func (c *enqueueCh) enqueue(peersMetadata []receivedBlockMetadata) error { 4094 c.Lock() 4095 if c.closed { 4096 c.Unlock() 4097 return errEnqueueChIsClosed 4098 } 4099 c.enqueued++ 4100 c.sending++ 4101 c.Unlock() 4102 c.peersMetadataCh <- peersMetadata 4103 c.Lock() 4104 c.sending-- 4105 c.Unlock() 4106 return nil 4107 } 4108 4109 func (c *enqueueCh) enqueueDelayed(numToEnqueue int) (enqueueDelayedFn, enqueueDelayedDoneFn, error) { 4110 c.Lock() 4111 if c.closed { 4112 c.Unlock() 4113 return nil, nil, errEnqueueChIsClosed 4114 } 4115 c.sending++ // NB(r): This is decremented by calling the returned enqueue done function 4116 c.enqueued += numToEnqueue 4117 c.Unlock() 4118 return c.enqueueDelayedFn, c.enqueueDelayedDoneFn, nil 4119 } 4120 4121 // read is always safe to call since you can safely range 4122 // over a closed channel, and/or do a checked read in case 4123 // it is closed (unlike when publishing to a channel). 4124 func (c *enqueueCh) read() <-chan []receivedBlockMetadata { 4125 return c.peersMetadataCh 4126 } 4127 4128 func (c *enqueueCh) trackPending(amount int) { 4129 c.Lock() 4130 c.enqueued += amount 4131 c.Unlock() 4132 } 4133 4134 func (c *enqueueCh) trackProcessed(amount int) { 4135 c.Lock() 4136 c.processed += amount 4137 c.Unlock() 4138 } 4139 4140 func (c *enqueueCh) unprocessedLen() int { 4141 c.Lock() 4142 unprocessed := c.unprocessedLenWithLock() 4143 c.Unlock() 4144 return unprocessed 4145 } 4146 4147 func (c *enqueueCh) unprocessedLenWithLock() int { 4148 return c.enqueued - c.processed 4149 } 4150 4151 func (c *enqueueCh) closeOnAllProcessed() { 4152 for { 4153 c.Lock() 4154 if c.unprocessedLenWithLock() == 0 && c.sending == 0 { 4155 close(c.peersMetadataCh) 4156 c.closed = true 4157 c.Unlock() 4158 return 4159 } 4160 c.Unlock() 4161 time.Sleep(100 * time.Millisecond) 4162 } 4163 } 4164 4165 type receivedBlocks struct { 4166 enqueued bool 4167 results []receivedBlockMetadata 4168 } 4169 4170 type processFn func(batch []receivedBlockMetadata) 4171 4172 // peerBlocksQueue is a per peer queue of blocks to be retrieved from a peer 4173 type peerBlocksQueue struct { 4174 sync.RWMutex 4175 closed bool 4176 peer peer 4177 queue []receivedBlockMetadata 4178 doneFns []func() 4179 assigned uint64 4180 completed uint64 4181 maxQueueSize int 4182 workers xsync.WorkerPool 4183 processFn processFn 4184 } 4185 4186 type newPeerBlocksQueueFn func( 4187 peer peer, 4188 maxQueueSize int, 4189 interval time.Duration, 4190 workers xsync.WorkerPool, 4191 processFn processFn, 4192 ) *peerBlocksQueue 4193 4194 func newPeerBlocksQueue( 4195 peer peer, 4196 maxQueueSize int, 4197 interval time.Duration, 4198 workers xsync.WorkerPool, 4199 processFn processFn, 4200 ) *peerBlocksQueue { 4201 q := &peerBlocksQueue{ 4202 peer: peer, 4203 maxQueueSize: maxQueueSize, 4204 workers: workers, 4205 processFn: processFn, 4206 } 4207 if interval > 0 { 4208 go q.drainEvery(interval) 4209 } 4210 return q 4211 } 4212 4213 func (q *peerBlocksQueue) drainEvery(interval time.Duration) { 4214 for { 4215 q.Lock() 4216 if q.closed { 4217 q.Unlock() 4218 return 4219 } 4220 q.drainWithLock() 4221 q.Unlock() 4222 time.Sleep(interval) 4223 } 4224 } 4225 4226 func (q *peerBlocksQueue) close() { 4227 q.Lock() 4228 defer q.Unlock() 4229 q.closed = true 4230 } 4231 4232 func (q *peerBlocksQueue) trackAssigned(amount int) { 4233 atomic.AddUint64(&q.assigned, uint64(amount)) 4234 } 4235 4236 func (q *peerBlocksQueue) trackCompleted(amount int) { 4237 atomic.AddUint64(&q.completed, uint64(amount)) 4238 } 4239 4240 func (q *peerBlocksQueue) enqueue(bl receivedBlockMetadata, doneFn func()) { 4241 q.Lock() 4242 4243 if len(q.queue) == 0 && cap(q.queue) < q.maxQueueSize { 4244 // Lazy initialize queue 4245 q.queue = make([]receivedBlockMetadata, 0, q.maxQueueSize) 4246 } 4247 if len(q.doneFns) == 0 && cap(q.doneFns) < q.maxQueueSize { 4248 // Lazy initialize doneFns 4249 q.doneFns = make([]func(), 0, q.maxQueueSize) 4250 } 4251 q.queue = append(q.queue, bl) 4252 if doneFn != nil { 4253 q.doneFns = append(q.doneFns, doneFn) 4254 } 4255 q.trackAssigned(1) 4256 4257 // Determine if should drain immediately 4258 if len(q.queue) < q.maxQueueSize { 4259 // Require more to fill up block 4260 q.Unlock() 4261 return 4262 } 4263 q.drainWithLock() 4264 4265 q.Unlock() 4266 } 4267 4268 func (q *peerBlocksQueue) drain() { 4269 q.Lock() 4270 q.drainWithLock() 4271 q.Unlock() 4272 } 4273 4274 func (q *peerBlocksQueue) drainWithLock() { 4275 if len(q.queue) == 0 { 4276 // None to drain 4277 return 4278 } 4279 enqueued := q.queue 4280 doneFns := q.doneFns 4281 q.queue = nil 4282 q.doneFns = nil 4283 q.workers.Go(func() { 4284 q.processFn(enqueued) 4285 // Call done callbacks 4286 for i := range doneFns { 4287 doneFns[i]() 4288 } 4289 // Track completed blocks 4290 q.trackCompleted(len(enqueued)) 4291 }) 4292 } 4293 4294 type peerBlocksQueues []*peerBlocksQueue 4295 4296 func (qs peerBlocksQueues) findQueue(peer peer) *peerBlocksQueue { 4297 for _, q := range qs { 4298 if q.peer == peer { 4299 return q 4300 } 4301 } 4302 return nil 4303 } 4304 4305 func (qs peerBlocksQueues) closeAll() { 4306 for _, q := range qs { 4307 q.close() 4308 } 4309 } 4310 4311 type receivedBlockMetadata struct { 4312 peer peer 4313 id ident.ID 4314 encodedTags checked.Bytes 4315 block blockMetadata 4316 } 4317 4318 type receivedBlockMetadatas []receivedBlockMetadata 4319 4320 func (arr receivedBlockMetadatas) swap(i, j int) { arr[i], arr[j] = arr[j], arr[i] } 4321 4322 type peerBlockMetadataByID []receivedBlockMetadata 4323 4324 func (arr peerBlockMetadataByID) Len() int { return len(arr) } 4325 func (arr peerBlockMetadataByID) Swap(i, j int) { arr[i], arr[j] = arr[j], arr[i] } 4326 func (arr peerBlockMetadataByID) Less(i, j int) bool { 4327 return strings.Compare(arr[i].peer.Host().ID(), arr[j].peer.Host().ID()) < 0 4328 } 4329 4330 type receivedBlockMetadataQueue struct { 4331 blockMetadata receivedBlockMetadata 4332 queue *peerBlocksQueue 4333 } 4334 4335 type receivedBlockMetadataQueuesByAttemptsAscOutstandingAsc []receivedBlockMetadataQueue 4336 4337 func (arr receivedBlockMetadataQueuesByAttemptsAscOutstandingAsc) Len() int { 4338 return len(arr) 4339 } 4340 4341 func (arr receivedBlockMetadataQueuesByAttemptsAscOutstandingAsc) Swap(i, j int) { 4342 arr[i], arr[j] = arr[j], arr[i] 4343 } 4344 4345 func (arr receivedBlockMetadataQueuesByAttemptsAscOutstandingAsc) Less(i, j int) bool { 4346 peerI := arr[i].queue.peer 4347 peerJ := arr[j].queue.peer 4348 attemptsI := arr[i].blockMetadata.block.reattempt.peerAttempts(peerI) 4349 attemptsJ := arr[j].blockMetadata.block.reattempt.peerAttempts(peerJ) 4350 if attemptsI != attemptsJ { 4351 return attemptsI < attemptsJ 4352 } 4353 4354 outstandingI := atomic.LoadUint64(&arr[i].queue.assigned) - 4355 atomic.LoadUint64(&arr[i].queue.completed) 4356 outstandingJ := atomic.LoadUint64(&arr[j].queue.assigned) - 4357 atomic.LoadUint64(&arr[j].queue.completed) 4358 return outstandingI < outstandingJ 4359 } 4360 4361 type blockMetadata struct { 4362 start xtime.UnixNano 4363 size int64 4364 checksum *uint32 4365 lastRead xtime.UnixNano 4366 reattempt blockMetadataReattempt 4367 } 4368 4369 type blockMetadataReattempt struct { 4370 attempt int 4371 fanoutFetchState *blockFanoutFetchState 4372 attempted []peer 4373 errs []error 4374 retryPeersMetadata []receivedBlockMetadata 4375 fetchedPeersMetadata []receivedBlockMetadata 4376 } 4377 4378 type blockFanoutFetchState struct { 4379 numPending int32 4380 numSuccess int32 4381 } 4382 4383 func newBlockFanoutFetchState( 4384 pending int, 4385 ) *blockFanoutFetchState { 4386 return &blockFanoutFetchState{ 4387 numPending: int32(pending), 4388 } 4389 } 4390 4391 func (s *blockFanoutFetchState) success() int { 4392 return int(atomic.LoadInt32(&s.numSuccess)) 4393 } 4394 4395 func (s *blockFanoutFetchState) incrementSuccess() { 4396 atomic.AddInt32(&s.numSuccess, 1) 4397 } 4398 4399 func (s *blockFanoutFetchState) decrementAndReturnPending() int { 4400 return int(atomic.AddInt32(&s.numPending, -1)) 4401 } 4402 4403 func (b blockMetadataReattempt) peerAttempts(p peer) int { 4404 r := 0 4405 for i := range b.attempted { 4406 if b.attempted[i] == p { 4407 r++ 4408 } 4409 } 4410 return r 4411 } 4412 4413 func newTimesByUnixNanos(values []int64) []time.Time { 4414 result := make([]time.Time, len(values)) 4415 for i := range values { 4416 result[i] = time.Unix(0, values[i]) 4417 } 4418 return result 4419 } 4420 4421 func newTimesByRPCBlocks(values []*rpc.Block) []time.Time { 4422 result := make([]time.Time, len(values)) 4423 for i := range values { 4424 result[i] = time.Unix(0, values[i].Start) 4425 } 4426 return result 4427 } 4428 4429 type metadataIter struct { 4430 inputCh <-chan receivedBlockMetadata 4431 errCh <-chan error 4432 host topology.Host 4433 metadata block.Metadata 4434 tagDecoderPool serialize.TagDecoderPool 4435 idPool ident.Pool 4436 done bool 4437 err error 4438 } 4439 4440 func newMetadataIter( 4441 inputCh <-chan receivedBlockMetadata, 4442 errCh <-chan error, 4443 tagDecoderPool serialize.TagDecoderPool, 4444 idPool ident.Pool, 4445 ) PeerBlockMetadataIter { 4446 return &metadataIter{ 4447 inputCh: inputCh, 4448 errCh: errCh, 4449 tagDecoderPool: tagDecoderPool, 4450 idPool: idPool, 4451 } 4452 } 4453 4454 func (it *metadataIter) Next() bool { 4455 if it.done || it.err != nil { 4456 return false 4457 } 4458 m, more := <-it.inputCh 4459 if !more { 4460 it.err = <-it.errCh 4461 it.done = true 4462 return false 4463 } 4464 var tags ident.Tags 4465 tags, it.err = newTagsFromEncodedTags(m.id, m.encodedTags, 4466 it.tagDecoderPool, it.idPool) 4467 if it.err != nil { 4468 return false 4469 } 4470 it.host = m.peer.Host() 4471 it.metadata = block.NewMetadata(m.id, tags, m.block.start, 4472 m.block.size, m.block.checksum, m.block.lastRead) 4473 return true 4474 } 4475 4476 func (it *metadataIter) Current() (topology.Host, block.Metadata) { 4477 return it.host, it.metadata 4478 } 4479 4480 func (it *metadataIter) Err() error { 4481 return it.err 4482 } 4483 4484 type idAndBlockStart struct { 4485 id ident.ID 4486 blockStart int64 4487 } 4488 4489 func newTagsFromEncodedTags( 4490 seriesID ident.ID, 4491 encodedTags checked.Bytes, 4492 tagDecoderPool serialize.TagDecoderPool, 4493 idPool ident.Pool, 4494 ) (ident.Tags, error) { 4495 if encodedTags == nil { 4496 return ident.Tags{}, nil 4497 } 4498 4499 encodedTags.IncRef() 4500 4501 tagDecoder := tagDecoderPool.Get() 4502 tagDecoder.Reset(encodedTags) 4503 defer tagDecoder.Close() 4504 4505 tags, err := idxconvert.TagsFromTagsIter(seriesID, tagDecoder, idPool) 4506 4507 encodedTags.DecRef() 4508 4509 return tags, err 4510 } 4511 4512 const ( 4513 // histogramDurationBucketsVersion must be bumped if histogramDurationBuckets is changed 4514 // to namespace the different buckets from each other so they don't overlap and cause the 4515 // histogram function to error out due to overlapping buckets in the same query. 4516 histogramDurationBucketsVersion = "v1" 4517 // histogramDurationBucketsVersionTag is the tag for the version of the buckets in use. 4518 histogramDurationBucketsVersionTag = "schema" 4519 ) 4520 4521 // histogramDurationBuckets is a high resolution set of duration buckets. 4522 func histogramDurationBuckets() tally.DurationBuckets { 4523 return append(tally.DurationBuckets{0}, 4524 tally.MustMakeExponentialDurationBuckets(time.Millisecond, 1.25, 60)...) 4525 } 4526 4527 // histogramWithDurationBuckets returns a histogram with the standard duration buckets. 4528 func histogramWithDurationBuckets(scope tally.Scope, name string) tally.Histogram { 4529 sub := scope.Tagged(map[string]string{ 4530 histogramDurationBucketsVersionTag: histogramDurationBucketsVersion, 4531 }) 4532 return sub.Histogram(name, histogramDurationBuckets()) 4533 } 4534 4535 func minDuration(x, y time.Duration) time.Duration { 4536 if x < y { 4537 return x 4538 } 4539 return y 4540 }