Revert "lib/mergeset: add sparse indexdb cache (#7269)"

This reverts commit 837d0d136d.
This commit is contained in:
Zakhar Bessarab 2024-11-04 10:29:14 -03:00
parent 5733e56e40
commit 9f9cc24e4c
No known key found for this signature in database
GPG key ID: 932B34D6FE062023
13 changed files with 16 additions and 122 deletions

View file

@ -67,8 +67,6 @@ var (
"See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning")
cacheSizeIndexDBDataBlocks = flagutil.NewBytes("storage.cacheSizeIndexDBDataBlocks", 0, "Overrides max size for indexdb/dataBlocks cache. "+
"See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning")
cacheSizeIndexDBDataBlocksSparse = flagutil.NewBytes("storage.cacheSizeIndexDBDataBlocksSparse", 0, "Overrides max size for indexdb/dataBlocksSparse cache. "+
"See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning")
cacheSizeIndexDBTagFilters = flagutil.NewBytes("storage.cacheSizeIndexDBTagFilters", 0, "Overrides max size for indexdb/tagFiltersToMetricIDs cache. "+
"See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning")
)
@ -102,7 +100,6 @@ func Init(resetCacheIfNeeded func(mrs []storage.MetricRow)) {
storage.SetTagFiltersCacheSize(cacheSizeIndexDBTagFilters.IntN())
mergeset.SetIndexBlocksCacheSize(cacheSizeIndexDBIndexBlocks.IntN())
mergeset.SetDataBlocksCacheSize(cacheSizeIndexDBDataBlocks.IntN())
mergeset.SetDataBlocksSparseCacheSize(cacheSizeIndexDBDataBlocksSparse.IntN())
if retentionPeriod.Duration() < 24*time.Hour {
logger.Fatalf("-retentionPeriod cannot be smaller than a day; got %s", retentionPeriod)
@ -584,7 +581,6 @@ func writeStorageMetrics(w io.Writer, strg *storage.Storage) {
metrics.WriteGaugeUint64(w, `vm_cache_entries{type="storage/next_day_metric_ids"}`, m.NextDayMetricIDCacheSize)
metrics.WriteGaugeUint64(w, `vm_cache_entries{type="storage/indexBlocks"}`, tm.IndexBlocksCacheSize)
metrics.WriteGaugeUint64(w, `vm_cache_entries{type="indexdb/dataBlocks"}`, idbm.DataBlocksCacheSize)
metrics.WriteGaugeUint64(w, `vm_cache_entries{type="indexdb/dataBlocksSparse"}`, idbm.DataBlocksSparseCacheSize)
metrics.WriteGaugeUint64(w, `vm_cache_entries{type="indexdb/indexBlocks"}`, idbm.IndexBlocksCacheSize)
metrics.WriteGaugeUint64(w, `vm_cache_entries{type="indexdb/tagFiltersToMetricIDs"}`, idbm.TagFiltersToMetricIDsCacheSize)
metrics.WriteGaugeUint64(w, `vm_cache_entries{type="storage/regexps"}`, uint64(storage.RegexpCacheSize()))
@ -596,7 +592,6 @@ func writeStorageMetrics(w io.Writer, strg *storage.Storage) {
metrics.WriteGaugeUint64(w, `vm_cache_size_bytes{type="storage/metricName"}`, m.MetricNameCacheSizeBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_bytes{type="storage/indexBlocks"}`, tm.IndexBlocksCacheSizeBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_bytes{type="indexdb/dataBlocks"}`, idbm.DataBlocksCacheSizeBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_bytes{type="indexdb/dataBlocksSparse"}`, idbm.DataBlocksSparseCacheSizeBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_bytes{type="indexdb/indexBlocks"}`, idbm.IndexBlocksCacheSizeBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_bytes{type="storage/date_metricID"}`, m.DateMetricIDCacheSizeBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_bytes{type="storage/hour_metric_ids"}`, m.HourMetricIDCacheSizeBytes)
@ -611,7 +606,6 @@ func writeStorageMetrics(w io.Writer, strg *storage.Storage) {
metrics.WriteGaugeUint64(w, `vm_cache_size_max_bytes{type="storage/metricName"}`, m.MetricNameCacheSizeMaxBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_max_bytes{type="storage/indexBlocks"}`, tm.IndexBlocksCacheSizeMaxBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_max_bytes{type="indexdb/dataBlocks"}`, idbm.DataBlocksCacheSizeMaxBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_max_bytes{type="indexdb/dataBlocksSparse"}`, idbm.DataBlocksSparseCacheSizeMaxBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_max_bytes{type="indexdb/indexBlocks"}`, idbm.IndexBlocksCacheSizeMaxBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_max_bytes{type="indexdb/tagFiltersToMetricIDs"}`, idbm.TagFiltersToMetricIDsCacheSizeMaxBytes)
metrics.WriteGaugeUint64(w, `vm_cache_size_max_bytes{type="storage/regexps"}`, uint64(storage.RegexpCacheMaxSizeBytes()))
@ -622,7 +616,6 @@ func writeStorageMetrics(w io.Writer, strg *storage.Storage) {
metrics.WriteCounterUint64(w, `vm_cache_requests_total{type="storage/metricName"}`, m.MetricNameCacheRequests)
metrics.WriteCounterUint64(w, `vm_cache_requests_total{type="storage/indexBlocks"}`, tm.IndexBlocksCacheRequests)
metrics.WriteCounterUint64(w, `vm_cache_requests_total{type="indexdb/dataBlocks"}`, idbm.DataBlocksCacheRequests)
metrics.WriteCounterUint64(w, `vm_cache_requests_total{type="indexdb/dataBlocksSparse"}`, idbm.DataBlocksSparseCacheRequests)
metrics.WriteCounterUint64(w, `vm_cache_requests_total{type="indexdb/indexBlocks"}`, idbm.IndexBlocksCacheRequests)
metrics.WriteCounterUint64(w, `vm_cache_requests_total{type="indexdb/tagFiltersToMetricIDs"}`, idbm.TagFiltersToMetricIDsCacheRequests)
metrics.WriteCounterUint64(w, `vm_cache_requests_total{type="storage/regexps"}`, storage.RegexpCacheRequests())
@ -633,7 +626,6 @@ func writeStorageMetrics(w io.Writer, strg *storage.Storage) {
metrics.WriteCounterUint64(w, `vm_cache_misses_total{type="storage/metricName"}`, m.MetricNameCacheMisses)
metrics.WriteCounterUint64(w, `vm_cache_misses_total{type="storage/indexBlocks"}`, tm.IndexBlocksCacheMisses)
metrics.WriteCounterUint64(w, `vm_cache_misses_total{type="indexdb/dataBlocks"}`, idbm.DataBlocksCacheMisses)
metrics.WriteCounterUint64(w, `vm_cache_misses_total{type="indexdb/dataBlocksSparse"}`, idbm.DataBlocksSparseCacheMisses)
metrics.WriteCounterUint64(w, `vm_cache_misses_total{type="indexdb/indexBlocks"}`, idbm.IndexBlocksCacheMisses)
metrics.WriteCounterUint64(w, `vm_cache_misses_total{type="indexdb/tagFiltersToMetricIDs"}`, idbm.TagFiltersToMetricIDsCacheMisses)
metrics.WriteCounterUint64(w, `vm_cache_misses_total{type="storage/regexps"}`, storage.RegexpCacheMisses())

View file

@ -1910,9 +1910,6 @@ Below is the output for `/path/to/vmstorage -help`:
-storage.cacheSizeIndexDBDataBlocks size
Overrides max size for indexdb/dataBlocks cache. See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning
Supports the following optional suffixes for size values: KB, MB, GB, TB, KiB, MiB, GiB, TiB (default 0)
-storage.cacheSizeIndexDBDataBlocksSparse size
Overrides max size for indexdb/dataBlocksSparse cache. See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning
Supports the following optional suffixes for size values: KB, MB, GB, TB, KiB, MiB, GiB, TiB (default 0)
-storage.cacheSizeIndexDBIndexBlocks size
Overrides max size for indexdb/indexBlocks cache. See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning
Supports the following optional suffixes for size values: KB, MB, GB, TB, KiB, MiB, GiB, TiB (default 0)

View file

@ -3267,9 +3267,6 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li
-storage.cacheSizeIndexDBDataBlocks size
Overrides max size for indexdb/dataBlocks cache. See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning
Supports the following optional suffixes for size values: KB, MB, GB, TB, KiB, MiB, GiB, TiB (default 0)
-storage.cacheSizeIndexDBDataBlocksSparse size
Overrides max size for indexdb/dataBlocksSparse cache. See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning
Supports the following optional suffixes for size values: KB, MB, GB, TB, KiB, MiB, GiB, TiB (default 0)
-storage.cacheSizeIndexDBIndexBlocks size
Overrides max size for indexdb/indexBlocks cache. See https://docs.victoriametrics.com/single-server-victoriametrics/#cache-tuning
Supports the following optional suffixes for size values: KB, MB, GB, TB, KiB, MiB, GiB, TiB (default 0)

View file

@ -166,7 +166,7 @@ func (idb *indexdb) getIndexSearch() *indexSearch {
}
}
is := v.(*indexSearch)
is.ts.Init(idb.tb, false)
is.ts.Init(idb.tb)
return is
}

View file

@ -14,7 +14,6 @@ import (
var idxbCache = blockcache.NewCache(getMaxIndexBlocksCacheSize)
var ibCache = blockcache.NewCache(getMaxInmemoryBlocksCacheSize)
var ibSparseCache = blockcache.NewCache(getMaxInmemoryBlocksSparseCacheSize)
// SetIndexBlocksCacheSize overrides the default size of indexdb/indexBlocks cache
func SetIndexBlocksCacheSize(size int) {
@ -43,32 +42,15 @@ func SetDataBlocksCacheSize(size int) {
func getMaxInmemoryBlocksCacheSize() int {
maxInmemoryBlockCacheSizeOnce.Do(func() {
if maxInmemoryBlockCacheSize <= 0 {
maxInmemoryBlockCacheSize = int(0.20 * float64(memory.Allowed()))
maxInmemoryBlockCacheSize = int(0.25 * float64(memory.Allowed()))
}
})
return maxInmemoryBlockCacheSize
}
// SetDataBlocksSparseCacheSize overrides the default size of indexdb/dataBlocksSparse cache
func SetDataBlocksSparseCacheSize(size int) {
maxInmemorySparseMergeCacheSize = size
}
func getMaxInmemoryBlocksSparseCacheSize() int {
maxInmemoryBlockSparseCacheSizeOnce.Do(func() {
if maxInmemorySparseMergeCacheSize <= 0 {
maxInmemorySparseMergeCacheSize = int(0.05 * float64(memory.Allowed()))
}
})
return maxInmemorySparseMergeCacheSize
}
var (
maxInmemoryBlockCacheSize int
maxInmemoryBlockCacheSizeOnce sync.Once
maxInmemorySparseMergeCacheSize int
maxInmemoryBlockSparseCacheSizeOnce sync.Once
)
type part struct {
@ -136,7 +118,6 @@ func (p *part) MustClose() {
idxbCache.RemoveBlocksForPart(p)
ibCache.RemoveBlocksForPart(p)
ibSparseCache.RemoveBlocksForPart(p)
}
type indexBlock struct {

View file

@ -36,8 +36,6 @@ type partSearch struct {
ib *inmemoryBlock
ibItemIdx int
sparse bool
}
func (ps *partSearch) reset() {
@ -59,11 +57,10 @@ func (ps *partSearch) reset() {
// Init initializes ps for search in the p.
//
// Use Seek for search in p.
func (ps *partSearch) Init(p *part, sparse bool) {
func (ps *partSearch) Init(p *part) {
ps.reset()
ps.p = p
ps.sparse = sparse
}
// Seek seeks for the first item greater or equal to k in ps.
@ -302,22 +299,18 @@ func (ps *partSearch) readIndexBlock(mr *metaindexRow) (*indexBlock, error) {
}
func (ps *partSearch) getInmemoryBlock(bh *blockHeader) (*inmemoryBlock, error) {
cache := ibCache
if ps.sparse {
cache = ibSparseCache
}
ibKey := blockcache.Key{
Part: ps.p,
Offset: bh.itemsBlockOffset,
}
b := cache.GetBlock(ibKey)
b := ibCache.GetBlock(ibKey)
if b == nil {
ib, err := ps.readInmemoryBlock(bh)
if err != nil {
return nil, err
}
b = ib
cache.PutBlock(ibKey, b)
ibCache.PutBlock(ibKey, b)
}
ib := b.(*inmemoryBlock)
return ib, nil

View file

@ -54,7 +54,7 @@ func testPartSearchConcurrent(p *part, items []string) error {
func testPartSearchSerial(r *rand.Rand, p *part, items []string) error {
var ps partSearch
ps.Init(p, true)
ps.Init(p)
var k []byte
// Search for the item smaller than the items[0]

View file

@ -574,12 +574,6 @@ type TableMetrics struct {
DataBlocksCacheRequests uint64
DataBlocksCacheMisses uint64
DataBlocksSparseCacheSize uint64
DataBlocksSparseCacheSizeBytes uint64
DataBlocksSparseCacheSizeMaxBytes uint64
DataBlocksSparseCacheRequests uint64
DataBlocksSparseCacheMisses uint64
IndexBlocksCacheSize uint64
IndexBlocksCacheSizeBytes uint64
IndexBlocksCacheSizeMaxBytes uint64
@ -641,12 +635,6 @@ func (tb *Table) UpdateMetrics(m *TableMetrics) {
m.DataBlocksCacheRequests = ibCache.Requests()
m.DataBlocksCacheMisses = ibCache.Misses()
m.DataBlocksSparseCacheSize = uint64(ibSparseCache.Len())
m.DataBlocksSparseCacheSizeBytes = uint64(ibSparseCache.SizeBytes())
m.DataBlocksSparseCacheSizeMaxBytes = uint64(ibSparseCache.SizeMaxBytes())
m.DataBlocksSparseCacheRequests = ibSparseCache.Requests()
m.DataBlocksSparseCacheMisses = ibSparseCache.Misses()
m.IndexBlocksCacheSize = uint64(idxbCache.Len())
m.IndexBlocksCacheSizeBytes = uint64(idxbCache.SizeBytes())
m.IndexBlocksCacheSizeMaxBytes = uint64(idxbCache.SizeMaxBytes())

View file

@ -59,7 +59,7 @@ func (ts *TableSearch) reset() {
// Init initializes ts for searching in the tb.
//
// MustClose must be called when the ts is no longer needed.
func (ts *TableSearch) Init(tb *Table, sparse bool) {
func (ts *TableSearch) Init(tb *Table) {
if ts.needClosing {
logger.Panicf("BUG: missing MustClose call before the next call to Init")
}
@ -74,7 +74,7 @@ func (ts *TableSearch) Init(tb *Table, sparse bool) {
// Initialize the psPool.
ts.psPool = slicesutil.SetLength(ts.psPool, len(ts.pws))
for i, pw := range ts.pws {
ts.psPool[i].Init(pw.p, sparse)
ts.psPool[i].Init(pw.p)
}
}

View file

@ -107,7 +107,7 @@ func testTableSearchConcurrent(tb *Table, items []string) error {
func testTableSearchSerial(tb *Table, items []string) error {
var ts TableSearch
ts.Init(tb, false)
ts.Init(tb)
for _, key := range []string{
"",
"123",

View file

@ -83,7 +83,7 @@ func benchmarkTableSearchKeysExt(b *testing.B, tb *Table, keys [][]byte, stripSu
b.RunParallel(func(pb *testing.PB) {
r := rand.New(rand.NewSource(1))
var ts TableSearch
ts.Init(tb, false)
ts.Init(tb)
defer ts.MustClose()
for pb.Next() {
startIdx := r.Intn(len(keys) - searchKeysCount)

View file

@ -139,9 +139,9 @@ func TestTableCreateSnapshotAt(t *testing.T) {
tb2 := MustOpenTable(snapshot2, nil, nil, &isReadOnly)
var ts, ts1, ts2 TableSearch
ts.Init(tb, false)
ts1.Init(tb1, false)
ts2.Init(tb2, false)
ts.Init(tb)
ts1.Init(tb1)
ts2.Init(tb2)
for i := 0; i < itemsCount; i++ {
key := []byte(fmt.Sprintf("item %d", i))
if err := ts.FirstItemWithPrefix(key); err != nil {

View file

@ -14,9 +14,6 @@ import (
"time"
"unsafe"
"github.com/VictoriaMetrics/fastcache"
"github.com/cespare/xxhash/v2"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
@ -28,6 +25,8 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/uint64set"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/workingsetcache"
"github.com/VictoriaMetrics/fastcache"
"github.com/cespare/xxhash/v2"
)
const (
@ -521,21 +520,7 @@ type indexSearch struct {
deadline uint64
}
// getIndexSearch returns an indexSearch with default configuration
func (db *indexDB) getIndexSearch(deadline uint64) *indexSearch {
return db.getIndexSearchInternal(deadline, false)
}
// getIndexSearchSparse returns an indexSearch with sparse cache
// It is useful for search operations that can scan through the large amount index entries
// Without the need to keep all the entries in the caches used for queries
// used in ENT version
// nolint:unused
func (db *indexDB) getIndexSearchSparse(deadline uint64) *indexSearch {
return db.getIndexSearchInternal(deadline, true)
}
func (db *indexDB) getIndexSearchInternal(deadline uint64, sparse bool) *indexSearch {
v := db.indexSearchPool.Get()
if v == nil {
v = &indexSearch{
@ -543,7 +528,7 @@ func (db *indexDB) getIndexSearchInternal(deadline uint64, sparse bool) *indexSe
}
}
is := v.(*indexSearch)
is.ts.Init(db.tb, sparse)
is.ts.Init(db.tb)
is.deadline = deadline
return is
}
@ -1579,45 +1564,6 @@ func (db *indexDB) searchMetricNameWithCache(dst []byte, metricID uint64) ([]byt
return dst, false
}
// searchMetricNameWithoutCache appends metric name for the given metricID to dst
// and returns the result.
// It does not cache the result and uses sparse cache for index scan.
// used in ENT version
// nolint:unused
func (db *indexDB) searchMetricNameWithoutCache(dst []byte, metricID uint64) ([]byte, bool) {
is := db.getIndexSearchSparse(noDeadline)
var ok bool
dst, ok = is.searchMetricName(dst, metricID)
db.putIndexSearch(is)
if ok {
// There is no need in verifying whether the given metricID is deleted,
// since the filtering must be performed before calling this func.
return dst, true
}
// Try searching in the external indexDB.
db.doExtDB(func(extDB *indexDB) {
is := extDB.getIndexSearchSparse(noDeadline)
dst, ok = is.searchMetricName(dst, metricID)
extDB.putIndexSearch(is)
})
if ok {
return dst, true
}
if db.s.wasMetricIDMissingBefore(metricID) {
// Cannot find the MetricName for the given metricID for the last 60 seconds.
// It is likely the indexDB contains incomplete set of metricID -> metricName entries
// after unclean shutdown or after restoring from a snapshot.
// Mark the metricID as deleted, so it is created again when new sample
// for the given time series is ingested next time.
db.missingMetricNamesForMetricID.Add(1)
db.deleteMetricIDs([]uint64{metricID})
}
return dst, false
}
// DeleteTSIDs marks as deleted all the TSIDs matching the given tfss and
// updates or resets all caches where TSIDs and the corresponding MetricIDs may
// be stored.