github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/index/block.go (about) 1 // Copyright (c) 2018 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package index 22 23 import ( 24 "bytes" 25 "errors" 26 "fmt" 27 "io" 28 "math" 29 "runtime" 30 "sync" 31 "time" 32 33 "github.com/m3db/m3/src/dbnode/namespace" 34 "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" 35 "github.com/m3db/m3/src/dbnode/storage/limits" 36 "github.com/m3db/m3/src/dbnode/tracepoint" 37 "github.com/m3db/m3/src/m3ninx/doc" 38 m3ninxindex "github.com/m3db/m3/src/m3ninx/index" 39 "github.com/m3db/m3/src/m3ninx/index/segment" 40 "github.com/m3db/m3/src/m3ninx/index/segment/fst" 41 "github.com/m3db/m3/src/m3ninx/persist" 42 "github.com/m3db/m3/src/m3ninx/search" 43 "github.com/m3db/m3/src/m3ninx/search/executor" 44 "github.com/m3db/m3/src/x/context" 45 xerrors "github.com/m3db/m3/src/x/errors" 46 "github.com/m3db/m3/src/x/ident" 47 "github.com/m3db/m3/src/x/instrument" 48 xresource "github.com/m3db/m3/src/x/resource" 49 xsync "github.com/m3db/m3/src/x/sync" 50 xtime "github.com/m3db/m3/src/x/time" 51 52 opentracinglog "github.com/opentracing/opentracing-go/log" 53 "github.com/uber-go/tally" 54 "go.uber.org/zap" 55 ) 56 57 var ( 58 // ErrUnableToQueryBlockClosed is returned when querying closed block. 59 ErrUnableToQueryBlockClosed = errors.New("unable to query, index block is closed") 60 // ErrUnableReportStatsBlockClosed is returned from Stats when the block is closed. 61 ErrUnableReportStatsBlockClosed = errors.New("unable to report stats, block is closed") 62 63 errUnableToWriteBlockClosed = errors.New("unable to write, index block is closed") 64 errUnableToWriteBlockSealed = errors.New("unable to write, index block is sealed") 65 errUnableToBootstrapBlockClosed = errors.New("unable to bootstrap, block is closed") 66 errUnableToTickBlockClosed = errors.New("unable to tick, block is closed") 67 errBlockAlreadyClosed = errors.New("unable to close, block already closed") 68 69 errUnableToSealBlockIllegalStateFmtString = "unable to seal, index block state: %v" 70 errUnableToWriteBlockUnknownStateFmtString = "unable to write, unknown index block state: %v" 71 ) 72 73 type blockState uint 74 75 const ( 76 blockStateOpen blockState = iota 77 blockStateSealed 78 blockStateClosed 79 80 defaultQueryDocsBatchSize = 256 81 defaultAggregateResultsEntryBatchSize = 256 82 83 compactDebugLogEvery = 1 // Emit debug log for every compaction 84 85 mmapIndexBlockName = "mmap.index.block" 86 ) 87 88 func (s blockState) String() string { 89 switch s { 90 case blockStateOpen: 91 return "open" 92 case blockStateSealed: 93 return "sealed" 94 case blockStateClosed: 95 return "closed" 96 } 97 return "unknown" 98 } 99 100 type newExecutorFn func() (search.Executor, error) 101 102 type shardRangesSegmentsByVolumeType map[persist.IndexVolumeType][]blockShardRangesSegments 103 104 func (s shardRangesSegmentsByVolumeType) forEachSegment(cb func(segment segment.Segment) error) error { 105 return s.forEachSegmentGroup(func(group blockShardRangesSegments) error { 106 for _, seg := range group.segments { 107 if err := cb(seg); err != nil { 108 return err 109 } 110 } 111 return nil 112 }) 113 } 114 115 func (s shardRangesSegmentsByVolumeType) forEachSegmentGroup(cb func(group blockShardRangesSegments) error) error { 116 for _, shardRangesSegments := range s { 117 for _, group := range shardRangesSegments { 118 if err := cb(group); err != nil { 119 return err 120 } 121 } 122 } 123 return nil 124 } 125 126 type addAggregateResultsFn func( 127 ctx context.Context, 128 results AggregateResults, 129 batch []AggregateResultsEntry, 130 source []byte, 131 ) ([]AggregateResultsEntry, int, int, error) 132 133 // nolint: maligned 134 type block struct { 135 sync.RWMutex 136 137 state blockState 138 139 cachedSearchesWorkers xsync.WorkerPool 140 141 mutableSegments *mutableSegments 142 coldMutableSegments []*mutableSegments 143 shardRangesSegmentsByVolumeType shardRangesSegmentsByVolumeType 144 newFieldsAndTermsIteratorFn newFieldsAndTermsIteratorFn 145 newExecutorWithRLockFn newExecutorFn 146 addAggregateResultsFn addAggregateResultsFn 147 blockStart xtime.UnixNano 148 blockEnd xtime.UnixNano 149 blockSize time.Duration 150 opts Options 151 iopts instrument.Options 152 blockOpts BlockOptions 153 nsMD namespace.Metadata 154 namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager 155 fetchDocsLimit limits.LookbackLimit 156 aggDocsLimit limits.LookbackLimit 157 158 metrics blockMetrics 159 logger *zap.Logger 160 } 161 162 type blockMetrics struct { 163 rotateActiveSegment tally.Counter 164 rotateActiveSegmentAge tally.Timer 165 rotateActiveSegmentSize tally.Histogram 166 segmentFreeMmapSuccess tally.Counter 167 segmentFreeMmapError tally.Counter 168 segmentFreeMmapSkipNotImmutable tally.Counter 169 querySeriesMatched tally.Histogram 170 queryDocsMatched tally.Histogram 171 aggregateSeriesMatched tally.Histogram 172 aggregateDocsMatched tally.Histogram 173 entryReconciledOnQuery tally.Counter 174 entryUnreconciledOnQuery tally.Counter 175 } 176 177 func newBlockMetrics(s tally.Scope) blockMetrics { 178 segmentFreeMmap := "segment-free-mmap" 179 buckets := append(tally.ValueBuckets{0}, tally.MustMakeExponentialValueBuckets(100, 2, 16)...) 180 return blockMetrics{ 181 rotateActiveSegment: s.Counter("rotate-active-segment"), 182 rotateActiveSegmentAge: s.Timer("rotate-active-segment-age"), 183 rotateActiveSegmentSize: s.Histogram("rotate-active-segment-size", buckets), 184 segmentFreeMmapSuccess: s.Tagged(map[string]string{ 185 "result": "success", 186 "skip_type": "none", 187 }).Counter(segmentFreeMmap), 188 segmentFreeMmapError: s.Tagged(map[string]string{ 189 "result": "error", 190 "skip_type": "none", 191 }).Counter(segmentFreeMmap), 192 segmentFreeMmapSkipNotImmutable: s.Tagged(map[string]string{ 193 "result": "skip", 194 "skip_type": "not-immutable", 195 }).Counter(segmentFreeMmap), 196 197 querySeriesMatched: s.Histogram("query-series-matched", buckets), 198 queryDocsMatched: s.Histogram("query-docs-matched", buckets), 199 aggregateSeriesMatched: s.Histogram("aggregate-series-matched", buckets), 200 aggregateDocsMatched: s.Histogram("aggregate-docs-matched", buckets), 201 entryReconciledOnQuery: s.Counter("entry-reconciled-on-query"), 202 entryUnreconciledOnQuery: s.Counter("entry-unreconciled-on-query"), 203 } 204 } 205 206 // blockShardsSegments is a collection of segments that has a mapping of what shards 207 // and time ranges they completely cover, this can only ever come from computing 208 // from data that has come from shards, either on an index flush or a bootstrap. 209 type blockShardRangesSegments struct { 210 shardTimeRanges result.ShardTimeRanges 211 segments []segment.Segment 212 } 213 214 // BlockOptions is a set of options used when constructing an index block. 215 type BlockOptions struct { 216 ForegroundCompactorMmapDocsData bool 217 BackgroundCompactorMmapDocsData bool 218 ActiveBlock bool 219 } 220 221 // NewBlockFn is a new block constructor. 222 type NewBlockFn func( 223 blockStart xtime.UnixNano, 224 md namespace.Metadata, 225 blockOpts BlockOptions, 226 namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, 227 opts Options, 228 ) (Block, error) 229 230 // Ensure NewBlock implements NewBlockFn. 231 var _ NewBlockFn = NewBlock 232 233 // NewBlock returns a new Block, representing a complete reverse index for the 234 // duration of time specified. It is backed by one or more segments. 235 func NewBlock( 236 blockStart xtime.UnixNano, 237 md namespace.Metadata, 238 blockOpts BlockOptions, 239 namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, 240 opts Options, 241 ) (Block, error) { 242 blockSize := md.Options().IndexOptions().BlockSize() 243 iopts := opts.InstrumentOptions() 244 scope := iopts.MetricsScope().SubScope("index").SubScope("block") 245 iopts = iopts.SetMetricsScope(scope) 246 247 cpus := int(math.Max(1, math.Ceil(0.25*float64(runtime.GOMAXPROCS(0))))) 248 cachedSearchesWorkers := xsync.NewWorkerPool(cpus) 249 cachedSearchesWorkers.Init() 250 251 segs := newMutableSegments( 252 md, 253 blockStart, 254 opts, 255 blockOpts, 256 cachedSearchesWorkers, 257 namespaceRuntimeOptsMgr, 258 iopts, 259 ) 260 261 coldSegs := newMutableSegments( 262 md, 263 blockStart, 264 opts, 265 blockOpts, 266 cachedSearchesWorkers, 267 namespaceRuntimeOptsMgr, 268 iopts, 269 ) 270 271 // NB(bodu): The length of coldMutableSegments is always at least 1. 272 coldMutableSegments := []*mutableSegments{coldSegs} 273 b := &block{ 274 state: blockStateOpen, 275 blockStart: blockStart, 276 blockEnd: blockStart.Add(blockSize), 277 blockSize: blockSize, 278 blockOpts: blockOpts, 279 cachedSearchesWorkers: cachedSearchesWorkers, 280 mutableSegments: segs, 281 coldMutableSegments: coldMutableSegments, 282 shardRangesSegmentsByVolumeType: make(shardRangesSegmentsByVolumeType), 283 opts: opts, 284 iopts: iopts, 285 nsMD: md, 286 namespaceRuntimeOptsMgr: namespaceRuntimeOptsMgr, 287 metrics: newBlockMetrics(scope), 288 logger: iopts.Logger(), 289 fetchDocsLimit: opts.QueryLimits().FetchDocsLimit(), 290 aggDocsLimit: opts.QueryLimits().AggregateDocsLimit(), 291 } 292 b.newFieldsAndTermsIteratorFn = newFieldsAndTermsIterator 293 b.newExecutorWithRLockFn = b.executorWithRLock 294 b.addAggregateResultsFn = b.addAggregateResults 295 296 return b, nil 297 } 298 299 func (b *block) StartTime() xtime.UnixNano { 300 return b.blockStart 301 } 302 303 func (b *block) EndTime() xtime.UnixNano { 304 return b.blockEnd 305 } 306 307 // BackgroundCompact background compacts eligible segments. 308 func (b *block) BackgroundCompact() { 309 b.mutableSegments.BackgroundCompact() 310 } 311 312 func (b *block) WriteBatch(inserts *WriteBatch) (WriteBatchResult, error) { 313 b.RLock() 314 if !b.writesAcceptedWithRLock() { 315 b.RUnlock() 316 return b.writeBatchResult(inserts, MutableSegmentsStats{}, 317 b.writeBatchErrorInvalidState(b.state)) 318 } 319 if b.state == blockStateSealed { 320 coldBlock := b.coldMutableSegments[len(b.coldMutableSegments)-1] 321 b.RUnlock() 322 _, err := coldBlock.WriteBatch(inserts) 323 // Don't pass stats back from insertion into a cold block, 324 // we only care about warm mutable segments stats. 325 return b.writeBatchResult(inserts, MutableSegmentsStats{}, err) 326 } 327 b.RUnlock() 328 stats, err := b.mutableSegments.WriteBatch(inserts) 329 return b.writeBatchResult(inserts, stats, err) 330 } 331 332 func (b *block) writeBatchResult( 333 inserts *WriteBatch, 334 stats MutableSegmentsStats, 335 err error, 336 ) (WriteBatchResult, error) { 337 if err == nil { 338 inserts.MarkUnmarkedEntriesSuccess() 339 return WriteBatchResult{ 340 NumSuccess: int64(inserts.Len()), 341 MutableSegmentsStats: stats, 342 }, nil 343 } 344 345 partialErr, ok := err.(*m3ninxindex.BatchPartialError) 346 if !ok { 347 // NB: marking all the inserts as failure, cause we don't know which ones failed. 348 inserts.MarkUnmarkedEntriesError(err) 349 return WriteBatchResult{ 350 NumError: int64(inserts.Len()), 351 MutableSegmentsStats: stats, 352 }, err 353 } 354 355 numErr := len(partialErr.Errs()) 356 for _, err := range partialErr.Errs() { 357 // Avoid marking these as success. 358 inserts.MarkUnmarkedEntryError(err.Err, err.Idx) 359 } 360 361 // Mark all non-error inserts success, so we don't repeatedly index them. 362 inserts.MarkUnmarkedEntriesSuccess() 363 return WriteBatchResult{ 364 NumSuccess: int64(inserts.Len() - numErr), 365 NumError: int64(numErr), 366 MutableSegmentsStats: stats, 367 }, partialErr 368 } 369 370 func (b *block) writesAcceptedWithRLock() bool { 371 if b.state == blockStateOpen { 372 return true 373 } 374 return b.state == blockStateSealed && 375 b.nsMD.Options().ColdWritesEnabled() 376 } 377 378 func (b *block) executorWithRLock() (search.Executor, error) { 379 readers, err := b.segmentReadersWithRLock() 380 if err != nil { 381 return nil, err 382 } 383 384 indexReaders := make([]m3ninxindex.Reader, 0, len(readers)) 385 for _, r := range readers { 386 indexReaders = append(indexReaders, r) 387 } 388 389 return executor.NewExecutor(indexReaders), nil 390 } 391 392 func (b *block) segmentReadersWithRLock() ([]segment.Reader, error) { 393 expectedReaders := b.mutableSegments.Len() 394 for _, coldSeg := range b.coldMutableSegments { 395 expectedReaders += coldSeg.Len() 396 } 397 b.shardRangesSegmentsByVolumeType.forEachSegmentGroup(func(group blockShardRangesSegments) error { 398 expectedReaders += len(group.segments) 399 return nil 400 }) 401 402 var ( 403 readers = make([]segment.Reader, 0, expectedReaders) 404 success = false 405 err error 406 ) 407 defer func() { 408 // Cleanup in case any of the readers below fail. 409 if !success { 410 for _, reader := range readers { 411 reader.Close() 412 } 413 } 414 }() 415 416 // Add mutable segments. 417 readers, err = b.mutableSegments.AddReaders(readers) 418 if err != nil { 419 return nil, err 420 } 421 422 // Add cold mutable segments. 423 for _, coldSeg := range b.coldMutableSegments { 424 readers, err = coldSeg.AddReaders(readers) 425 if err != nil { 426 return nil, err 427 } 428 } 429 430 // Loop over the segments associated to shard time ranges. 431 if err := b.shardRangesSegmentsByVolumeType.forEachSegment(func(seg segment.Segment) error { 432 reader, err := seg.Reader() 433 if err != nil { 434 return err 435 } 436 readers = append(readers, reader) 437 return nil 438 }); err != nil { 439 return nil, err 440 } 441 442 success = true 443 return readers, nil 444 } 445 446 // QueryIter acquires a read lock on the block to get the set of segments for the returned iterator. However, the 447 // segments are searched and results are processed lazily in the returned iterator. The segments are finalized when 448 // the ctx is finalized to ensure the mmaps are not freed until the ctx closes. This allows the returned results to 449 // reference data in the mmap without copying. 450 func (b *block) QueryIter(ctx context.Context, query Query) (QueryIterator, error) { 451 b.RLock() 452 defer b.RUnlock() 453 454 if b.state == blockStateClosed { 455 return nil, ErrUnableToQueryBlockClosed 456 } 457 exec, err := b.newExecutorWithRLockFn() 458 if err != nil { 459 return nil, err 460 } 461 462 // FOLLOWUP(prateek): push down QueryOptions to restrict results 463 docIter, err := exec.Execute(ctx, query.Query.SearchQuery()) 464 if err != nil { 465 b.closeAsync(exec) 466 return nil, err 467 } 468 469 // Register the executor to close when context closes 470 // so can avoid copying the results into the map and just take 471 // references to it. 472 ctx.RegisterFinalizer(xresource.FinalizerFn(func() { 473 b.closeAsync(exec) 474 })) 475 476 return NewQueryIter(docIter), nil 477 } 478 479 // nolint: dupl 480 func (b *block) QueryWithIter( 481 ctx context.Context, 482 opts QueryOptions, 483 iter QueryIterator, 484 results DocumentResults, 485 deadline time.Time, 486 logFields []opentracinglog.Field, 487 ) error { 488 ctx, sp := ctx.StartTraceSpan(tracepoint.BlockQuery) 489 sp.LogFields(logFields...) 490 defer sp.Finish() 491 492 err := b.queryWithSpan(ctx, opts, iter, results, deadline) 493 if err != nil { 494 sp.LogFields(opentracinglog.Error(err)) 495 } 496 if iter.Done() { 497 docs, series := iter.Counts() 498 b.metrics.queryDocsMatched.RecordValue(float64(docs)) 499 b.metrics.querySeriesMatched.RecordValue(float64(series)) 500 } 501 return err 502 } 503 504 func (b *block) queryWithSpan( 505 ctx context.Context, 506 opts QueryOptions, 507 iter QueryIterator, 508 results DocumentResults, 509 deadline time.Time, 510 ) error { 511 var ( 512 err error 513 source = opts.Source 514 sizeBefore = results.Size() 515 docsCountBefore = results.TotalDocsCount() 516 size = sizeBefore 517 docsCount = docsCountBefore 518 docsPool = b.opts.DocumentArrayPool() 519 batch = docsPool.Get() 520 batchSize = cap(batch) 521 ) 522 if batchSize == 0 { 523 batchSize = defaultQueryDocsBatchSize 524 } 525 526 // Register local data structures that need closing. 527 defer docsPool.Put(batch) 528 529 for time.Now().Before(deadline) && iter.Next(ctx) { 530 if opts.LimitsExceeded(size, docsCount) { 531 break 532 } 533 534 // the caller (nsIndex) has canceled this before the query has timed out. 535 // only check once per batch to limit the overhead. worst case nsIndex will need to wait for an additional batch 536 // to be processed after the query timeout. we check when the batch is empty to cover 2 cases, the initial doc 537 // when includes the search time, and subsequent batch resets. 538 if len(batch) == 0 { 539 select { 540 case <-ctx.GoContext().Done(): 541 // indexNs will log something useful. 542 return ctx.GoContext().Err() 543 default: 544 } 545 } 546 547 // Ensure that the block contains any of the relevant time segments for the query range. 548 doc := iter.Current() 549 if !b.docWithinQueryRange(doc, opts) { 550 continue 551 } 552 553 batch = append(batch, doc) 554 if len(batch) < batchSize { 555 continue 556 } 557 558 batch, size, docsCount, err = b.addQueryResults(ctx, results, batch, source) 559 if err != nil { 560 return err 561 } 562 } 563 if err := iter.Err(); err != nil { 564 return err 565 } 566 567 // Add last batch to results if remaining. 568 if len(batch) > 0 { 569 batch, size, docsCount, err = b.addQueryResults(ctx, results, batch, source) 570 if err != nil { 571 return err 572 } 573 } 574 575 iter.AddSeries(size - sizeBefore) 576 iter.AddDocs(docsCount - docsCountBefore) 577 578 return nil 579 } 580 581 func (b *block) docWithinQueryRange(doc doc.Document, opts QueryOptions) bool { 582 md, ok := doc.Metadata() 583 if !ok || md.OnIndexSeries == nil { 584 return true 585 } 586 587 onIndexSeries, closer, reconciled := md.OnIndexSeries.ReconciledOnIndexSeries() 588 if reconciled { 589 b.metrics.entryReconciledOnQuery.Inc(1) 590 } else { 591 b.metrics.entryUnreconciledOnQuery.Inc(1) 592 } 593 594 defer closer.Close() 595 596 var ( 597 inBlock bool 598 currentBlock = opts.StartInclusive.Truncate(b.blockSize) 599 endExclusive = opts.EndExclusive 600 minIndexed, maxIndexed = onIndexSeries.IndexedRange() 601 ) 602 if maxIndexed == 0 { 603 // Empty range. 604 return false 605 } 606 607 // Narrow down the range of blocks to scan because the client could have 608 // queried for an arbitrary wide range. 609 if currentBlock.Before(minIndexed) { 610 currentBlock = minIndexed 611 } 612 maxIndexedExclusive := maxIndexed.Add(time.Nanosecond) 613 if endExclusive.After(maxIndexedExclusive) { 614 endExclusive = maxIndexedExclusive 615 } 616 617 for !inBlock && currentBlock.Before(endExclusive) { 618 inBlock = onIndexSeries.IndexedForBlockStart(currentBlock) 619 currentBlock = currentBlock.Add(b.blockSize) 620 } 621 622 return inBlock 623 } 624 625 func (b *block) closeAsync(closer io.Closer) { 626 if err := closer.Close(); err != nil { 627 // Note: This only happens if closing the readers isn't clean. 628 instrument.EmitAndLogInvariantViolation( 629 b.iopts, 630 func(l *zap.Logger) { 631 l.Error("could not close query index block resource", zap.Error(err)) 632 }) 633 } 634 } 635 636 func (b *block) addQueryResults( 637 ctx context.Context, 638 results DocumentResults, 639 batch []doc.Document, 640 source []byte, 641 ) ([]doc.Document, int, int, error) { 642 // update recently queried docs to monitor memory. 643 if results.EnforceLimits() { 644 if err := b.fetchDocsLimit.Inc(len(batch), source); err != nil { 645 return batch, 0, 0, err 646 } 647 } 648 649 _, sp := ctx.StartTraceSpan(tracepoint.NSIdxBlockQueryAddDocuments) 650 defer sp.Finish() 651 // try to add the docs to the resource. 652 size, docsCount, err := results.AddDocuments(batch) 653 654 // reset batch. 655 var emptyDoc doc.Document 656 for i := range batch { 657 batch[i] = emptyDoc 658 } 659 batch = batch[:0] 660 661 // return results. 662 return batch, size, docsCount, err 663 } 664 665 // AggregateIter acquires a read lock on the block to get the set of segments for the returned iterator. However, the 666 // segments are searched and results are processed lazily in the returned iterator. The segments are finalized when 667 // the ctx is finalized to ensure the mmaps are not freed until the ctx closes. This allows the returned results to 668 // reference data in the mmap without copying. 669 func (b *block) AggregateIter(ctx context.Context, aggOpts AggregateResultsOptions) (AggregateIterator, error) { 670 b.RLock() 671 defer b.RUnlock() 672 673 if b.state == blockStateClosed { 674 return nil, ErrUnableToQueryBlockClosed 675 } 676 677 iterateOpts := fieldsAndTermsIteratorOpts{ 678 restrictByQuery: aggOpts.RestrictByQuery, 679 iterateTerms: aggOpts.Type == AggregateTagNamesAndValues, 680 allowFn: func(field []byte) bool { 681 // skip any field names that we shouldn't allow. 682 if bytes.Equal(field, doc.IDReservedFieldName) { 683 return false 684 } 685 return aggOpts.FieldFilter.Allow(field) 686 }, 687 fieldIterFn: func(r segment.Reader) (segment.FieldsPostingsListIterator, error) { 688 // NB(prateek): we default to using the regular (FST) fields iterator 689 // unless we have a predefined list of fields we know we need to restrict 690 // our search to, in which case we iterate that list and check if known values 691 // in the FST to restrict our search. This is going to be significantly faster 692 // while len(FieldsFilter) < 5-10 elements; 693 // but there will exist a ratio between the len(FieldFilter) v size(FST) after which 694 // iterating the entire FST is faster. 695 // Here, we chose to avoid factoring that in to our choice because almost all input 696 // to this function is expected to have (FieldsFilter) pretty small. If that changes 697 // in the future, we can revisit this. 698 if len(aggOpts.FieldFilter) == 0 { 699 return r.FieldsPostingsList() 700 } 701 return newFilterFieldsIterator(r, aggOpts.FieldFilter) 702 }, 703 } 704 readers, err := b.segmentReadersWithRLock() 705 if err != nil { 706 return nil, err 707 } 708 // Make sure to close readers at end of query since results can 709 // include references to the underlying bytes from the index segment 710 // read by the readers. 711 for _, reader := range readers { 712 reader := reader // Capture for inline function. 713 ctx.RegisterFinalizer(xresource.FinalizerFn(func() { 714 b.closeAsync(reader) 715 })) 716 } 717 718 return &aggregateIter{ 719 readers: readers, 720 iterateOpts: iterateOpts, 721 newIterFn: b.newFieldsAndTermsIteratorFn, 722 }, nil 723 } 724 725 // nolint: dupl 726 func (b *block) AggregateWithIter( 727 ctx context.Context, 728 iter AggregateIterator, 729 opts QueryOptions, 730 results AggregateResults, 731 deadline time.Time, 732 logFields []opentracinglog.Field, 733 ) error { 734 ctx, sp := ctx.StartTraceSpan(tracepoint.BlockAggregate) 735 sp.LogFields(logFields...) 736 defer sp.Finish() 737 738 err := b.aggregateWithSpan(ctx, iter, opts, results, deadline) 739 if err != nil { 740 sp.LogFields(opentracinglog.Error(err)) 741 } 742 if iter.Done() { 743 docs, series := iter.Counts() 744 b.metrics.aggregateDocsMatched.RecordValue(float64(docs)) 745 b.metrics.aggregateSeriesMatched.RecordValue(float64(series)) 746 } 747 748 return err 749 } 750 751 func (b *block) aggregateWithSpan( 752 ctx context.Context, 753 iter AggregateIterator, 754 opts QueryOptions, 755 results AggregateResults, 756 deadline time.Time, 757 ) error { 758 var ( 759 err error 760 source = opts.Source 761 size = results.Size() 762 docsCount = results.TotalDocsCount() 763 batch = b.opts.AggregateResultsEntryArrayPool().Get() 764 maxBatch = cap(batch) 765 fieldAppended bool 766 termAppended bool 767 lastField []byte 768 batchedFields int 769 currFields int 770 currTerms int 771 ) 772 if maxBatch == 0 { 773 maxBatch = defaultAggregateResultsEntryBatchSize 774 } 775 776 // cleanup at the end 777 defer b.opts.AggregateResultsEntryArrayPool().Put(batch) 778 779 if opts.SeriesLimit > 0 && opts.SeriesLimit < maxBatch { 780 maxBatch = opts.SeriesLimit 781 } 782 783 if opts.DocsLimit > 0 && opts.DocsLimit < maxBatch { 784 maxBatch = opts.DocsLimit 785 } 786 787 for time.Now().Before(deadline) && iter.Next(ctx) { 788 if opts.LimitsExceeded(size, docsCount) { 789 break 790 } 791 792 // the caller (nsIndex) has canceled this before the query has timed out. 793 // only check once per batch to limit the overhead. worst case nsIndex will need to wait for an additional 794 // batch to be processed after the query timeout. we check when the batch is empty to cover 2 cases, the 795 // initial result when includes the search time, and subsequent batch resets. 796 if len(batch) == 0 { 797 select { 798 case <-ctx.GoContext().Done(): 799 return ctx.GoContext().Err() 800 default: 801 } 802 } 803 804 field, term := iter.Current() 805 806 // TODO: remove this legacy doc tracking implementation when alternative 807 // limits are in place. 808 if results.EnforceLimits() { 809 if lastField == nil { 810 lastField = append(lastField, field...) 811 batchedFields++ 812 if err := b.fetchDocsLimit.Inc(1, source); err != nil { 813 return err 814 } 815 } else if !bytes.Equal(lastField, field) { 816 lastField = lastField[:0] 817 lastField = append(lastField, field...) 818 batchedFields++ 819 if err := b.fetchDocsLimit.Inc(1, source); err != nil { 820 return err 821 } 822 } 823 824 // NB: this logic increments the doc count to account for where the 825 // legacy limits would have been updated. It increments by two to 826 // reflect the term appearing as both the last element of the previous 827 // batch, as well as the first element in the next batch. 828 if batchedFields > maxBatch { 829 if err := b.fetchDocsLimit.Inc(2, source); err != nil { 830 return err 831 } 832 833 batchedFields = 1 834 } 835 } 836 837 batch, fieldAppended, termAppended = b.appendFieldAndTermToBatch(batch, field, term, 838 iter.fieldsAndTermsIteratorOpts().iterateTerms) 839 if fieldAppended { 840 currFields++ 841 } 842 if termAppended { 843 currTerms++ 844 } 845 // continue appending to the batch until we hit our max batch size. 846 if currFields+currTerms < maxBatch { 847 continue 848 } 849 850 batch, size, docsCount, err = b.addAggregateResultsFn(ctx, results, batch, source) 851 if err != nil { 852 return err 853 } 854 855 currFields = 0 856 currTerms = 0 857 } 858 859 if err := iter.Err(); err != nil { 860 return err 861 } 862 863 // Add last batch to results if remaining. 864 for len(batch) > 0 { 865 batch, size, docsCount, err = b.addAggregateResultsFn(ctx, results, batch, source) 866 if err != nil { 867 return err 868 } 869 } 870 871 iter.AddSeries(size) 872 iter.AddDocs(docsCount) 873 874 return nil 875 } 876 877 // appendFieldAndTermToBatch adds the provided field / term onto the batch, 878 // optionally reusing the last element of the batch if it pertains to the same field. 879 // First boolean result indicates that a unique field was added to the batch 880 // and the second boolean indicates if a unique term was added. 881 func (b *block) appendFieldAndTermToBatch( 882 batch []AggregateResultsEntry, 883 field, term []byte, 884 includeTerms bool, 885 ) ([]AggregateResultsEntry, bool, bool) { 886 // NB(prateek): we make a copy of the (field, term) entries returned 887 // by the iterator during traversal, because the []byte are only valid per entry during 888 // the traversal (i.e. calling Next() invalidates the []byte). We choose to do this 889 // instead of checking if the entry is required (duplicates may exist in the results map 890 // already), as it reduces contention on the map itself. Further, the ownership of these 891 // idents is transferred to the results map, which either hangs on to them (if they are new), 892 // or finalizes them if they are duplicates. 893 var ( 894 entry AggregateResultsEntry 895 lastField []byte 896 lastFieldIsValid bool 897 reuseLastEntry bool 898 newFieldAdded, newTermAdded bool 899 ) 900 // we are iterating multiple segments so we may receive duplicates (same field/term), but 901 // as we are iterating one segment at a time, and because the underlying index structures 902 // are FSTs, we rely on the fact that iterator traversal is in order to avoid creating duplicate 903 // entries for the same fields, by checking the last batch entry to see if the bytes are 904 // the same. 905 // It's easier to consider an example, say we have a segment with fields/terms: 906 // (f1, t1), (f1, t2), ..., (fn, t1), ..., (fn, tn) 907 // as we iterate in order, we receive (f1, t1) and then (f1, t2) we can avoid the repeated f1 908 // allocation if the previous entry has the same value. 909 // NB: this isn't strictly true because when we switch iterating between segments, 910 // the fields/terms switch in an order which doesn't have to be strictly lexicographic. In that 911 // instance however, the only downside is we would be allocating more. i.e. this is just an 912 // optimisation, it doesn't affect correctness. 913 if len(batch) > 0 { 914 lastFieldIsValid = true 915 lastField = batch[len(batch)-1].Field.Bytes() 916 } 917 if lastFieldIsValid && bytes.Equal(lastField, field) { 918 reuseLastEntry = true 919 entry = batch[len(batch)-1] // avoid alloc cause we already have the field 920 } else { 921 newFieldAdded = true 922 // allocate id because this is the first time we've seen it 923 // NB(r): Iterating fields FST, this byte slice is only temporarily available 924 // since we are pushing/popping characters from the stack as we iterate 925 // the fields FST and reusing the same byte slice. 926 entry.Field = b.pooledID(field) 927 } 928 929 if includeTerms { 930 newTermAdded = true 931 // terms are always new (as far we know without checking the map for duplicates), so we allocate 932 // NB(r): Iterating terms FST, this byte slice is only temporarily available 933 // since we are pushing/popping characters from the stack as we iterate 934 // the terms FST and reusing the same byte slice. 935 entry.Terms = append(entry.Terms, b.pooledID(term)) 936 } 937 938 if reuseLastEntry { 939 batch[len(batch)-1] = entry 940 } else { 941 batch = append(batch, entry) 942 } 943 944 return batch, newFieldAdded, newTermAdded 945 } 946 947 func (b *block) pooledID(id []byte) ident.ID { 948 data := b.opts.CheckedBytesPool().Get(len(id)) 949 data.IncRef() 950 data.AppendAll(id) 951 data.DecRef() 952 return b.opts.IdentifierPool().BinaryID(data) 953 } 954 955 // addAggregateResults adds the fields on the batch 956 // to the provided results and resets the batch to be reused. 957 func (b *block) addAggregateResults( 958 ctx context.Context, 959 results AggregateResults, 960 batch []AggregateResultsEntry, 961 source []byte, 962 ) ([]AggregateResultsEntry, int, int, error) { 963 _, sp := ctx.StartTraceSpan(tracepoint.NSIdxBlockAggregateQueryAddDocuments) 964 defer sp.Finish() 965 // try to add the docs to the resource. 966 size, docsCount := results.AddFields(batch) 967 968 aggDocs := len(batch) 969 for i := range batch { 970 aggDocs += len(batch[i].Terms) 971 } 972 973 // update recently queried docs to monitor memory. 974 if results.EnforceLimits() { 975 if err := b.aggDocsLimit.Inc(aggDocs, source); err != nil { 976 return batch, 0, 0, err 977 } 978 } 979 980 // reset batch. 981 var emptyField AggregateResultsEntry 982 for i := range batch { 983 batch[i] = emptyField 984 } 985 batch = batch[:0] 986 987 // return results. 988 return batch, size, docsCount, nil 989 } 990 991 func (b *block) AddResults( 992 resultsByVolumeType result.IndexBlockByVolumeType, 993 ) error { 994 b.Lock() 995 defer b.Unlock() 996 997 multiErr := xerrors.NewMultiError() 998 for volumeType, results := range resultsByVolumeType.Iter() { 999 multiErr = multiErr.Add(b.addResults(volumeType, results)) 1000 } 1001 1002 return multiErr.FinalError() 1003 } 1004 1005 func (b *block) addResults( 1006 volumeType persist.IndexVolumeType, 1007 results result.IndexBlock, 1008 ) error { 1009 // NB(prateek): we have to allow bootstrap to succeed even if we're Sealed because 1010 // of topology changes. i.e. if the current m3db process is assigned new shards, 1011 // we need to include their data in the index. 1012 1013 // i.e. the only state we do not accept bootstrapped data is if we are closed. 1014 if b.state == blockStateClosed { 1015 return errUnableToBootstrapBlockClosed 1016 } 1017 1018 // First check fulfilled is correct 1019 min, max := results.Fulfilled().MinMax() 1020 if min.Before(b.blockStart) || max.After(b.blockEnd) { 1021 blockRange := xtime.Range{Start: b.blockStart, End: b.blockEnd} 1022 return fmt.Errorf("fulfilled range %s is outside of index block range: %s", 1023 results.Fulfilled().SummaryString(), blockRange.String()) 1024 } 1025 1026 shardRangesSegments, ok := b.shardRangesSegmentsByVolumeType[volumeType] 1027 if !ok { 1028 shardRangesSegments = make([]blockShardRangesSegments, 0) 1029 b.shardRangesSegmentsByVolumeType[volumeType] = shardRangesSegments 1030 } 1031 1032 var ( 1033 plCaches = ReadThroughSegmentCaches{ 1034 SegmentPostingsListCache: b.opts.PostingsListCache(), 1035 SearchPostingsListCache: b.opts.SearchPostingsListCache(), 1036 } 1037 readThroughOpts = b.opts.ReadThroughSegmentOptions() 1038 segments = results.Segments() 1039 ) 1040 readThroughSegments := make([]segment.Segment, 0, len(segments)) 1041 for _, seg := range segments { 1042 elem := seg.Segment() 1043 if immSeg, ok := elem.(segment.ImmutableSegment); ok { 1044 // only wrap the immutable segments with a read through cache. 1045 elem = NewReadThroughSegment(immSeg, plCaches, readThroughOpts) 1046 } 1047 readThroughSegments = append(readThroughSegments, elem) 1048 } 1049 1050 entry := blockShardRangesSegments{ 1051 shardTimeRanges: results.Fulfilled(), 1052 segments: readThroughSegments, 1053 } 1054 1055 // first see if this block can cover all our current blocks covering shard 1056 // time ranges. 1057 currFulfilled := result.NewShardTimeRanges() 1058 for _, existing := range shardRangesSegments { 1059 currFulfilled.AddRanges(existing.shardTimeRanges) 1060 } 1061 1062 unfulfilledBySegments := currFulfilled.Copy() 1063 unfulfilledBySegments.Subtract(results.Fulfilled()) 1064 if !unfulfilledBySegments.IsEmpty() { 1065 // This is the case where it cannot wholly replace the current set of blocks 1066 // so simply append the segments in this case. 1067 b.shardRangesSegmentsByVolumeType[volumeType] = append(shardRangesSegments, entry) 1068 return nil 1069 } 1070 1071 // This is the case where the new segments can wholly replace the 1072 // current set of blocks since unfullfilled by the new segments is zero. 1073 multiErr := xerrors.NewMultiError() 1074 for i, group := range shardRangesSegments { 1075 for _, seg := range group.segments { 1076 // Make sure to close the existing segments. 1077 multiErr = multiErr.Add(seg.Close()) 1078 } 1079 shardRangesSegments[i] = blockShardRangesSegments{} 1080 } 1081 b.shardRangesSegmentsByVolumeType[volumeType] = append(shardRangesSegments[:0], entry) 1082 1083 return multiErr.FinalError() 1084 } 1085 1086 func (b *block) Tick(c context.Cancellable) (BlockTickResult, error) { 1087 b.Lock() 1088 defer b.Unlock() 1089 result := BlockTickResult{} 1090 if b.state == blockStateClosed { 1091 return result, errUnableToTickBlockClosed 1092 } 1093 1094 // Add foreground/background segments. 1095 numSegments, numDocs := b.mutableSegments.NumSegmentsAndDocs() 1096 for _, coldSeg := range b.coldMutableSegments { 1097 coldNumSegments, coldNumDocs := coldSeg.NumSegmentsAndDocs() 1098 numSegments += coldNumSegments 1099 numDocs += coldNumDocs 1100 } 1101 result.NumSegments += numSegments 1102 result.NumSegmentsMutable += numSegments 1103 result.NumDocs += numDocs 1104 1105 multiErr := xerrors.NewMultiError() 1106 1107 // Any segments covering persisted shard ranges. 1108 b.shardRangesSegmentsByVolumeType.forEachSegment(func(seg segment.Segment) error { 1109 result.NumSegments++ 1110 result.NumSegmentsBootstrapped++ 1111 result.NumDocs += seg.Size() 1112 1113 immSeg, ok := seg.(segment.ImmutableSegment) 1114 if !ok { 1115 b.metrics.segmentFreeMmapSkipNotImmutable.Inc(1) 1116 return nil 1117 } 1118 1119 // TODO(bodu): Revist this and implement a more sophisticated free strategy. 1120 if err := immSeg.FreeMmap(); err != nil { 1121 multiErr = multiErr.Add(err) 1122 b.metrics.segmentFreeMmapError.Inc(1) 1123 return nil 1124 } 1125 1126 result.FreeMmap++ 1127 b.metrics.segmentFreeMmapSuccess.Inc(1) 1128 return nil 1129 }) 1130 1131 return result, multiErr.FinalError() 1132 } 1133 1134 func (b *block) Seal() error { 1135 b.Lock() 1136 defer b.Unlock() 1137 1138 // Ensure we only Seal if we're marked Open. 1139 if b.state != blockStateOpen { 1140 return fmt.Errorf(errUnableToSealBlockIllegalStateFmtString, b.state) 1141 } 1142 b.state = blockStateSealed 1143 1144 // All foreground/background segments and added mutable segments can't 1145 // be written to and they don't need to be sealed since we don't flush 1146 // these segments. 1147 return nil 1148 } 1149 1150 func (b *block) Stats(reporter BlockStatsReporter) error { 1151 b.RLock() 1152 defer b.RUnlock() 1153 1154 if b.state != blockStateOpen { 1155 return ErrUnableReportStatsBlockClosed 1156 } 1157 1158 b.mutableSegments.Stats(reporter) 1159 for _, coldSeg := range b.coldMutableSegments { 1160 // TODO(bodu): Cold segment stats should prob be of a 1161 // diff type or something. 1162 coldSeg.Stats(reporter) 1163 } 1164 1165 b.shardRangesSegmentsByVolumeType.forEachSegment(func(seg segment.Segment) error { 1166 _, mutable := seg.(segment.MutableSegment) 1167 reporter.ReportSegmentStats(BlockSegmentStats{ 1168 Type: FlushedSegment, 1169 Mutable: mutable, 1170 Size: seg.Size(), 1171 }) 1172 return nil 1173 }) 1174 return nil 1175 } 1176 1177 func (b *block) IsOpen() bool { 1178 b.RLock() 1179 defer b.RUnlock() 1180 return b.state == blockStateOpen 1181 } 1182 1183 func (b *block) IsSealedWithRLock() bool { 1184 return b.state == blockStateSealed 1185 } 1186 1187 func (b *block) IsSealed() bool { 1188 b.RLock() 1189 defer b.RUnlock() 1190 return b.IsSealedWithRLock() 1191 } 1192 1193 func (b *block) NeedsMutableSegmentsEvicted() bool { 1194 b.RLock() 1195 defer b.RUnlock() 1196 1197 // Check any mutable segments that can be evicted after a flush. 1198 anyMutableSegmentNeedsEviction := b.mutableSegments.NeedsEviction() 1199 1200 // Check bootstrapped segments and to see if any of them need an eviction. 1201 b.shardRangesSegmentsByVolumeType.forEachSegment(func(seg segment.Segment) error { 1202 if mutableSeg, ok := seg.(segment.MutableSegment); ok { 1203 anyMutableSegmentNeedsEviction = anyMutableSegmentNeedsEviction || mutableSeg.Size() > 0 1204 } 1205 return nil 1206 }) 1207 1208 return anyMutableSegmentNeedsEviction 1209 } 1210 1211 func (b *block) EvictMutableSegments() error { 1212 b.Lock() 1213 defer b.Unlock() 1214 if b.state != blockStateSealed { 1215 return fmt.Errorf("unable to evict mutable segments, block must be sealed, found: %v", b.state) 1216 } 1217 1218 b.mutableSegments.Close() 1219 1220 // Close any other mutable segments that was added. 1221 multiErr := xerrors.NewMultiError() 1222 for _, shardRangesSegments := range b.shardRangesSegmentsByVolumeType { 1223 for idx := range shardRangesSegments { 1224 segments := make([]segment.Segment, 0, len(shardRangesSegments[idx].segments)) 1225 for _, seg := range shardRangesSegments[idx].segments { 1226 mutableSeg, ok := seg.(segment.MutableSegment) 1227 if !ok { 1228 segments = append(segments, seg) 1229 continue 1230 } 1231 multiErr = multiErr.Add(mutableSeg.Close()) 1232 } 1233 shardRangesSegments[idx].segments = segments 1234 } 1235 } 1236 1237 return multiErr.FinalError() 1238 } 1239 1240 func (b *block) NeedsColdMutableSegmentsEvicted() bool { 1241 b.RLock() 1242 defer b.RUnlock() 1243 var anyColdMutableSegmentNeedsEviction bool 1244 for _, coldSeg := range b.coldMutableSegments { 1245 anyColdMutableSegmentNeedsEviction = anyColdMutableSegmentNeedsEviction || coldSeg.NeedsEviction() 1246 } 1247 return b.state == blockStateSealed && anyColdMutableSegmentNeedsEviction 1248 } 1249 1250 func (b *block) EvictColdMutableSegments() error { 1251 b.Lock() 1252 defer b.Unlock() 1253 if b.state != blockStateSealed { 1254 return fmt.Errorf("unable to evict cold mutable segments, block must be sealed, found: %v", b.state) 1255 } 1256 1257 // Evict/remove all but the most recent cold mutable segment (That is the one we are actively writing to). 1258 for i, coldSeg := range b.coldMutableSegments { 1259 if i < len(b.coldMutableSegments)-1 { 1260 coldSeg.Close() 1261 b.coldMutableSegments[i] = nil 1262 } 1263 } 1264 // Swap last w/ first and truncate the slice. 1265 lastIdx := len(b.coldMutableSegments) - 1 1266 b.coldMutableSegments[0], b.coldMutableSegments[lastIdx] = b.coldMutableSegments[lastIdx], b.coldMutableSegments[0] 1267 b.coldMutableSegments = b.coldMutableSegments[:1] 1268 return nil 1269 } 1270 1271 func (b *block) RotateColdMutableSegments() error { 1272 b.Lock() 1273 defer b.Unlock() 1274 coldSegs := newMutableSegments( 1275 b.nsMD, 1276 b.blockStart, 1277 b.opts, 1278 b.blockOpts, 1279 b.cachedSearchesWorkers, 1280 b.namespaceRuntimeOptsMgr, 1281 b.iopts, 1282 ) 1283 b.coldMutableSegments = append(b.coldMutableSegments, coldSegs) 1284 return nil 1285 } 1286 1287 func (b *block) MemorySegmentsData(ctx context.Context) ([]fst.SegmentData, error) { 1288 b.RLock() 1289 defer b.RUnlock() 1290 if b.state == blockStateClosed { 1291 return nil, errBlockAlreadyClosed 1292 } 1293 data, err := b.mutableSegments.MemorySegmentsData(ctx) 1294 if err != nil { 1295 return nil, err 1296 } 1297 for _, coldSeg := range b.coldMutableSegments { 1298 coldData, err := coldSeg.MemorySegmentsData(ctx) 1299 if err != nil { 1300 return nil, err 1301 } 1302 data = append(data, coldData...) 1303 } 1304 return data, nil 1305 } 1306 1307 func (b *block) Close() error { 1308 b.Lock() 1309 defer b.Unlock() 1310 if b.state == blockStateClosed { 1311 return errBlockAlreadyClosed 1312 } 1313 b.state = blockStateClosed 1314 1315 b.mutableSegments.Close() 1316 for _, coldSeg := range b.coldMutableSegments { 1317 coldSeg.Close() 1318 } 1319 1320 // Close any other added segments too. 1321 var multiErr xerrors.MultiError 1322 b.shardRangesSegmentsByVolumeType.forEachSegment(func(seg segment.Segment) error { 1323 multiErr = multiErr.Add(seg.Close()) 1324 return nil 1325 }) 1326 1327 for volumeType := range b.shardRangesSegmentsByVolumeType { 1328 b.shardRangesSegmentsByVolumeType[volumeType] = nil 1329 } 1330 1331 return multiErr.FinalError() 1332 } 1333 1334 func (b *block) writeBatchErrorInvalidState(state blockState) error { 1335 switch state { 1336 case blockStateClosed: 1337 return errUnableToWriteBlockClosed 1338 case blockStateSealed: 1339 return errUnableToWriteBlockSealed 1340 default: // should never happen 1341 err := fmt.Errorf(errUnableToWriteBlockUnknownStateFmtString, state) 1342 instrument.EmitAndLogInvariantViolation(b.opts.InstrumentOptions(), func(l *zap.Logger) { 1343 l.Error(err.Error()) 1344 }) 1345 return err 1346 } 1347 }