all: replace atomic.Value with atomic.Pointer[T]

This eliminates the need in .(*T) casting for results obtained from Load()

Leave atomic.Value for map, since atomic.Pointer[map[...]...] makes double pointer to map,
because map is already a pointer type.
This commit is contained in:
Aliaksandr Valialkin 2023-07-19 17:37:49 -07:00
parent 399dc39ee9
commit 140e7b6b74
No known key found for this signature in database
GPG key ID: A72BEC6CD3D0DED1
12 changed files with 87 additions and 89 deletions

View file

@ -93,7 +93,7 @@ func MultitenancyEnabled() bool {
} }
// Contains the current relabelConfigs. // Contains the current relabelConfigs.
var allRelabelConfigs atomic.Value var allRelabelConfigs atomic.Pointer[relabelConfigs]
// maxQueues limits the maximum value for `-remoteWrite.queues`. There is no sense in setting too high value, // maxQueues limits the maximum value for `-remoteWrite.queues`. There is no sense in setting too high value,
// since it may lead to high memory usage due to big number of buffers. // since it may lead to high memory usage due to big number of buffers.
@ -346,7 +346,7 @@ func Push(at *auth.Token, wr *prompbmarshal.WriteRequest) {
} }
var rctx *relabelCtx var rctx *relabelCtx
rcs := allRelabelConfigs.Load().(*relabelConfigs) rcs := allRelabelConfigs.Load()
pcsGlobal := rcs.global pcsGlobal := rcs.global
if pcsGlobal.Len() > 0 || len(labelsGlobal) > 0 { if pcsGlobal.Len() > 0 || len(labelsGlobal) > 0 {
rctx = getRelabelCtx() rctx = getRelabelCtx()
@ -612,7 +612,7 @@ func (rwctx *remoteWriteCtx) Push(tss []prompbmarshal.TimeSeries) {
// Apply relabeling // Apply relabeling
var rctx *relabelCtx var rctx *relabelCtx
var v *[]prompbmarshal.TimeSeries var v *[]prompbmarshal.TimeSeries
rcs := allRelabelConfigs.Load().(*relabelConfigs) rcs := allRelabelConfigs.Load()
pcs := rcs.perURL[rwctx.idx] pcs := rcs.perURL[rwctx.idx]
if pcs.Len() > 0 { if pcs.Len() > 0 {
rctx = getRelabelCtx() rctx = getRelabelCtx()

View file

@ -69,7 +69,7 @@ var (
configTimestamp = metrics.NewCounter(`vm_relabel_config_last_reload_success_timestamp_seconds`) configTimestamp = metrics.NewCounter(`vm_relabel_config_last_reload_success_timestamp_seconds`)
) )
var pcsGlobal atomic.Value var pcsGlobal atomic.Pointer[promrelabel.ParsedConfigs]
// CheckRelabelConfig checks config pointed by -relabelConfig // CheckRelabelConfig checks config pointed by -relabelConfig
func CheckRelabelConfig() error { func CheckRelabelConfig() error {
@ -90,7 +90,7 @@ func loadRelabelConfig() (*promrelabel.ParsedConfigs, error) {
// HasRelabeling returns true if there is global relabeling. // HasRelabeling returns true if there is global relabeling.
func HasRelabeling() bool { func HasRelabeling() bool {
pcs := pcsGlobal.Load().(*promrelabel.ParsedConfigs) pcs := pcsGlobal.Load()
return pcs.Len() > 0 || *usePromCompatibleNaming return pcs.Len() > 0 || *usePromCompatibleNaming
} }
@ -110,7 +110,7 @@ func (ctx *Ctx) Reset() {
// //
// The returned labels are valid until the next call to ApplyRelabeling. // The returned labels are valid until the next call to ApplyRelabeling.
func (ctx *Ctx) ApplyRelabeling(labels []prompb.Label) []prompb.Label { func (ctx *Ctx) ApplyRelabeling(labels []prompb.Label) []prompb.Label {
pcs := pcsGlobal.Load().(*promrelabel.ParsedConfigs) pcs := pcsGlobal.Load()
if pcs.Len() == 0 && !*usePromCompatibleNaming { if pcs.Len() == 0 && !*usePromCompatibleNaming {
// There are no relabeling rules. // There are no relabeling rules.
return labels return labels

View file

@ -63,7 +63,7 @@ func checkRollupResultCacheReset() {
for { for {
time.Sleep(checkRollupResultCacheResetInterval) time.Sleep(checkRollupResultCacheResetInterval)
if atomic.SwapUint32(&needRollupResultCacheReset, 0) > 0 { if atomic.SwapUint32(&needRollupResultCacheReset, 0) > 0 {
mr := rollupResultResetMetricRowSample.Load().(*storage.MetricRow) mr := rollupResultResetMetricRowSample.Load()
d := int64(fasttime.UnixTimestamp()*1000) - mr.Timestamp - cacheTimestampOffset.Milliseconds() d := int64(fasttime.UnixTimestamp()*1000) - mr.Timestamp - cacheTimestampOffset.Milliseconds()
logger.Warnf("resetting rollup result cache because the metric %s has a timestamp older than -search.cacheTimestampOffset=%s by %.3fs", logger.Warnf("resetting rollup result cache because the metric %s has a timestamp older than -search.cacheTimestampOffset=%s by %.3fs",
mr.String(), cacheTimestampOffset, float64(d)/1e3) mr.String(), cacheTimestampOffset, float64(d)/1e3)
@ -76,7 +76,7 @@ const checkRollupResultCacheResetInterval = 5 * time.Second
var needRollupResultCacheReset uint32 var needRollupResultCacheReset uint32
var checkRollupResultCacheResetOnce sync.Once var checkRollupResultCacheResetOnce sync.Once
var rollupResultResetMetricRowSample atomic.Value var rollupResultResetMetricRowSample atomic.Pointer[storage.MetricRow]
var rollupResultCacheV = &rollupResultCache{ var rollupResultCacheV = &rollupResultCache{
c: workingsetcache.New(1024 * 1024), // This is a cache for testing. c: workingsetcache.New(1024 * 1024), // This is a cache for testing.

View file

@ -32,14 +32,14 @@ func WritePrometheusMetrics(w io.Writer) {
} }
metricsCacheLock.Unlock() metricsCacheLock.Unlock()
bb := metricsCache.Load().(*bytesutil.ByteBuffer) bb := metricsCache.Load()
_, _ = w.Write(bb.B) _, _ = w.Write(bb.B)
} }
var ( var (
metricsCacheLock sync.Mutex metricsCacheLock sync.Mutex
metricsCacheLastUpdateTime time.Time metricsCacheLastUpdateTime time.Time
metricsCache atomic.Value metricsCache atomic.Pointer[bytesutil.ByteBuffer]
) )
func writePrometheusMetrics(w io.Writer) { func writePrometheusMetrics(w io.Writer) {

View file

@ -11,7 +11,7 @@ import (
// It is safe using the Limiter from concurrent goroutines. // It is safe using the Limiter from concurrent goroutines.
type Limiter struct { type Limiter struct {
maxItems int maxItems int
v atomic.Value v atomic.Pointer[limiter]
wg sync.WaitGroup wg sync.WaitGroup
stopCh chan struct{} stopCh chan struct{}
@ -55,7 +55,7 @@ func (l *Limiter) MaxItems() int {
// CurrentItems return the current number of items registered in l. // CurrentItems return the current number of items registered in l.
func (l *Limiter) CurrentItems() int { func (l *Limiter) CurrentItems() int {
lm := l.v.Load().(*limiter) lm := l.v.Load()
n := atomic.LoadUint64(&lm.currentItems) n := atomic.LoadUint64(&lm.currentItems)
return int(n) return int(n)
} }
@ -67,7 +67,7 @@ func (l *Limiter) CurrentItems() int {
// True is returned if h is added or already exists in l. // True is returned if h is added or already exists in l.
// False is returned if h cannot be added to l, since it already has maxItems unique items. // False is returned if h cannot be added to l, since it already has maxItems unique items.
func (l *Limiter) Add(h uint64) bool { func (l *Limiter) Add(h uint64) bool {
lm := l.v.Load().(*limiter) lm := l.v.Load()
return lm.Add(h) return lm.Add(h)
} }

View file

@ -15,7 +15,7 @@ var (
decoder *zstd.Decoder decoder *zstd.Decoder
mu sync.Mutex mu sync.Mutex
av atomic.Value av atomic.Pointer[registry]
) )
type registry map[int]*zstd.Encoder type registry map[int]*zstd.Encoder
@ -45,7 +45,7 @@ func CompressLevel(dst, src []byte, compressionLevel int) []byte {
} }
func getEncoder(compressionLevel int) *zstd.Encoder { func getEncoder(compressionLevel int) *zstd.Encoder {
r := av.Load().(registry) r := av.Load()
e := r[compressionLevel] e := r[compressionLevel]
if e != nil { if e != nil {
return e return e
@ -54,7 +54,7 @@ func getEncoder(compressionLevel int) *zstd.Encoder {
mu.Lock() mu.Lock()
// Create the encoder under lock in order to prevent from wasted work // Create the encoder under lock in order to prevent from wasted work
// when concurrent goroutines create encoder for the same compressionLevel. // when concurrent goroutines create encoder for the same compressionLevel.
r1 := av.Load().(registry) r1 := av.Load()
if e = r1[compressionLevel]; e == nil { if e = r1[compressionLevel]; e == nil {
e = newEncoder(compressionLevel) e = newEncoder(compressionLevel)
r2 := make(registry) r2 := make(registry)

View file

@ -27,7 +27,7 @@ type apiConfig struct {
apiPath string apiPath string
// labels contains the latest discovered labels. // labels contains the latest discovered labels.
labels atomic.Value labels atomic.Pointer[[]*promutils.Labels]
cancel context.CancelFunc cancel context.CancelFunc
wg sync.WaitGroup wg sync.WaitGroup

View file

@ -38,8 +38,7 @@ func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config for kuma_sd: %w", err) return nil, fmt.Errorf("cannot get API config for kuma_sd: %w", err)
} }
v := cfg.labels.Load() pLabels := cfg.labels.Load()
pLabels := v.(*[]*promutils.Labels)
return *pLabels, nil return *pLabels, nil
} }

View file

@ -84,18 +84,17 @@ var (
PendingScrapeConfigs int32 PendingScrapeConfigs int32
// configData contains -promscrape.config data // configData contains -promscrape.config data
configData atomic.Value configData atomic.Pointer[[]byte]
) )
// WriteConfigData writes -promscrape.config contents to w // WriteConfigData writes -promscrape.config contents to w
func WriteConfigData(w io.Writer) { func WriteConfigData(w io.Writer) {
v := configData.Load() p := configData.Load()
if v == nil { if p == nil {
// Nothing to write to w // Nothing to write to w
return return
} }
b := v.(*[]byte) _, _ = w.Write(*p)
_, _ = w.Write(*b)
} }
func runScraper(configFile string, pushData func(at *auth.Token, wr *prompbmarshal.WriteRequest), globalStopCh <-chan struct{}) { func runScraper(configFile string, pushData func(at *auth.Token, wr *prompbmarshal.WriteRequest), globalStopCh <-chan struct{}) {

View file

@ -60,7 +60,7 @@ type Storage struct {
// lock file for exclusive access to the storage on the given path. // lock file for exclusive access to the storage on the given path.
flockF *os.File flockF *os.File
idbCurr atomic.Value idbCurr atomic.Pointer[indexDB]
tb *table tb *table
@ -81,16 +81,16 @@ type Storage struct {
dateMetricIDCache *dateMetricIDCache dateMetricIDCache *dateMetricIDCache
// Fast cache for MetricID values occurred during the current hour. // Fast cache for MetricID values occurred during the current hour.
currHourMetricIDs atomic.Value currHourMetricIDs atomic.Pointer[hourMetricIDs]
// Fast cache for MetricID values occurred during the previous hour. // Fast cache for MetricID values occurred during the previous hour.
prevHourMetricIDs atomic.Value prevHourMetricIDs atomic.Pointer[hourMetricIDs]
// Fast cache for pre-populating per-day inverted index for the next day. // Fast cache for pre-populating per-day inverted index for the next day.
// This is needed in order to remove CPU usage spikes at 00:00 UTC // This is needed in order to remove CPU usage spikes at 00:00 UTC
// due to creation of per-day inverted index for active time series. // due to creation of per-day inverted index for active time series.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/430 for details. // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/430 for details.
nextDayMetricIDs atomic.Value nextDayMetricIDs atomic.Pointer[byDateMetricIDEntry]
// Pending MetricID values to be added to currHourMetricIDs. // Pending MetricID values to be added to currHourMetricIDs.
pendingHourEntriesLock sync.Mutex pendingHourEntriesLock sync.Mutex
@ -101,7 +101,7 @@ type Storage struct {
pendingNextDayMetricIDs *uint64set.Set pendingNextDayMetricIDs *uint64set.Set
// prefetchedMetricIDs contains metricIDs for pre-fetched metricNames in the prefetchMetricNames function. // prefetchedMetricIDs contains metricIDs for pre-fetched metricNames in the prefetchMetricNames function.
prefetchedMetricIDs atomic.Value prefetchedMetricIDs atomic.Pointer[uint64set.Set]
// 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 uint64 prefetchedMetricIDsDeadline uint64
@ -129,7 +129,7 @@ type Storage struct {
// //
// It is safe to keep the set in memory even for big number of deleted // It is safe to keep the set in memory even for big number of deleted
// metricIDs, since it usually requires 1 bit per deleted metricID. // metricIDs, since it usually requires 1 bit per deleted metricID.
deletedMetricIDs atomic.Value deletedMetricIDs atomic.Pointer[uint64set.Set]
deletedMetricIDsUpdateLock sync.Mutex deletedMetricIDsUpdateLock sync.Mutex
isReadOnly uint32 isReadOnly uint32
@ -268,7 +268,7 @@ func getTSIDCacheSize() int {
} }
func (s *Storage) getDeletedMetricIDs() *uint64set.Set { func (s *Storage) getDeletedMetricIDs() *uint64set.Set {
return s.deletedMetricIDs.Load().(*uint64set.Set) return s.deletedMetricIDs.Load()
} }
func (s *Storage) setDeletedMetricIDs(dmis *uint64set.Set) { func (s *Storage) setDeletedMetricIDs(dmis *uint64set.Set) {
@ -434,7 +434,7 @@ func (s *Storage) DeleteStaleSnapshots(maxAge time.Duration) error {
} }
func (s *Storage) idb() *indexDB { func (s *Storage) idb() *indexDB {
return s.idbCurr.Load().(*indexDB) return s.idbCurr.Load()
} }
// Metrics contains essential metrics for the Storage. // Metrics contains essential metrics for the Storage.
@ -569,8 +569,8 @@ func (s *Storage) UpdateMetrics(m *Metrics) {
m.DateMetricIDCacheSyncsCount += atomic.LoadUint64(&s.dateMetricIDCache.syncsCount) m.DateMetricIDCacheSyncsCount += atomic.LoadUint64(&s.dateMetricIDCache.syncsCount)
m.DateMetricIDCacheResetsCount += atomic.LoadUint64(&s.dateMetricIDCache.resetsCount) m.DateMetricIDCacheResetsCount += atomic.LoadUint64(&s.dateMetricIDCache.resetsCount)
hmCurr := s.currHourMetricIDs.Load().(*hourMetricIDs) hmCurr := s.currHourMetricIDs.Load()
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
hourMetricIDsLen := hmPrev.m.Len() hourMetricIDsLen := hmPrev.m.Len()
if hmCurr.m.Len() > hourMetricIDsLen { if hmCurr.m.Len() > hourMetricIDsLen {
hourMetricIDsLen = hmCurr.m.Len() hourMetricIDsLen = hmCurr.m.Len()
@ -579,11 +579,11 @@ func (s *Storage) UpdateMetrics(m *Metrics) {
m.HourMetricIDCacheSizeBytes += hmCurr.m.SizeBytes() m.HourMetricIDCacheSizeBytes += hmCurr.m.SizeBytes()
m.HourMetricIDCacheSizeBytes += hmPrev.m.SizeBytes() m.HourMetricIDCacheSizeBytes += hmPrev.m.SizeBytes()
nextDayMetricIDs := &s.nextDayMetricIDs.Load().(*byDateMetricIDEntry).v nextDayMetricIDs := &s.nextDayMetricIDs.Load().v
m.NextDayMetricIDCacheSize += uint64(nextDayMetricIDs.Len()) m.NextDayMetricIDCacheSize += uint64(nextDayMetricIDs.Len())
m.NextDayMetricIDCacheSizeBytes += nextDayMetricIDs.SizeBytes() m.NextDayMetricIDCacheSizeBytes += nextDayMetricIDs.SizeBytes()
prefetchedMetricIDs := s.prefetchedMetricIDs.Load().(*uint64set.Set) prefetchedMetricIDs := s.prefetchedMetricIDs.Load()
m.PrefetchedMetricIDsSize += uint64(prefetchedMetricIDs.Len()) m.PrefetchedMetricIDsSize += uint64(prefetchedMetricIDs.Len())
m.PrefetchedMetricIDsSizeBytes += uint64(prefetchedMetricIDs.SizeBytes()) m.PrefetchedMetricIDsSizeBytes += uint64(prefetchedMetricIDs.SizeBytes())
@ -797,12 +797,12 @@ func (s *Storage) MustClose() {
s.mustSaveCache(s.metricNameCache, "metricID_metricName") s.mustSaveCache(s.metricNameCache, "metricID_metricName")
s.metricNameCache.Stop() s.metricNameCache.Stop()
hmCurr := s.currHourMetricIDs.Load().(*hourMetricIDs) hmCurr := s.currHourMetricIDs.Load()
s.mustSaveHourMetricIDs(hmCurr, "curr_hour_metric_ids") s.mustSaveHourMetricIDs(hmCurr, "curr_hour_metric_ids")
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
s.mustSaveHourMetricIDs(hmPrev, "prev_hour_metric_ids") s.mustSaveHourMetricIDs(hmPrev, "prev_hour_metric_ids")
nextDayMetricIDs := s.nextDayMetricIDs.Load().(*byDateMetricIDEntry) nextDayMetricIDs := s.nextDayMetricIDs.Load()
s.mustSaveNextDayMetricIDs(nextDayMetricIDs) s.mustSaveNextDayMetricIDs(nextDayMetricIDs)
// Release lock file. // Release lock file.
@ -1093,7 +1093,7 @@ func (s *Storage) prefetchMetricNames(qt *querytracer.Tracer, srcMetricIDs []uin
return nil return nil
} }
var metricIDs uint64Sorter var metricIDs uint64Sorter
prefetchedMetricIDs := s.prefetchedMetricIDs.Load().(*uint64set.Set) prefetchedMetricIDs := s.prefetchedMetricIDs.Load()
for _, metricID := range srcMetricIDs { for _, metricID := range srcMetricIDs {
if prefetchedMetricIDs.Has(metricID) { if prefetchedMetricIDs.Has(metricID) {
continue continue
@ -1906,10 +1906,10 @@ func (s *Storage) updatePerDateData(rows []rawRow, mrs []*MetricRow) error {
prevDate uint64 prevDate uint64
prevMetricID uint64 prevMetricID uint64
) )
hm := s.currHourMetricIDs.Load().(*hourMetricIDs) hm := s.currHourMetricIDs.Load()
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
hmPrevDate := hmPrev.hour / 24 hmPrevDate := hmPrev.hour / 24
nextDayMetricIDs := &s.nextDayMetricIDs.Load().(*byDateMetricIDEntry).v nextDayMetricIDs := &s.nextDayMetricIDs.Load().v
ts := fasttime.UnixTimestamp() ts := fasttime.UnixTimestamp()
// Start pre-populating the next per-day inverted index during the last hour of the current day. // Start pre-populating the next per-day inverted index during the last hour of the current day.
// pMin linearly increases from 0 to 1 during the last hour of the day. // pMin linearly increases from 0 to 1 during the last hour of the day.
@ -2055,7 +2055,7 @@ type dateMetricIDCache struct {
resetsCount uint64 resetsCount uint64
// Contains immutable map // Contains immutable map
byDate atomic.Value byDate atomic.Pointer[byDateMetricIDMap]
// Contains mutable map protected by mu // Contains mutable map protected by mu
byDateMutable *byDateMetricIDMap byDateMutable *byDateMetricIDMap
@ -2085,7 +2085,7 @@ func (dmc *dateMetricIDCache) resetLocked() {
} }
func (dmc *dateMetricIDCache) EntriesCount() int { func (dmc *dateMetricIDCache) EntriesCount() int {
byDate := dmc.byDate.Load().(*byDateMetricIDMap) byDate := dmc.byDate.Load()
n := 0 n := 0
for _, e := range byDate.m { for _, e := range byDate.m {
n += e.v.Len() n += e.v.Len()
@ -2094,7 +2094,7 @@ func (dmc *dateMetricIDCache) EntriesCount() int {
} }
func (dmc *dateMetricIDCache) SizeBytes() uint64 { func (dmc *dateMetricIDCache) SizeBytes() uint64 {
byDate := dmc.byDate.Load().(*byDateMetricIDMap) byDate := dmc.byDate.Load()
n := uint64(0) n := uint64(0)
for _, e := range byDate.m { for _, e := range byDate.m {
n += e.v.SizeBytes() n += e.v.SizeBytes()
@ -2103,7 +2103,7 @@ func (dmc *dateMetricIDCache) SizeBytes() uint64 {
} }
func (dmc *dateMetricIDCache) Has(date, metricID uint64) bool { func (dmc *dateMetricIDCache) Has(date, metricID uint64) bool {
byDate := dmc.byDate.Load().(*byDateMetricIDMap) byDate := dmc.byDate.Load()
v := byDate.get(date) v := byDate.get(date)
if v.Has(metricID) { if v.Has(metricID) {
// Fast path. // Fast path.
@ -2169,7 +2169,7 @@ func (dmc *dateMetricIDCache) syncLocked() {
// Nothing to sync. // Nothing to sync.
return return
} }
byDate := dmc.byDate.Load().(*byDateMetricIDMap) byDate := dmc.byDate.Load()
byDateMutable := dmc.byDateMutable byDateMutable := dmc.byDateMutable
for date, e := range byDateMutable.m { for date, e := range byDateMutable.m {
v := byDate.get(date) v := byDate.get(date)
@ -2182,7 +2182,7 @@ func (dmc *dateMetricIDCache) syncLocked() {
date: date, date: date,
v: *v, v: *v,
} }
if date == byDateMutable.hotEntry.Load().(*byDateMetricIDEntry).date { if date == byDateMutable.hotEntry.Load().date {
byDateMutable.hotEntry.Store(dme) byDateMutable.hotEntry.Store(dme)
} }
byDateMutable.m[date] = dme byDateMutable.m[date] = dme
@ -2205,7 +2205,7 @@ func (dmc *dateMetricIDCache) syncLocked() {
} }
type byDateMetricIDMap struct { type byDateMetricIDMap struct {
hotEntry atomic.Value hotEntry atomic.Pointer[byDateMetricIDEntry]
m map[uint64]*byDateMetricIDEntry m map[uint64]*byDateMetricIDEntry
} }
@ -2218,7 +2218,7 @@ func newByDateMetricIDMap() *byDateMetricIDMap {
} }
func (dmm *byDateMetricIDMap) get(date uint64) *uint64set.Set { func (dmm *byDateMetricIDMap) get(date uint64) *uint64set.Set {
hotEntry := dmm.hotEntry.Load().(*byDateMetricIDEntry) hotEntry := dmm.hotEntry.Load()
if hotEntry.date == date { if hotEntry.date == date {
// Fast path // Fast path
return &hotEntry.v return &hotEntry.v
@ -2250,7 +2250,7 @@ type byDateMetricIDEntry struct {
} }
func (s *Storage) updateNextDayMetricIDs(date uint64) { func (s *Storage) updateNextDayMetricIDs(date uint64) {
e := s.nextDayMetricIDs.Load().(*byDateMetricIDEntry) e := s.nextDayMetricIDs.Load()
s.pendingNextDayMetricIDsLock.Lock() s.pendingNextDayMetricIDsLock.Lock()
pendingMetricIDs := s.pendingNextDayMetricIDs pendingMetricIDs := s.pendingNextDayMetricIDs
s.pendingNextDayMetricIDs = &uint64set.Set{} s.pendingNextDayMetricIDs = &uint64set.Set{}
@ -2277,7 +2277,7 @@ func (s *Storage) updateNextDayMetricIDs(date uint64) {
} }
func (s *Storage) updateCurrHourMetricIDs(hour uint64) { func (s *Storage) updateCurrHourMetricIDs(hour uint64) {
hm := s.currHourMetricIDs.Load().(*hourMetricIDs) hm := s.currHourMetricIDs.Load()
s.pendingHourEntriesLock.Lock() s.pendingHourEntriesLock.Lock()
newMetricIDs := s.pendingHourEntries newMetricIDs := s.pendingHourEntries
s.pendingHourEntries = &uint64set.Set{} s.pendingHourEntries = &uint64set.Set{}

View file

@ -169,7 +169,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
hmOrig.m.Add(34) hmOrig.m.Add(34)
s.currHourMetricIDs.Store(hmOrig) s.currHourMetricIDs.Store(hmOrig)
s.updateCurrHourMetricIDs(hour) s.updateCurrHourMetricIDs(hour)
hmCurr := s.currHourMetricIDs.Load().(*hourMetricIDs) hmCurr := s.currHourMetricIDs.Load()
if hmCurr.hour != hour { if hmCurr.hour != hour {
t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour) t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour)
} }
@ -177,7 +177,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
t.Fatalf("unexpected length of hm.m; got %d; want %d", hmCurr.m.Len(), 0) t.Fatalf("unexpected length of hm.m; got %d; want %d", hmCurr.m.Len(), 0)
} }
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
if !reflect.DeepEqual(hmPrev, hmOrig) { if !reflect.DeepEqual(hmPrev, hmOrig) {
t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmOrig) t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmOrig)
} }
@ -197,7 +197,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
hmOrig.m.Add(34) hmOrig.m.Add(34)
s.currHourMetricIDs.Store(hmOrig) s.currHourMetricIDs.Store(hmOrig)
s.updateCurrHourMetricIDs(hour) s.updateCurrHourMetricIDs(hour)
hmCurr := s.currHourMetricIDs.Load().(*hourMetricIDs) hmCurr := s.currHourMetricIDs.Load()
if hmCurr.hour != hour { if hmCurr.hour != hour {
t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour) t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour)
} }
@ -205,7 +205,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
t.Fatalf("unexpected hmCurr; got %v; want %v", hmCurr, hmOrig) t.Fatalf("unexpected hmCurr; got %v; want %v", hmCurr, hmOrig)
} }
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
hmEmpty := &hourMetricIDs{} hmEmpty := &hourMetricIDs{}
if !reflect.DeepEqual(hmPrev, hmEmpty) { if !reflect.DeepEqual(hmPrev, hmEmpty) {
t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmEmpty) t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmEmpty)
@ -235,7 +235,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
hmOrig.m.Add(34) hmOrig.m.Add(34)
s.currHourMetricIDs.Store(hmOrig) s.currHourMetricIDs.Store(hmOrig)
s.updateCurrHourMetricIDs(hour) s.updateCurrHourMetricIDs(hour)
hmCurr := s.currHourMetricIDs.Load().(*hourMetricIDs) hmCurr := s.currHourMetricIDs.Load()
if hmCurr.hour != hour { if hmCurr.hour != hour {
t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour) t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour)
} }
@ -243,7 +243,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
t.Fatalf("unexpected hmCurr.m; got %v; want %v", hmCurr.m, pendingHourEntries) t.Fatalf("unexpected hmCurr.m; got %v; want %v", hmCurr.m, pendingHourEntries)
} }
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
if !reflect.DeepEqual(hmPrev, hmOrig) { if !reflect.DeepEqual(hmPrev, hmOrig) {
t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmOrig) t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmOrig)
} }
@ -269,7 +269,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
hmOrig.m.Add(34) hmOrig.m.Add(34)
s.currHourMetricIDs.Store(hmOrig) s.currHourMetricIDs.Store(hmOrig)
s.updateCurrHourMetricIDs(hour) s.updateCurrHourMetricIDs(hour)
hmCurr := s.currHourMetricIDs.Load().(*hourMetricIDs) hmCurr := s.currHourMetricIDs.Load()
if hmCurr.hour != hour { if hmCurr.hour != hour {
t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour) t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour)
} }
@ -284,7 +284,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
t.Fatalf("unexpected hm.m; got %v; want %v", hmCurr.m, m) t.Fatalf("unexpected hm.m; got %v; want %v", hmCurr.m, m)
} }
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
hmEmpty := &hourMetricIDs{} hmEmpty := &hourMetricIDs{}
if !reflect.DeepEqual(hmPrev, hmEmpty) { if !reflect.DeepEqual(hmPrev, hmEmpty) {
t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmEmpty) t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmEmpty)
@ -312,7 +312,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
hmOrig.m.Add(34) hmOrig.m.Add(34)
s.currHourMetricIDs.Store(hmOrig) s.currHourMetricIDs.Store(hmOrig)
s.updateCurrHourMetricIDs(hour) s.updateCurrHourMetricIDs(hour)
hmCurr := s.currHourMetricIDs.Load().(*hourMetricIDs) hmCurr := s.currHourMetricIDs.Load()
if hmCurr.hour != hour { if hmCurr.hour != hour {
t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour) t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour)
} }
@ -327,7 +327,7 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
t.Fatalf("unexpected hm.m; got %v; want %v", hmCurr.m, m) t.Fatalf("unexpected hm.m; got %v; want %v", hmCurr.m, m)
} }
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
hmEmpty := &hourMetricIDs{} hmEmpty := &hourMetricIDs{}
if !reflect.DeepEqual(hmPrev, hmEmpty) { if !reflect.DeepEqual(hmPrev, hmEmpty) {
t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmEmpty) t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmEmpty)
@ -355,14 +355,14 @@ func TestUpdateCurrHourMetricIDs(t *testing.T) {
} }
s.currHourMetricIDs.Store(hmOrig) s.currHourMetricIDs.Store(hmOrig)
s.updateCurrHourMetricIDs(hour) s.updateCurrHourMetricIDs(hour)
hmCurr := s.currHourMetricIDs.Load().(*hourMetricIDs) hmCurr := s.currHourMetricIDs.Load()
if hmCurr.hour != hour { if hmCurr.hour != hour {
t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour) t.Fatalf("unexpected hmCurr.hour; got %d; want %d", hmCurr.hour, hour)
} }
if hmCurr.m.Len() != 0 { if hmCurr.m.Len() != 0 {
t.Fatalf("unexpected non-empty hmCurr.m; got %v", hmCurr.m.AppendTo(nil)) t.Fatalf("unexpected non-empty hmCurr.m; got %v", hmCurr.m.AppendTo(nil))
} }
hmPrev := s.prevHourMetricIDs.Load().(*hourMetricIDs) hmPrev := s.prevHourMetricIDs.Load()
if !reflect.DeepEqual(hmPrev, hmOrig) { if !reflect.DeepEqual(hmPrev, hmOrig) {
t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmOrig) t.Fatalf("unexpected hmPrev; got %v; want %v", hmPrev, hmOrig)
} }

View file

@ -30,8 +30,8 @@ const (
// The cache evicts inactive entries after the given expireDuration. // The cache evicts inactive entries after the given expireDuration.
// Recently accessed entries survive expireDuration. // Recently accessed entries survive expireDuration.
type Cache struct { type Cache struct {
curr atomic.Value curr atomic.Pointer[fastcache.Cache]
prev atomic.Value prev atomic.Pointer[fastcache.Cache]
// csHistory holds cache stats history // csHistory holds cache stats history
csHistory fastcache.Stats csHistory fastcache.Stats
@ -148,8 +148,8 @@ func (c *Cache) expirationWatcher(expireDuration time.Duration) {
return return
} }
// Reset prev cache and swap it with the curr cache. // Reset prev cache and swap it with the curr cache.
prev := c.prev.Load().(*fastcache.Cache) prev := c.prev.Load()
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
c.prev.Store(curr) c.prev.Store(curr)
var cs fastcache.Stats var cs fastcache.Stats
prev.UpdateStats(&cs) prev.UpdateStats(&cs)
@ -188,8 +188,8 @@ func (c *Cache) prevCacheWatcher() {
c.mu.Unlock() c.mu.Unlock()
return return
} }
prev := c.prev.Load().(*fastcache.Cache) prev := c.prev.Load()
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
var csCurr, csPrev fastcache.Stats var csCurr, csPrev fastcache.Stats
curr.UpdateStats(&csCurr) curr.UpdateStats(&csCurr)
prev.UpdateStats(&csPrev) prev.UpdateStats(&csPrev)
@ -232,7 +232,7 @@ func (c *Cache) cacheSizeWatcher() {
continue continue
} }
var cs fastcache.Stats var cs fastcache.Stats
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
curr.UpdateStats(&cs) curr.UpdateStats(&cs)
if cs.BytesSize >= uint64(0.9*float64(cs.MaxBytesSize)) { if cs.BytesSize >= uint64(0.9*float64(cs.MaxBytesSize)) {
maxBytesSize = cs.MaxBytesSize maxBytesSize = cs.MaxBytesSize
@ -254,8 +254,8 @@ func (c *Cache) cacheSizeWatcher() {
c.mu.Lock() c.mu.Lock()
c.setMode(switching) c.setMode(switching)
prev := c.prev.Load().(*fastcache.Cache) prev := c.prev.Load()
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
c.prev.Store(curr) c.prev.Store(curr)
var cs fastcache.Stats var cs fastcache.Stats
prev.UpdateStats(&cs) prev.UpdateStats(&cs)
@ -273,7 +273,7 @@ func (c *Cache) cacheSizeWatcher() {
case <-t.C: case <-t.C:
} }
var cs fastcache.Stats var cs fastcache.Stats
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
curr.UpdateStats(&cs) curr.UpdateStats(&cs)
if cs.BytesSize >= maxBytesSize { if cs.BytesSize >= maxBytesSize {
break break
@ -282,7 +282,7 @@ func (c *Cache) cacheSizeWatcher() {
c.mu.Lock() c.mu.Lock()
c.setMode(whole) c.setMode(whole)
prev = c.prev.Load().(*fastcache.Cache) prev = c.prev.Load()
c.prev.Store(fastcache.New(1024)) c.prev.Store(fastcache.New(1024))
cs.Reset() cs.Reset()
prev.UpdateStats(&cs) prev.UpdateStats(&cs)
@ -293,7 +293,7 @@ func (c *Cache) cacheSizeWatcher() {
// Save saves the cache to filePath. // Save saves the cache to filePath.
func (c *Cache) Save(filePath string) error { func (c *Cache) Save(filePath string) error {
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
concurrency := cgroup.AvailableCPUs() concurrency := cgroup.AvailableCPUs()
return curr.SaveToFileConcurrent(filePath, concurrency) return curr.SaveToFileConcurrent(filePath, concurrency)
} }
@ -311,10 +311,10 @@ func (c *Cache) Stop() {
// Reset resets the cache. // Reset resets the cache.
func (c *Cache) Reset() { func (c *Cache) Reset() {
var cs fastcache.Stats var cs fastcache.Stats
prev := c.prev.Load().(*fastcache.Cache) prev := c.prev.Load()
prev.UpdateStats(&cs) prev.UpdateStats(&cs)
prev.Reset() prev.Reset()
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
curr.UpdateStats(&cs) curr.UpdateStats(&cs)
updateCacheStatsHistory(&c.csHistory, &cs) updateCacheStatsHistory(&c.csHistory, &cs)
curr.Reset() curr.Reset()
@ -335,11 +335,11 @@ func (c *Cache) UpdateStats(fcs *fastcache.Stats) {
updateCacheStatsHistory(fcs, &c.csHistory) updateCacheStatsHistory(fcs, &c.csHistory)
var cs fastcache.Stats var cs fastcache.Stats
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
curr.UpdateStats(&cs) curr.UpdateStats(&cs)
updateCacheStats(fcs, &cs) updateCacheStats(fcs, &cs)
prev := c.prev.Load().(*fastcache.Cache) prev := c.prev.Load()
cs.Reset() cs.Reset()
prev.UpdateStats(&cs) prev.UpdateStats(&cs)
updateCacheStats(fcs, &cs) updateCacheStats(fcs, &cs)
@ -369,7 +369,7 @@ func updateCacheStatsHistory(dst, src *fastcache.Stats) {
// Get appends the found value for the given key to dst and returns the result. // Get appends the found value for the given key to dst and returns the result.
func (c *Cache) Get(dst, key []byte) []byte { func (c *Cache) Get(dst, key []byte) []byte {
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
result := curr.Get(dst, key) result := curr.Get(dst, key)
if len(result) > len(dst) { if len(result) > len(dst) {
// Fast path - the entry is found in the current cache. // Fast path - the entry is found in the current cache.
@ -381,7 +381,7 @@ func (c *Cache) Get(dst, key []byte) []byte {
} }
// Search for the entry in the previous cache. // Search for the entry in the previous cache.
prev := c.prev.Load().(*fastcache.Cache) prev := c.prev.Load()
result = prev.Get(dst, key) result = prev.Get(dst, key)
if len(result) <= len(dst) { if len(result) <= len(dst) {
// Nothing found. // Nothing found.
@ -394,14 +394,14 @@ func (c *Cache) Get(dst, key []byte) []byte {
// Has verifies whether the cache contains the given key. // Has verifies whether the cache contains the given key.
func (c *Cache) Has(key []byte) bool { func (c *Cache) Has(key []byte) bool {
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
if curr.Has(key) { if curr.Has(key) {
return true return true
} }
if c.loadMode() == whole { if c.loadMode() == whole {
return false return false
} }
prev := c.prev.Load().(*fastcache.Cache) prev := c.prev.Load()
if !prev.Has(key) { if !prev.Has(key) {
return false return false
} }
@ -417,13 +417,13 @@ var tmpBufPool bytesutil.ByteBufferPool
// Set sets the given value for the given key. // Set sets the given value for the given key.
func (c *Cache) Set(key, value []byte) { func (c *Cache) Set(key, value []byte) {
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
curr.Set(key, value) curr.Set(key, value)
} }
// GetBig appends the found value for the given key to dst and returns the result. // GetBig appends the found value for the given key to dst and returns the result.
func (c *Cache) GetBig(dst, key []byte) []byte { func (c *Cache) GetBig(dst, key []byte) []byte {
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
result := curr.GetBig(dst, key) result := curr.GetBig(dst, key)
if len(result) > len(dst) { if len(result) > len(dst) {
// Fast path - the entry is found in the current cache. // Fast path - the entry is found in the current cache.
@ -435,7 +435,7 @@ func (c *Cache) GetBig(dst, key []byte) []byte {
} }
// Search for the entry in the previous cache. // Search for the entry in the previous cache.
prev := c.prev.Load().(*fastcache.Cache) prev := c.prev.Load()
result = prev.GetBig(dst, key) result = prev.GetBig(dst, key)
if len(result) <= len(dst) { if len(result) <= len(dst) {
// Nothing found. // Nothing found.
@ -448,7 +448,7 @@ func (c *Cache) GetBig(dst, key []byte) []byte {
// SetBig sets the given value for the given key. // SetBig sets the given value for the given key.
func (c *Cache) SetBig(key, value []byte) { func (c *Cache) SetBig(key, value []byte) {
curr := c.curr.Load().(*fastcache.Cache) curr := c.curr.Load()
curr.SetBig(key, value) curr.SetBig(key, value)
} }