lib/storage: consistently use stopCh instead of stop

This commit is contained in:
Aliaksandr Valialkin 2024-04-02 21:24:57 +03:00
parent be36ceb1cf
commit c3a72b6cdb
No known key found for this signature in database
GPG key ID: 52C003EE2BCDB9EB
3 changed files with 24 additions and 22 deletions

View file

@ -1201,7 +1201,7 @@ func (pt *partition) mergePartsToFiles(pws []*partWrapper, stopCh <-chan struct{
} }
// ForceMergeAllParts runs merge for all the parts in pt. // ForceMergeAllParts runs merge for all the parts in pt.
func (pt *partition) ForceMergeAllParts(stop chan struct{}) error { func (pt *partition) ForceMergeAllParts(stopCh <-chan struct{}) error {
pws := pt.getAllPartsForMerge() pws := pt.getAllPartsForMerge()
if len(pws) == 0 { if len(pws) == 0 {
// Nothing to merge. // Nothing to merge.
@ -1221,7 +1221,7 @@ func (pt *partition) ForceMergeAllParts(stop chan struct{}) error {
// If len(pws) == 1, then the merge must run anyway. // If len(pws) == 1, then the merge must run anyway.
// This allows applying the configured retention, removing the deleted series // This allows applying the configured retention, removing the deleted series
// and performing de-duplication if needed. // and performing de-duplication if needed.
if err := pt.mergePartsToFiles(pws, stop, bigPartsConcurrencyCh); err != nil { if err := pt.mergePartsToFiles(pws, stopCh, bigPartsConcurrencyCh); err != nil {
return fmt.Errorf("cannot force merge %d parts from partition %q: %w", len(pws), pt.name, err) return fmt.Errorf("cannot force merge %d parts from partition %q: %w", len(pws), pt.name, err)
} }
@ -1330,10 +1330,10 @@ func (pt *partition) releasePartsToMerge(pws []*partWrapper) {
pt.partsLock.Unlock() pt.partsLock.Unlock()
} }
func (pt *partition) runFinalDedup() error { func (pt *partition) runFinalDedup(stopCh <-chan struct{}) error {
t := time.Now() t := time.Now()
logger.Infof("start removing duplicate samples from partition (%s, %s)", pt.bigPartsPath, pt.smallPartsPath) logger.Infof("start removing duplicate samples from partition (%s, %s)", pt.bigPartsPath, pt.smallPartsPath)
if err := pt.ForceMergeAllParts(pt.stopCh); err != nil { if err := pt.ForceMergeAllParts(stopCh); err != nil {
return fmt.Errorf("cannot remove duplicate samples from partition (%s, %s): %w", pt.bigPartsPath, pt.smallPartsPath, err) return fmt.Errorf("cannot remove duplicate samples from partition (%s, %s): %w", pt.bigPartsPath, pt.smallPartsPath, err)
} }
logger.Infof("duplicate samples have been removed from partition (%s, %s) in %.3f seconds", pt.bigPartsPath, pt.smallPartsPath, time.Since(t).Seconds()) logger.Infof("duplicate samples have been removed from partition (%s, %s) in %.3f seconds", pt.bigPartsPath, pt.smallPartsPath, time.Since(t).Seconds())

View file

@ -124,7 +124,7 @@ type Storage struct {
// prefetchedMetricIDsDeadline is used for periodic reset of prefetchedMetricIDs in order to limit its size under high rate of creating new series. // prefetchedMetricIDsDeadline is used for periodic reset of prefetchedMetricIDs in order to limit its size under high rate of creating new series.
prefetchedMetricIDsDeadline atomic.Uint64 prefetchedMetricIDsDeadline atomic.Uint64
stop chan struct{} stopCh chan struct{}
currHourMetricIDsUpdaterWG sync.WaitGroup currHourMetricIDsUpdaterWG sync.WaitGroup
nextDayMetricIDsUpdaterWG sync.WaitGroup nextDayMetricIDsUpdaterWG sync.WaitGroup
@ -173,7 +173,7 @@ func MustOpenStorage(path string, retention time.Duration, maxHourlySeries, maxD
path: path, path: path,
cachePath: filepath.Join(path, cacheDirname), cachePath: filepath.Join(path, cacheDirname),
retentionMsecs: retention.Milliseconds(), retentionMsecs: retention.Milliseconds(),
stop: make(chan struct{}), stopCh: make(chan struct{}),
} }
fs.MustMkdirIfNotExist(path) fs.MustMkdirIfNotExist(path)
@ -693,7 +693,7 @@ func (s *Storage) startFreeDiskSpaceWatcher() {
defer ticker.Stop() defer ticker.Stop()
for { for {
select { select {
case <-s.stop: case <-s.stopCh:
return return
case <-ticker.C: case <-ticker.C:
f() f()
@ -724,7 +724,7 @@ func (s *Storage) retentionWatcher() {
for { for {
d := s.nextRetentionSeconds() d := s.nextRetentionSeconds()
select { select {
case <-s.stop: case <-s.stopCh:
return return
case currentTime := <-time.After(time.Second * time.Duration(d)): case currentTime := <-time.After(time.Second * time.Duration(d)):
s.mustRotateIndexDB(currentTime) s.mustRotateIndexDB(currentTime)
@ -754,7 +754,7 @@ func (s *Storage) currHourMetricIDsUpdater() {
defer ticker.Stop() defer ticker.Stop()
for { for {
select { select {
case <-s.stop: case <-s.stopCh:
hour := fasttime.UnixHour() hour := fasttime.UnixHour()
s.updateCurrHourMetricIDs(hour) s.updateCurrHourMetricIDs(hour)
return return
@ -771,7 +771,7 @@ func (s *Storage) nextDayMetricIDsUpdater() {
defer ticker.Stop() defer ticker.Stop()
for { for {
select { select {
case <-s.stop: case <-s.stopCh:
date := fasttime.UnixDate() date := fasttime.UnixDate()
s.updateNextDayMetricIDs(date) s.updateNextDayMetricIDs(date)
return return
@ -856,7 +856,7 @@ func (s *Storage) resetAndSaveTSIDCache() {
// //
// It is expected that the s is no longer used during the close. // It is expected that the s is no longer used during the close.
func (s *Storage) MustClose() { func (s *Storage) MustClose() {
close(s.stop) close(s.stopCh)
s.freeDiskSpaceWatcherWG.Wait() s.freeDiskSpaceWatcherWG.Wait()
s.retentionWatcherWG.Wait() s.retentionWatcherWG.Wait()

View file

@ -25,11 +25,11 @@ type table struct {
ptws []*partitionWrapper ptws []*partitionWrapper
ptwsLock sync.Mutex ptwsLock sync.Mutex
stop chan struct{} stopCh chan struct{}
retentionWatcherWG sync.WaitGroup retentionWatcherWG sync.WaitGroup
finalDedupWatcherWG sync.WaitGroup finalDedupWatcherWG sync.WaitGroup
forceMergesWG sync.WaitGroup forceMergeWG sync.WaitGroup
} }
// partitionWrapper provides refcounting mechanism for the partition. // partitionWrapper provides refcounting mechanism for the partition.
@ -108,7 +108,7 @@ func mustOpenTable(path string, s *Storage) *table {
bigPartitionsPath: bigPartitionsPath, bigPartitionsPath: bigPartitionsPath,
s: s, s: s,
stop: make(chan struct{}), stopCh: make(chan struct{}),
} }
for _, pt := range pts { for _, pt := range pts {
tb.addPartitionNolock(pt) tb.addPartitionNolock(pt)
@ -166,10 +166,10 @@ func (tb *table) addPartitionNolock(pt *partition) {
// MustClose closes the table. // MustClose closes the table.
// It is expected that all the pending searches on the table are finished before calling MustClose. // It is expected that all the pending searches on the table are finished before calling MustClose.
func (tb *table) MustClose() { func (tb *table) MustClose() {
close(tb.stop) close(tb.stopCh)
tb.retentionWatcherWG.Wait() tb.retentionWatcherWG.Wait()
tb.finalDedupWatcherWG.Wait() tb.finalDedupWatcherWG.Wait()
tb.forceMergesWG.Wait() tb.forceMergeWG.Wait()
tb.ptwsLock.Lock() tb.ptwsLock.Lock()
ptws := tb.ptws ptws := tb.ptws
@ -244,15 +244,17 @@ func (tb *table) UpdateMetrics(m *TableMetrics) {
func (tb *table) ForceMergePartitions(partitionNamePrefix string) error { func (tb *table) ForceMergePartitions(partitionNamePrefix string) error {
ptws := tb.GetPartitions(nil) ptws := tb.GetPartitions(nil)
defer tb.PutPartitions(ptws) defer tb.PutPartitions(ptws)
tb.forceMergesWG.Add(1)
defer tb.forceMergesWG.Done() tb.forceMergeWG.Add(1)
defer tb.forceMergeWG.Done()
for _, ptw := range ptws { for _, ptw := range ptws {
if !strings.HasPrefix(ptw.pt.name, partitionNamePrefix) { if !strings.HasPrefix(ptw.pt.name, partitionNamePrefix) {
continue continue
} }
logger.Infof("starting forced merge for partition %q", ptw.pt.name) logger.Infof("starting forced merge for partition %q", ptw.pt.name)
startTime := time.Now() startTime := time.Now()
if err := ptw.pt.ForceMergeAllParts(tb.stop); err != nil { if err := ptw.pt.ForceMergeAllParts(tb.stopCh); err != nil {
return fmt.Errorf("cannot complete forced merge for partition %q: %w", ptw.pt.name, err) return fmt.Errorf("cannot complete forced merge for partition %q: %w", ptw.pt.name, err)
} }
logger.Infof("forced merge for partition %q has been finished in %.3f seconds", ptw.pt.name, time.Since(startTime).Seconds()) logger.Infof("forced merge for partition %q has been finished in %.3f seconds", ptw.pt.name, time.Since(startTime).Seconds())
@ -390,7 +392,7 @@ func (tb *table) retentionWatcher() {
defer ticker.Stop() defer ticker.Stop()
for { for {
select { select {
case <-tb.stop: case <-tb.stopCh:
return return
case <-ticker.C: case <-ticker.C:
} }
@ -457,7 +459,7 @@ func (tb *table) finalDedupWatcher() {
ptwsToDedup = append(ptwsToDedup, ptw) ptwsToDedup = append(ptwsToDedup, ptw)
} }
for _, ptw := range ptwsToDedup { for _, ptw := range ptwsToDedup {
if err := ptw.pt.runFinalDedup(); err != nil { if err := ptw.pt.runFinalDedup(tb.stopCh); err != nil {
logger.Errorf("cannot run final dedup for partition %s: %s", ptw.pt.name, err) logger.Errorf("cannot run final dedup for partition %s: %s", ptw.pt.name, err)
} }
ptw.pt.isDedupScheduled.Store(false) ptw.pt.isDedupScheduled.Store(false)
@ -468,7 +470,7 @@ func (tb *table) finalDedupWatcher() {
defer t.Stop() defer t.Stop()
for { for {
select { select {
case <-tb.stop: case <-tb.stopCh:
return return
case <-t.C: case <-t.C:
f() f()