mirror of
https://github.com/VictoriaMetrics/VictoriaMetrics.git
synced 2024-11-21 14:44:00 +00:00
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:
parent
399dc39ee9
commit
140e7b6b74
12 changed files with 87 additions and 89 deletions
|
@ -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()
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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{}) {
|
||||||
|
|
|
@ -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{}
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in a new issue