diff --git a/app/vmstorage/main.go b/app/vmstorage/main.go index 698277502..01235c0c2 100644 --- a/app/vmstorage/main.go +++ b/app/vmstorage/main.go @@ -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()) diff --git a/docs/Cluster-VictoriaMetrics.md b/docs/Cluster-VictoriaMetrics.md index adaf8541e..2ff400a52 100644 --- a/docs/Cluster-VictoriaMetrics.md +++ b/docs/Cluster-VictoriaMetrics.md @@ -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) diff --git a/docs/README.md b/docs/README.md index 1aec6a539..e4885bab0 100644 --- a/docs/README.md +++ b/docs/README.md @@ -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) diff --git a/lib/logstorage/indexdb.go b/lib/logstorage/indexdb.go index 89efb6015..811d2b536 100644 --- a/lib/logstorage/indexdb.go +++ b/lib/logstorage/indexdb.go @@ -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 } diff --git a/lib/mergeset/part.go b/lib/mergeset/part.go index 9fc6e1ed6..da47f1ccc 100644 --- a/lib/mergeset/part.go +++ b/lib/mergeset/part.go @@ -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 { diff --git a/lib/mergeset/part_search.go b/lib/mergeset/part_search.go index 48f854b99..29a0a6723 100644 --- a/lib/mergeset/part_search.go +++ b/lib/mergeset/part_search.go @@ -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 diff --git a/lib/mergeset/part_search_test.go b/lib/mergeset/part_search_test.go index 0cd076ca2..fb141b3f7 100644 --- a/lib/mergeset/part_search_test.go +++ b/lib/mergeset/part_search_test.go @@ -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] diff --git a/lib/mergeset/table.go b/lib/mergeset/table.go index 6559a2c48..f4b7edcdc 100644 --- a/lib/mergeset/table.go +++ b/lib/mergeset/table.go @@ -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()) diff --git a/lib/mergeset/table_search.go b/lib/mergeset/table_search.go index c006f91f3..2da5f0cca 100644 --- a/lib/mergeset/table_search.go +++ b/lib/mergeset/table_search.go @@ -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) } } diff --git a/lib/mergeset/table_search_test.go b/lib/mergeset/table_search_test.go index 5bc2ae02a..79a652ff7 100644 --- a/lib/mergeset/table_search_test.go +++ b/lib/mergeset/table_search_test.go @@ -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", diff --git a/lib/mergeset/table_search_timing_test.go b/lib/mergeset/table_search_timing_test.go index f97d55f5f..f8deb4d17 100644 --- a/lib/mergeset/table_search_timing_test.go +++ b/lib/mergeset/table_search_timing_test.go @@ -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) diff --git a/lib/mergeset/table_test.go b/lib/mergeset/table_test.go index eb474471d..810dd5a04 100644 --- a/lib/mergeset/table_test.go +++ b/lib/mergeset/table_test.go @@ -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 { diff --git a/lib/storage/index_db.go b/lib/storage/index_db.go index a7c4068e5..c935e33b8 100644 --- a/lib/storage/index_db.go +++ b/lib/storage/index_db.go @@ -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.