mirror of
https://github.com/VictoriaMetrics/VictoriaMetrics.git
synced 2026-05-19 09:46:57 +03:00
lib/storage: implement partition index (#8134)
This should reduce disk space occupied by indexDBs as they get deleted along with the corresponding partitions once those partitions become outside the retention window. - Motivation: https://github.com/VictoriaMetrics/VictoriaMetrics/issues/7599 - What to expect: https://github.com/VictoriaMetrics/VictoriaMetrics/issues/8134 Signed-off-by: Artem Fetishev <rtm@victoriametrics.com> Co-authored-by: Andrei Baidarov <baidarov@nebius.com>
This commit is contained in:
18
Makefile
18
Makefile
@@ -471,7 +471,23 @@ integration-test:
|
||||
|
||||
apptest:
|
||||
$(MAKE) victoria-metrics vmagent vmalert vmauth vmctl vmbackup vmrestore
|
||||
go test ./apptest/... -skip="^TestCluster.*"
|
||||
go test ./apptest/... -skip="^Test(Cluster|Legacy).*"
|
||||
|
||||
integration-test-legacy: victoria-metrics vmbackup vmrestore
|
||||
OS=$$(uname | tr '[:upper:]' '[:lower:]'); \
|
||||
ARCH=$$(uname -m | tr '[:upper:]' '[:lower:]' | sed 's/x86_64/amd64/'); \
|
||||
VERSION=v1.132.0; \
|
||||
VMSINGLE=victoria-metrics-$${OS}-$${ARCH}-$${VERSION}.tar.gz; \
|
||||
VMCLUSTER=victoria-metrics-$${OS}-$${ARCH}-$${VERSION}-cluster.tar.gz; \
|
||||
URL=https://github.com/VictoriaMetrics/VictoriaMetrics/releases/download/$${VERSION}; \
|
||||
DIR=/tmp/$${VERSION}; \
|
||||
test -d $${DIR} || (mkdir $${DIR} && \
|
||||
curl --output-dir /tmp -LO $${URL}/$${VMSINGLE} && tar xzf /tmp/$${VMSINGLE} -C $${DIR} && \
|
||||
curl --output-dir /tmp -LO $${URL}/$${VMCLUSTER} && tar xzf /tmp/$${VMCLUSTER} -C $${DIR} \
|
||||
); \
|
||||
VM_LEGACY_VMSINGLE_PATH=$${DIR}/victoria-metrics-prod \
|
||||
VM_LEGACY_VMSTORAGE_PATH=$${DIR}/vmstorage-prod \
|
||||
go test ./apptest/tests -run="^TestLegacySingle.*"
|
||||
|
||||
benchmark:
|
||||
GOEXPERIMENT=synctest go test -bench=. ./lib/...
|
||||
|
||||
@@ -118,7 +118,7 @@ func Init(resetCacheIfNeeded func(mrs []storage.MetricRow)) {
|
||||
}
|
||||
|
||||
resetResponseCacheIfNeeded = resetCacheIfNeeded
|
||||
storage.SetRetentionTimezoneOffset(*retentionTimezoneOffset)
|
||||
storage.LegacySetRetentionTimezoneOffset(*retentionTimezoneOffset)
|
||||
storage.SetFreeDiskSpaceLimit(minFreeDiskSpaceBytes.N)
|
||||
storage.SetTSIDCacheSize(cacheSizeStorageTSID.IntN())
|
||||
storage.SetTagFiltersCacheSize(cacheSizeIndexDBTagFilters.IntN())
|
||||
@@ -500,7 +500,7 @@ func writeStorageMetrics(w io.Writer, strg *storage.Storage) {
|
||||
var m storage.Metrics
|
||||
strg.UpdateMetrics(&m)
|
||||
tm := &m.TableMetrics
|
||||
idbm := &m.IndexDBMetrics
|
||||
idbm := &m.TableMetrics.IndexDBMetrics
|
||||
|
||||
metrics.WriteGaugeUint64(w, fmt.Sprintf(`vm_free_disk_space_bytes{path=%q}`, *DataPath), fs.MustGetFreeSpace(*DataPath))
|
||||
metrics.WriteGaugeUint64(w, fmt.Sprintf(`vm_free_disk_space_limit_bytes{path=%q}`, *DataPath), uint64(minFreeDiskSpaceBytes.N))
|
||||
|
||||
898
apptest/tests/legacy_indexdb_test.go
Normal file
898
apptest/tests/legacy_indexdb_test.go
Normal file
@@ -0,0 +1,898 @@
|
||||
package tests
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"slices"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
at "github.com/VictoriaMetrics/VictoriaMetrics/apptest"
|
||||
)
|
||||
|
||||
var (
|
||||
legacyVmsinglePath = os.Getenv("VM_LEGACY_VMSINGLE_PATH")
|
||||
legacyVmstoragePath = os.Getenv("VM_LEGACY_VMSTORAGE_PATH")
|
||||
)
|
||||
|
||||
type testLegacyDeleteSeriesOpts struct {
|
||||
startLegacySUT func() at.PrometheusWriteQuerier
|
||||
startNewSUT func() at.PrometheusWriteQuerier
|
||||
stopLegacySUT func()
|
||||
stopNewSUT func()
|
||||
}
|
||||
|
||||
func TestLegacySingleDeleteSeries(t *testing.T) {
|
||||
tc := at.NewTestCase(t)
|
||||
defer tc.Stop()
|
||||
|
||||
storageDataPath := filepath.Join(tc.Dir(), "vmsingle")
|
||||
|
||||
opts := testLegacyDeleteSeriesOpts{
|
||||
startLegacySUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartVmsingleAt("vmsingle-legacy", legacyVmsinglePath, []string{
|
||||
"-storageDataPath=" + storageDataPath,
|
||||
"-retentionPeriod=100y",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
})
|
||||
},
|
||||
startNewSUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartVmsingle("vmsingle-new", []string{
|
||||
"-storageDataPath=" + storageDataPath,
|
||||
"-retentionPeriod=100y",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
})
|
||||
},
|
||||
stopLegacySUT: func() {
|
||||
tc.StopApp("vmsingle-legacy")
|
||||
},
|
||||
stopNewSUT: func() {
|
||||
tc.StopApp("vmsingle-new")
|
||||
},
|
||||
}
|
||||
|
||||
testLegacyDeleteSeries(tc, opts)
|
||||
}
|
||||
|
||||
func TestLegacyClusterDeleteSeries(t *testing.T) {
|
||||
tc := at.NewTestCase(t)
|
||||
defer tc.Stop()
|
||||
|
||||
storage1DataPath := filepath.Join(tc.Dir(), "vmstorage1")
|
||||
storage2DataPath := filepath.Join(tc.Dir(), "vmstorage2")
|
||||
|
||||
opts := testLegacyDeleteSeriesOpts{
|
||||
startLegacySUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartCluster(&at.ClusterOptions{
|
||||
Vmstorage1Instance: "vmstorage1-legacy",
|
||||
Vmstorage1Binary: legacyVmstoragePath,
|
||||
Vmstorage1Flags: []string{
|
||||
"-storageDataPath=" + storage1DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
Vmstorage2Instance: "vmstorage2-legacy",
|
||||
Vmstorage2Binary: legacyVmstoragePath,
|
||||
Vmstorage2Flags: []string{
|
||||
"-storageDataPath=" + storage2DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
VminsertInstance: "vminsert",
|
||||
VminsertFlags: []string{},
|
||||
VmselectInstance: "vmselect",
|
||||
VmselectFlags: []string{
|
||||
"-search.maxStalenessInterval=1m",
|
||||
},
|
||||
})
|
||||
},
|
||||
startNewSUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartCluster(&at.ClusterOptions{
|
||||
Vmstorage1Instance: "vmstorage1-new",
|
||||
Vmstorage1Flags: []string{
|
||||
"-storageDataPath=" + storage1DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
Vmstorage2Instance: "vmstorage2-new",
|
||||
Vmstorage2Flags: []string{
|
||||
"-storageDataPath=" + storage2DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
VminsertInstance: "vminsert",
|
||||
VminsertFlags: []string{},
|
||||
VmselectInstance: "vmselect",
|
||||
VmselectFlags: []string{
|
||||
"-search.maxStalenessInterval=1m",
|
||||
},
|
||||
})
|
||||
},
|
||||
stopLegacySUT: func() {
|
||||
tc.StopApp("vminsert")
|
||||
tc.StopApp("vmselect")
|
||||
tc.StopApp("vmstorage1-legacy")
|
||||
tc.StopApp("vmstorage2-legacy")
|
||||
},
|
||||
stopNewSUT: func() {
|
||||
tc.StopApp("vminsert")
|
||||
tc.StopApp("vmselect")
|
||||
tc.StopApp("vmstorage1-new")
|
||||
tc.StopApp("vmstorage2-new")
|
||||
},
|
||||
}
|
||||
|
||||
testLegacyDeleteSeries(tc, opts)
|
||||
}
|
||||
|
||||
func testLegacyDeleteSeries(tc *at.TestCase, opts testLegacyDeleteSeriesOpts) {
|
||||
t := tc.T()
|
||||
|
||||
type want struct {
|
||||
series []map[string]string
|
||||
queryResults []*at.QueryResult
|
||||
}
|
||||
|
||||
genData := func(prefix string, start, end, step int64, value float64) (recs []string, w *want) {
|
||||
count := (end - start) / step
|
||||
recs = make([]string, count)
|
||||
w = &want{
|
||||
series: make([]map[string]string, count),
|
||||
queryResults: make([]*at.QueryResult, count),
|
||||
}
|
||||
for i := range count {
|
||||
name := fmt.Sprintf("%s_%03d", prefix, i)
|
||||
timestamp := start + int64(i)*step
|
||||
|
||||
recs[i] = fmt.Sprintf("%s %f %d", name, value, timestamp)
|
||||
w.series[i] = map[string]string{"__name__": name}
|
||||
w.queryResults[i] = &at.QueryResult{
|
||||
Metric: map[string]string{"__name__": name},
|
||||
Samples: []*at.Sample{{Timestamp: timestamp, Value: value}},
|
||||
}
|
||||
}
|
||||
return recs, w
|
||||
}
|
||||
|
||||
assertSearchResults := func(app at.PrometheusQuerier, query string, start, end int64, step string, want *want) {
|
||||
t.Helper()
|
||||
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/series response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1Series(t, query, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
}).Sort()
|
||||
},
|
||||
Want: &at.PrometheusAPIV1SeriesResponse{
|
||||
Status: "success",
|
||||
Data: want.series,
|
||||
},
|
||||
FailNow: true,
|
||||
})
|
||||
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/query_range response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1QueryRange(t, query, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
Step: step,
|
||||
})
|
||||
},
|
||||
Want: &at.PrometheusAPIV1QueryResponse{
|
||||
Status: "success",
|
||||
Data: &at.QueryData{
|
||||
ResultType: "matrix",
|
||||
Result: want.queryResults,
|
||||
},
|
||||
},
|
||||
FailNow: true,
|
||||
})
|
||||
}
|
||||
|
||||
// - start legacy vmsingle
|
||||
// - insert data1
|
||||
// - confirm that metric names and samples are searcheable
|
||||
// - stop legacy vmsingle
|
||||
const step = 24 * 3600 * 1000 // 24h
|
||||
start1 := time.Date(2000, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
end1 := time.Date(2000, 1, 10, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
data1, want1 := genData("metric", start1, end1, step, 1)
|
||||
legacySUT := opts.startLegacySUT()
|
||||
legacySUT.PrometheusAPIV1ImportPrometheus(t, data1, at.QueryOpts{})
|
||||
legacySUT.ForceFlush(t)
|
||||
assertSearchResults(legacySUT, `{__name__=~".*"}`, start1, end1, "1d", want1)
|
||||
opts.stopLegacySUT()
|
||||
|
||||
// - start new vmsingle
|
||||
// - confirm that data1 metric names and samples are searcheable
|
||||
// - delete data1
|
||||
// - confirm that data1 metric names and samples are not searcheable anymore
|
||||
// - insert data2 (same metric names, different dates)
|
||||
// - confirm that metric names become searcheable again
|
||||
// - confirm that data1 samples are not searchable and data2 samples are searcheable
|
||||
|
||||
newSUT := opts.startNewSUT()
|
||||
assertSearchResults(newSUT, `{__name__=~".*"}`, start1, end1, "1d", want1)
|
||||
|
||||
newSUT.APIV1AdminTSDBDeleteSeries(t, `{__name__=~".*"}`, at.QueryOpts{})
|
||||
wantNoResults := &want{
|
||||
series: []map[string]string{},
|
||||
queryResults: []*at.QueryResult{},
|
||||
}
|
||||
assertSearchResults(newSUT, `{__name__=~".*"}`, start1, end1, "1d", wantNoResults)
|
||||
|
||||
start2 := time.Date(2000, 1, 11, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
end2 := time.Date(2000, 1, 20, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
data2, want2 := genData("metric", start2, end2, step, 2)
|
||||
newSUT.PrometheusAPIV1ImportPrometheus(t, data2, at.QueryOpts{})
|
||||
newSUT.ForceFlush(t)
|
||||
assertSearchResults(newSUT, `{__name__=~".*"}`, start1, end2, "1d", want2)
|
||||
|
||||
// - restart new vmsingle
|
||||
// - confirm that metric names still searchable, data1 samples are not
|
||||
// searchable, and data2 samples are searcheable
|
||||
|
||||
opts.stopNewSUT()
|
||||
newSUT = opts.startNewSUT()
|
||||
assertSearchResults(newSUT, `{__name__=~".*"}`, start1, end2, "1d", want2)
|
||||
opts.stopNewSUT()
|
||||
}
|
||||
|
||||
type testLegacyBackupRestoreOpts struct {
|
||||
startLegacySUT func() at.PrometheusWriteQuerier
|
||||
startNewSUT func() at.PrometheusWriteQuerier
|
||||
stopLegacySUT func()
|
||||
stopNewSUT func()
|
||||
storageDataPaths []string
|
||||
snapshotCreateURLs func(at.PrometheusWriteQuerier) []string
|
||||
}
|
||||
|
||||
func TestLegacySingleBackupRestore(t *testing.T) {
|
||||
tc := at.NewTestCase(t)
|
||||
defer tc.Stop()
|
||||
|
||||
storageDataPath := filepath.Join(tc.Dir(), "vmsingle")
|
||||
|
||||
opts := testLegacyBackupRestoreOpts{
|
||||
startLegacySUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartVmsingleAt("vmsingle-legacy", legacyVmsinglePath, []string{
|
||||
"-storageDataPath=" + storageDataPath,
|
||||
"-retentionPeriod=100y",
|
||||
"-search.disableCache=true",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
})
|
||||
},
|
||||
startNewSUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartVmsingle("vmsingle-new", []string{
|
||||
"-storageDataPath=" + storageDataPath,
|
||||
"-retentionPeriod=100y",
|
||||
"-search.disableCache=true",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
})
|
||||
},
|
||||
stopLegacySUT: func() {
|
||||
tc.StopApp("vmsingle-legacy")
|
||||
},
|
||||
stopNewSUT: func() {
|
||||
tc.StopApp("vmsingle-new")
|
||||
},
|
||||
storageDataPaths: []string{
|
||||
storageDataPath,
|
||||
},
|
||||
snapshotCreateURLs: func(sut at.PrometheusWriteQuerier) []string {
|
||||
return []string{
|
||||
sut.(*at.Vmsingle).SnapshotCreateURL(),
|
||||
}
|
||||
},
|
||||
}
|
||||
|
||||
testLegacyBackupRestore(tc, opts)
|
||||
}
|
||||
|
||||
func TestLegacyClusterBackupRestore(t *testing.T) {
|
||||
tc := at.NewTestCase(t)
|
||||
defer tc.Stop()
|
||||
|
||||
storage1DataPath := filepath.Join(tc.Dir(), "vmstorage1")
|
||||
storage2DataPath := filepath.Join(tc.Dir(), "vmstorage2")
|
||||
|
||||
opts := testLegacyBackupRestoreOpts{
|
||||
startLegacySUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartCluster(&at.ClusterOptions{
|
||||
Vmstorage1Instance: "vmstorage1-legacy",
|
||||
Vmstorage1Binary: legacyVmstoragePath,
|
||||
Vmstorage1Flags: []string{
|
||||
"-storageDataPath=" + storage1DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
Vmstorage2Instance: "vmstorage2-legacy",
|
||||
Vmstorage2Binary: legacyVmstoragePath,
|
||||
Vmstorage2Flags: []string{
|
||||
"-storageDataPath=" + storage2DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
VminsertInstance: "vminsert",
|
||||
VminsertFlags: []string{},
|
||||
VmselectInstance: "vmselect",
|
||||
VmselectFlags: []string{
|
||||
"-search.disableCache=true",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
},
|
||||
})
|
||||
},
|
||||
startNewSUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartCluster(&at.ClusterOptions{
|
||||
Vmstorage1Instance: "vmstorage1-new",
|
||||
Vmstorage1Flags: []string{
|
||||
"-storageDataPath=" + storage1DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
Vmstorage2Instance: "vmstorage2-new",
|
||||
Vmstorage2Flags: []string{
|
||||
"-storageDataPath=" + storage2DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
VminsertInstance: "vminsert",
|
||||
VmselectInstance: "vmselect",
|
||||
VmselectFlags: []string{
|
||||
"-search.disableCache=true",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
},
|
||||
})
|
||||
},
|
||||
stopLegacySUT: func() {
|
||||
tc.StopApp("vminsert")
|
||||
tc.StopApp("vmselect")
|
||||
tc.StopApp("vmstorage1-legacy")
|
||||
tc.StopApp("vmstorage2-legacy")
|
||||
},
|
||||
stopNewSUT: func() {
|
||||
tc.StopApp("vminsert")
|
||||
tc.StopApp("vmselect")
|
||||
tc.StopApp("vmstorage1-new")
|
||||
tc.StopApp("vmstorage2-new")
|
||||
},
|
||||
storageDataPaths: []string{
|
||||
storage1DataPath,
|
||||
storage2DataPath,
|
||||
},
|
||||
snapshotCreateURLs: func(sut at.PrometheusWriteQuerier) []string {
|
||||
c := sut.(*at.Vmcluster)
|
||||
return []string{
|
||||
c.Vmstorages[0].SnapshotCreateURL(),
|
||||
c.Vmstorages[1].SnapshotCreateURL(),
|
||||
}
|
||||
},
|
||||
}
|
||||
|
||||
testLegacyBackupRestore(tc, opts)
|
||||
}
|
||||
|
||||
func testLegacyBackupRestore(tc *at.TestCase, opts testLegacyBackupRestoreOpts) {
|
||||
t := tc.T()
|
||||
|
||||
const msecPerMinute = 60 * 1000
|
||||
// Use the same number of metrics and time range for all the data ingestions
|
||||
// below.
|
||||
const numMetrics = 1000
|
||||
start := time.Date(2025, 3, 1, 10, 0, 0, 0, time.UTC).Add(-numMetrics * time.Minute).UnixMilli()
|
||||
end := time.Date(2025, 3, 1, 10, 0, 0, 0, time.UTC).UnixMilli()
|
||||
genData := func(prefix string) (recs []string, wantSeries []map[string]string, wantQueryResults []*at.QueryResult) {
|
||||
recs = make([]string, numMetrics)
|
||||
wantSeries = make([]map[string]string, numMetrics)
|
||||
wantQueryResults = make([]*at.QueryResult, numMetrics)
|
||||
for i := range numMetrics {
|
||||
name := fmt.Sprintf("%s_%03d", prefix, i)
|
||||
value := float64(i)
|
||||
timestamp := start + int64(i)*msecPerMinute
|
||||
|
||||
recs[i] = fmt.Sprintf("%s %f %d", name, value, timestamp)
|
||||
wantSeries[i] = map[string]string{"__name__": name}
|
||||
wantQueryResults[i] = &at.QueryResult{
|
||||
Metric: map[string]string{"__name__": name},
|
||||
Samples: []*at.Sample{{Timestamp: timestamp, Value: value}},
|
||||
}
|
||||
}
|
||||
return recs, wantSeries, wantQueryResults
|
||||
}
|
||||
|
||||
backupBaseDir, err := filepath.Abs(filepath.Join(tc.Dir(), "backups"))
|
||||
if err != nil {
|
||||
t.Fatalf("could not get absolute path for the backup base dir")
|
||||
}
|
||||
|
||||
// assertSeries issues various queries to the app and compares the query
|
||||
// results with the expected ones.
|
||||
assertQueries := func(app at.PrometheusQuerier, query string, wantSeries []map[string]string, wantQueryResults []*at.QueryResult) {
|
||||
t.Helper()
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/series response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1Series(t, query, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
}).Sort()
|
||||
},
|
||||
Want: &at.PrometheusAPIV1SeriesResponse{
|
||||
Status: "success",
|
||||
Data: wantSeries,
|
||||
},
|
||||
FailNow: true,
|
||||
})
|
||||
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/query_range response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1QueryRange(t, query, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
Step: "60s",
|
||||
})
|
||||
},
|
||||
Want: &at.PrometheusAPIV1QueryResponse{
|
||||
Status: "success",
|
||||
Data: &at.QueryData{
|
||||
ResultType: "matrix",
|
||||
Result: wantQueryResults,
|
||||
},
|
||||
},
|
||||
Retries: 300,
|
||||
FailNow: true,
|
||||
})
|
||||
}
|
||||
|
||||
createBackup := func(sut at.PrometheusWriteQuerier, name string) {
|
||||
t.Helper()
|
||||
for i, storageDataPath := range opts.storageDataPaths {
|
||||
replica := fmt.Sprintf("replica-%d", i)
|
||||
instance := fmt.Sprintf("vmbackup-%s-%s", name, replica)
|
||||
snapshotCreateURL := opts.snapshotCreateURLs(sut)[i]
|
||||
backupPath := "fs://" + filepath.Join(backupBaseDir, name, replica)
|
||||
tc.MustStartVmbackup(instance, storageDataPath, snapshotCreateURL, backupPath)
|
||||
}
|
||||
}
|
||||
|
||||
restoreFromBackup := func(name string) {
|
||||
t.Helper()
|
||||
for i, storageDataPath := range opts.storageDataPaths {
|
||||
replica := fmt.Sprintf("replica-%d", i)
|
||||
instance := fmt.Sprintf("vmrestore-%s-%s", name, replica)
|
||||
backupPath := "fs://" + filepath.Join(backupBaseDir, name, replica)
|
||||
tc.MustStartVmrestore(instance, backupPath, storageDataPath)
|
||||
}
|
||||
}
|
||||
|
||||
legacy1Data, wantLegacy1Series, wantLegacy1QueryResults := genData("legacy1")
|
||||
legacy2Data, wantLegacy2Series, wantLegacy2QueryResults := genData("legacy2")
|
||||
new1Data, wantNew1Series, wantNew1QueryResults := genData("new1")
|
||||
new2Data, wantNew2Series, wantNew2QueryResults := genData("new2")
|
||||
wantLegacy12Series := slices.Concat(wantLegacy1Series, wantLegacy2Series)
|
||||
wantLegacy12QueryResults := slices.Concat(wantLegacy1QueryResults, wantLegacy2QueryResults)
|
||||
wantLegacy1New1Series := slices.Concat(wantLegacy1Series, wantNew1Series)
|
||||
wantLegacy1New1QueryResults := slices.Concat(wantLegacy1QueryResults, wantNew1QueryResults)
|
||||
wantLegacy1New12Series := slices.Concat(wantLegacy1New1Series, wantNew2Series)
|
||||
wantLegacy1New12QueryResults := slices.Concat(wantLegacy1New1QueryResults, wantNew2QueryResults)
|
||||
var legacySUT, newSUT at.PrometheusWriteQuerier
|
||||
|
||||
// Verify backup/restore with legacy SUT.
|
||||
|
||||
// Start legacy SUT with empty storage data dir.
|
||||
legacySUT = opts.startLegacySUT()
|
||||
|
||||
// Ingest legacy1 records, ensure the queries return legacy1, and create
|
||||
// legacy1 backup.
|
||||
legacySUT.PrometheusAPIV1ImportPrometheus(t, legacy1Data, at.QueryOpts{})
|
||||
legacySUT.ForceFlush(t)
|
||||
assertQueries(legacySUT, `{__name__=~".*"}`, wantLegacy1Series, wantLegacy1QueryResults)
|
||||
createBackup(legacySUT, "legacy1")
|
||||
|
||||
// Ingest legacy2 records, ensure the queries return legacy1+legacy2, and
|
||||
// create legacy1+legacy2 backup.
|
||||
legacySUT.PrometheusAPIV1ImportPrometheus(t, legacy2Data, at.QueryOpts{})
|
||||
legacySUT.ForceFlush(t)
|
||||
assertQueries(legacySUT, `{__name__=~"legacy.*"}`, wantLegacy12Series, wantLegacy12QueryResults)
|
||||
createBackup(legacySUT, "legacy12")
|
||||
|
||||
// Stop legacy SUT and restore legacy1 data.
|
||||
// Start legacy SUT and ensure the queries return legacy1.
|
||||
opts.stopLegacySUT()
|
||||
restoreFromBackup("legacy1")
|
||||
legacySUT = opts.startLegacySUT()
|
||||
assertQueries(legacySUT, `{__name__=~".*"}`, wantLegacy1Series, wantLegacy1QueryResults)
|
||||
|
||||
opts.stopLegacySUT()
|
||||
|
||||
// Verify backup/restore with new SUT.
|
||||
|
||||
// Start new SUT (with partition indexDBs) with storage containing legacy1
|
||||
// data and Ensure that queries return legacy1 data.
|
||||
newSUT = opts.startNewSUT()
|
||||
assertQueries(newSUT, `{__name__=~".*"}`, wantLegacy1Series, wantLegacy1QueryResults)
|
||||
|
||||
// Ingest new1 records, ensure that queries now return legacy1+new1, and
|
||||
// create the legacy1+new1 backup.
|
||||
newSUT.PrometheusAPIV1ImportPrometheus(t, new1Data, at.QueryOpts{})
|
||||
newSUT.ForceFlush(t)
|
||||
assertQueries(newSUT, `{__name__=~"(legacy|new).*"}`, wantLegacy1New1Series, wantLegacy1New1QueryResults)
|
||||
createBackup(newSUT, "legacy1-new1")
|
||||
|
||||
// Ingest new2 records, ensure that queries now return legacy1+new1+new2,
|
||||
// and create the legacy1+new1+new2 backup.
|
||||
newSUT.PrometheusAPIV1ImportPrometheus(t, new2Data, at.QueryOpts{})
|
||||
newSUT.ForceFlush(t)
|
||||
assertQueries(newSUT, `{__name__=~"(legacy|new1|new2).*"}`, wantLegacy1New12Series, wantLegacy1New12QueryResults)
|
||||
createBackup(newSUT, "legacy1-new12")
|
||||
|
||||
// Stop new SUT and restore legacy1+new1 data.
|
||||
// Start new SUT and ensure queries return legacy1+new1 data.
|
||||
opts.stopNewSUT()
|
||||
restoreFromBackup("legacy1-new1")
|
||||
newSUT = opts.startNewSUT()
|
||||
assertQueries(newSUT, `{__name__=~".*"}`, wantLegacy1New1Series, wantLegacy1New1QueryResults)
|
||||
|
||||
opts.stopNewSUT()
|
||||
|
||||
// Verify backup/restore with legacy SUT again.
|
||||
|
||||
// Start legacy SUT with storage containing legacy1+new1 data.
|
||||
//
|
||||
// Ensure that the /series and /query_range queries return legacy1 data only.
|
||||
// new1 data is not returned because legacy vmsingle does not know about
|
||||
// partition indexDBs.
|
||||
legacySUT = opts.startLegacySUT()
|
||||
assertQueries(legacySUT, `{__name__=~".*"}`, wantLegacy1Series, wantLegacy1QueryResults)
|
||||
|
||||
// Stop legacy SUT and restore legacy1+legacy2 data.
|
||||
// Start legacy SUT and ensure that queries now return legacy1+legacy2 data.
|
||||
opts.stopLegacySUT()
|
||||
restoreFromBackup("legacy12")
|
||||
legacySUT = opts.startLegacySUT()
|
||||
assertQueries(legacySUT, `{__name__=~".*"}`, wantLegacy12Series, wantLegacy12QueryResults)
|
||||
|
||||
opts.stopLegacySUT()
|
||||
|
||||
// Verify backup/restore with new vmsingle again.
|
||||
|
||||
// Start new vmsingle with storage containing legacy1+legacy2 data and
|
||||
// ensure that queries return legacy1+legacy2 data.
|
||||
newSUT = opts.startNewSUT()
|
||||
assertQueries(newSUT, `{__name__=~".*"}`, wantLegacy12Series, wantLegacy12QueryResults)
|
||||
|
||||
// Stop new SUT and restore legacy1+new1+new2 data.
|
||||
// Start new SUT and ensure that queries return legacy1+new1+new2 data.
|
||||
opts.stopNewSUT()
|
||||
restoreFromBackup("legacy1-new12")
|
||||
newSUT = opts.startNewSUT()
|
||||
assertQueries(newSUT, `{__name__=~"(legacy|new).*"}`, wantLegacy1New12Series, wantLegacy1New12QueryResults)
|
||||
|
||||
opts.stopNewSUT()
|
||||
}
|
||||
|
||||
type testLegacyDowngradeOpts struct {
|
||||
startLegacySUT func() at.PrometheusWriteQuerier
|
||||
startNewSUT func() at.PrometheusWriteQuerier
|
||||
stopLegacySUT func()
|
||||
stopNewSUT func()
|
||||
}
|
||||
|
||||
func TestLegacySingleDowngrade(t *testing.T) {
|
||||
tc := at.NewTestCase(t)
|
||||
defer tc.Stop()
|
||||
|
||||
storageDataPath := filepath.Join(tc.Dir(), "vmsingle")
|
||||
|
||||
opts := testLegacyDowngradeOpts{
|
||||
startLegacySUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartVmsingleAt("vmsingle-legacy", legacyVmsinglePath, []string{
|
||||
"-storageDataPath=" + storageDataPath,
|
||||
"-retentionPeriod=100y",
|
||||
"-search.disableCache=true",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
})
|
||||
},
|
||||
startNewSUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartVmsingle("vmsingle-new", []string{
|
||||
"-storageDataPath=" + storageDataPath,
|
||||
"-retentionPeriod=100y",
|
||||
"-search.disableCache=true",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
})
|
||||
},
|
||||
stopLegacySUT: func() {
|
||||
tc.StopApp("vmsingle-legacy")
|
||||
},
|
||||
stopNewSUT: func() {
|
||||
tc.StopApp("vmsingle-new")
|
||||
},
|
||||
}
|
||||
|
||||
testLegacyDowngrade(tc, opts)
|
||||
}
|
||||
|
||||
func TestLegacyClusterDowngrade(t *testing.T) {
|
||||
tc := at.NewTestCase(t)
|
||||
defer tc.Stop()
|
||||
|
||||
storage1DataPath := filepath.Join(tc.Dir(), "vmstorage1")
|
||||
storage2DataPath := filepath.Join(tc.Dir(), "vmstorage2")
|
||||
|
||||
opts := testLegacyDowngradeOpts{
|
||||
startLegacySUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartCluster(&at.ClusterOptions{
|
||||
Vmstorage1Instance: "vmstorage1-legacy",
|
||||
Vmstorage1Binary: legacyVmstoragePath,
|
||||
Vmstorage1Flags: []string{
|
||||
"-storageDataPath=" + storage1DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
Vmstorage2Instance: "vmstorage2-legacy",
|
||||
Vmstorage2Binary: legacyVmstoragePath,
|
||||
Vmstorage2Flags: []string{
|
||||
"-storageDataPath=" + storage2DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
VminsertInstance: "vminsert",
|
||||
VminsertFlags: []string{},
|
||||
VmselectInstance: "vmselect",
|
||||
VmselectFlags: []string{
|
||||
"-search.disableCache=true",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
},
|
||||
})
|
||||
},
|
||||
startNewSUT: func() at.PrometheusWriteQuerier {
|
||||
return tc.MustStartCluster(&at.ClusterOptions{
|
||||
Vmstorage1Instance: "vmstorage1-new",
|
||||
Vmstorage1Flags: []string{
|
||||
"-storageDataPath=" + storage1DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
Vmstorage2Instance: "vmstorage2-new",
|
||||
Vmstorage2Flags: []string{
|
||||
"-storageDataPath=" + storage2DataPath,
|
||||
"-retentionPeriod=100y",
|
||||
},
|
||||
VminsertInstance: "vminsert",
|
||||
VminsertFlags: []string{},
|
||||
VmselectInstance: "vmselect",
|
||||
VmselectFlags: []string{
|
||||
"-search.disableCache=true",
|
||||
"-search.maxStalenessInterval=1m",
|
||||
},
|
||||
})
|
||||
},
|
||||
stopLegacySUT: func() {
|
||||
tc.StopApp("vminsert")
|
||||
tc.StopApp("vmselect")
|
||||
tc.StopApp("vmstorage1-legacy")
|
||||
tc.StopApp("vmstorage2-legacy")
|
||||
},
|
||||
stopNewSUT: func() {
|
||||
tc.StopApp("vminsert")
|
||||
tc.StopApp("vmselect")
|
||||
tc.StopApp("vmstorage1-new")
|
||||
tc.StopApp("vmstorage2-new")
|
||||
},
|
||||
}
|
||||
|
||||
testLegacyDowngrade(tc, opts)
|
||||
}
|
||||
|
||||
func testLegacyDowngrade(tc *at.TestCase, opts testLegacyDowngradeOpts) {
|
||||
t := tc.T()
|
||||
|
||||
type want struct {
|
||||
series []map[string]string
|
||||
labels []string
|
||||
labelValues []string
|
||||
queryResults []*at.QueryResult
|
||||
queryRangeResults []*at.QueryResult
|
||||
}
|
||||
|
||||
uniq := func(s []string) []string {
|
||||
slices.Sort(s)
|
||||
return slices.Compact(s)
|
||||
}
|
||||
|
||||
mergeWant := func(want1, want2 want) want {
|
||||
var result want
|
||||
result.series = slices.Concat(want1.series, want2.series)
|
||||
result.labels = uniq(slices.Concat(want1.labels, want2.labels))
|
||||
result.labelValues = slices.Concat(want1.labelValues, want2.labelValues)
|
||||
result.queryResults = slices.Concat(want1.queryResults, want2.queryResults)
|
||||
result.queryRangeResults = slices.Concat(want1.queryRangeResults, want2.queryRangeResults)
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// Use the same number of metrics and time range for all the data batches below.
|
||||
const numMetrics = 1000
|
||||
const labelName = "prefix"
|
||||
start := time.Date(2025, 3, 1, 10, 0, 0, 0, time.UTC).UnixMilli()
|
||||
end := start
|
||||
genData := func(prefix string) (recs []string, want want) {
|
||||
labelValue := prefix
|
||||
recs = make([]string, numMetrics)
|
||||
want.series = make([]map[string]string, numMetrics)
|
||||
want.labels = []string{"__name__", labelName}
|
||||
want.labelValues = []string{labelValue}
|
||||
want.queryResults = make([]*at.QueryResult, numMetrics)
|
||||
want.queryRangeResults = make([]*at.QueryResult, numMetrics)
|
||||
for i := range numMetrics {
|
||||
name := fmt.Sprintf("%s_%03d", prefix, i)
|
||||
value := float64(i)
|
||||
timestamp := start
|
||||
|
||||
recs[i] = fmt.Sprintf("%s{%s=\"%s\"} %f %d", name, labelName, labelValue, value, timestamp)
|
||||
want.series[i] = map[string]string{"__name__": name, labelName: labelValue}
|
||||
want.queryResults[i] = &at.QueryResult{
|
||||
Metric: map[string]string{"__name__": name, labelName: labelValue},
|
||||
Sample: &at.Sample{Timestamp: timestamp, Value: value},
|
||||
}
|
||||
want.queryRangeResults[i] = &at.QueryResult{
|
||||
Metric: map[string]string{"__name__": name, labelName: labelValue},
|
||||
Samples: []*at.Sample{{Timestamp: timestamp, Value: value}},
|
||||
}
|
||||
}
|
||||
return recs, want
|
||||
}
|
||||
|
||||
// assertSeries issues various queries to the app and compares the query
|
||||
// results with the expected ones.
|
||||
assertQueries := func(app at.PrometheusQuerier, query string, want want, wantSeriesCount uint64) {
|
||||
t.Helper()
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/series response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1Series(t, query, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
}).Sort()
|
||||
},
|
||||
Want: &at.PrometheusAPIV1SeriesResponse{
|
||||
Status: "success",
|
||||
Data: want.series,
|
||||
},
|
||||
FailNow: true,
|
||||
})
|
||||
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/series/count response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1SeriesCount(t, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
})
|
||||
},
|
||||
Want: &at.PrometheusAPIV1SeriesCountResponse{
|
||||
Status: "success",
|
||||
Data: []uint64{wantSeriesCount},
|
||||
},
|
||||
FailNow: true,
|
||||
})
|
||||
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/labels response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1Labels(t, query, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
})
|
||||
},
|
||||
Want: &at.PrometheusAPIV1LabelsResponse{
|
||||
Status: "success",
|
||||
Data: want.labels,
|
||||
},
|
||||
FailNow: true,
|
||||
})
|
||||
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/label/../values response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1LabelValues(t, labelName, query, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
})
|
||||
},
|
||||
Want: &at.PrometheusAPIV1LabelValuesResponse{
|
||||
Status: "success",
|
||||
Data: want.labelValues,
|
||||
},
|
||||
FailNow: true,
|
||||
})
|
||||
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/query response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1Query(t, query, at.QueryOpts{
|
||||
Time: fmt.Sprintf("%d", start),
|
||||
Step: "10m",
|
||||
})
|
||||
},
|
||||
Want: &at.PrometheusAPIV1QueryResponse{
|
||||
Status: "success",
|
||||
Data: &at.QueryData{
|
||||
ResultType: "vector",
|
||||
Result: want.queryResults,
|
||||
},
|
||||
},
|
||||
Retries: 300,
|
||||
FailNow: true,
|
||||
})
|
||||
|
||||
tc.Assert(&at.AssertOptions{
|
||||
Msg: "unexpected /api/v1/query_range response",
|
||||
Got: func() any {
|
||||
return app.PrometheusAPIV1QueryRange(t, query, at.QueryOpts{
|
||||
Start: fmt.Sprintf("%d", start),
|
||||
End: fmt.Sprintf("%d", end),
|
||||
Step: "60s",
|
||||
})
|
||||
},
|
||||
Want: &at.PrometheusAPIV1QueryResponse{
|
||||
Status: "success",
|
||||
Data: &at.QueryData{
|
||||
ResultType: "matrix",
|
||||
Result: want.queryRangeResults,
|
||||
},
|
||||
},
|
||||
Retries: 300,
|
||||
FailNow: true,
|
||||
})
|
||||
}
|
||||
|
||||
wantEmpty := want{
|
||||
series: []map[string]string{},
|
||||
labels: []string{"__name__"},
|
||||
labelValues: []string{},
|
||||
queryResults: []*at.QueryResult{},
|
||||
queryRangeResults: []*at.QueryResult{},
|
||||
}
|
||||
|
||||
legacy1Data, wantLegacy1 := genData("legacy1")
|
||||
legacy2Data, wantLegacy2 := genData("legacy2")
|
||||
new1Data, wantNew1 := genData("new1")
|
||||
wantLegacy1New1 := mergeWant(wantLegacy1, wantNew1)
|
||||
wantLegacy2New1 := mergeWant(wantLegacy2, wantNew1)
|
||||
var legacySUT, newSUT at.PrometheusWriteQuerier
|
||||
|
||||
// Start legacy SUT with empty storage data dir.
|
||||
// Ingest legacy1 records, ensure the queries return legacy1
|
||||
legacySUT = opts.startLegacySUT()
|
||||
legacySUT.PrometheusAPIV1ImportPrometheus(t, legacy1Data, at.QueryOpts{})
|
||||
legacySUT.ForceFlush(t)
|
||||
assertQueries(legacySUT, `{__name__=~".*"}`, wantLegacy1, numMetrics)
|
||||
opts.stopLegacySUT()
|
||||
|
||||
// Start new SUT (with partition indexDBs) with storage containing legacy1
|
||||
// data and ensure that queries return new1 and legacy1 data.
|
||||
newSUT = opts.startNewSUT()
|
||||
newSUT.PrometheusAPIV1ImportPrometheus(t, new1Data, at.QueryOpts{})
|
||||
newSUT.ForceFlush(t)
|
||||
assertQueries(newSUT, `{__name__=~".*"}`, wantLegacy1New1, 2*numMetrics)
|
||||
opts.stopNewSUT()
|
||||
|
||||
// Downgrade to legacy SUT, ensure the queries return only legacy1.
|
||||
// Delete all series, ensure that queries return no series.
|
||||
// Ingest legacy2 records, ensure the queries return only legacy2.
|
||||
legacySUT = opts.startLegacySUT()
|
||||
assertQueries(legacySUT, `{__name__=~".*"}`, wantLegacy1, numMetrics)
|
||||
legacySUT.APIV1AdminTSDBDeleteSeries(t, `{__name__=~".*"}`, at.QueryOpts{})
|
||||
assertQueries(legacySUT, `{__name__=~".*"}`, wantEmpty, numMetrics)
|
||||
legacySUT.PrometheusAPIV1ImportPrometheus(t, legacy2Data, at.QueryOpts{})
|
||||
legacySUT.ForceFlush(t)
|
||||
// series count includes deleted metrics
|
||||
assertQueries(legacySUT, `{__name__=~".*"}`, wantLegacy2, 2*numMetrics)
|
||||
opts.stopLegacySUT()
|
||||
|
||||
// Upgrade to new SUT, ensure the queries return recently ingested legacy2 and new1
|
||||
// since legacy SUT cannot delete them.
|
||||
// Delete all series, ensure that queries return no series.
|
||||
newSUT = opts.startNewSUT()
|
||||
// series count includes deleted metrics
|
||||
assertQueries(newSUT, `{__name__=~".*"}`, wantLegacy2New1, 3*numMetrics)
|
||||
newSUT.APIV1AdminTSDBDeleteSeries(t, `{__name__=~".*"}`, at.QueryOpts{})
|
||||
// series count includes deleted metrics
|
||||
assertQueries(newSUT, `{__name__=~".*"}`, wantEmpty, 3*numMetrics)
|
||||
opts.stopNewSUT()
|
||||
}
|
||||
@@ -26,10 +26,13 @@ See also [LTS releases](https://docs.victoriametrics.com/victoriametrics/lts-rel
|
||||
|
||||
## tip
|
||||
|
||||
**Update Note 1:** [vmsingle](https://docs.victoriametrics.com/victoriametrics/single-server-victoriametrics/) and `vmstorage` in [VictoriaMetrics cluster](https://docs.victoriametrics.com/victoriametrics/cluster-victoriametrics/): Upgrading to per-partition index requires registering all active time series. Expect slow down of data ingestion and queries during upgrade roll-out. This is a one-time operation. Additionally, for users with retention periods shorter than 1 month the disk usage may increase.
|
||||
|
||||
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent/) and [vmsingle](https://docs.victoriametrics.com/victoriametrics/single-server-victoriametrics/): add support for global `sampleLimit` setting. This allows users to efficiently limit the number of samples accepted per scrape target. This also ensures target-level `sample_limit` can correctly override the global setting. See [#10145](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/10145). Thanks to the @kobylyanskiy.
|
||||
* FEATURE: [vmauth](https://docs.victoriametrics.com/victoriametrics/vmauth/): expose `vmauth_user_request_backend_requests_total` and `vmauth_unauthorized_user_request_backend_requests_total` [metrics](https://docs.victoriametrics.com/victoriametrics/vmauth/#monitoring), which track the number of requests sent to backends. These counts may exceed `vmauth_user_requests_total` and `vmauth_unauthorized_user_requests_total` when requests are retried across multiple backends.
|
||||
* FEATURE: [vmauth](https://docs.victoriametrics.com/victoriametrics/vmauth/): expose `vmauth_user_request_errors_total` and `vmauth_unauthorized_user_request_errors_total` [metrics](https://docs.victoriametrics.com/victoriametrics/vmauth/#monitoring), which track the number of user request errors.
|
||||
* FEATURE: [vmauth](https://docs.victoriametrics.com/victoriametrics/vmauth/): add `-maxQueueDuration` command-line flag for waiting until the incoming request could be executed if `-maxConcurrentRequests` are already executed. This should help with graceful handling of a short spike in the number of concurrent requests without leading to a retry storm. See [#10078](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/10078).
|
||||
* FEATURE: [vmsingle](https://docs.victoriametrics.com/victoriametrics/single-server-victoriametrics/) and `vmstorage` in [VictoriaMetrics cluster](https://docs.victoriametrics.com/victoriametrics/cluster-victoriametrics/): introduce per-partition index. This should reduce disk space occupied by indexDBs as they get deleted along with the corresponding partitions once those partitions become outside the retention window. Read more about the motivation behind this feature at [7599](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/7599) and what to expect at [8134](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/8134).
|
||||
|
||||
* BUGFIX: [vmagent](https://docs.victoriametrics.com/victoriametrics/vmagent/) and [vmsingle](https://docs.victoriametrics.com/victoriametrics/single-server-victoriametrics/): properly add `exported_` prefix to scraped metrics that have the same names as [auto-generated metrics](https://docs.victoriametrics.com/victoriametrics/vmagent/#automatically-generated-metrics). Previously, some auto-generated metrics were not recognized, so scraped metrics with the same names could overwrite them.
|
||||
* BUGFIX: [vmagent](https://docs.victoriametrics.com/victoriametrics/vmagent/): fix `vmagent_rows_inserted_total{type="newrelic"}` metric to correctly count samples.
|
||||
|
||||
@@ -1561,9 +1561,6 @@ func mustOpenParts(path string) []*partWrapper {
|
||||
// or a problem with the underlying file system (such as insufficient
|
||||
// permissions).
|
||||
func (tb *Table) MustCreateSnapshotAt(dstDir string) {
|
||||
logger.Infof("creating Table snapshot of %q...", tb.path)
|
||||
startTime := time.Now()
|
||||
|
||||
var err error
|
||||
srcDir := tb.path
|
||||
srcDir, err = filepath.Abs(srcDir)
|
||||
@@ -1602,8 +1599,6 @@ func (tb *Table) MustCreateSnapshotAt(dstDir string) {
|
||||
}
|
||||
|
||||
fs.MustSyncPathAndParentDir(dstDir)
|
||||
|
||||
logger.Infof("created Table snapshot of %q at %q in %.3f seconds", srcDir, dstDir, time.Since(startTime).Seconds())
|
||||
}
|
||||
|
||||
func mustWritePartNames(pws []*partWrapper, dstDir string) {
|
||||
|
||||
15
lib/mergeset/table_legacy.go
Normal file
15
lib/mergeset/table_legacy.go
Normal file
@@ -0,0 +1,15 @@
|
||||
package mergeset
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// LegacyMustCreateSnapshotAt is used for creating snapshots for legacy IndexDBs.
|
||||
func (tb *Table) LegacyMustCreateSnapshotAt(dstDir string) {
|
||||
logger.Infof("creating legacy IndexDB snapshot of %q...", tb.path)
|
||||
startTime := time.Now()
|
||||
tb.MustCreateSnapshotAt(dstDir)
|
||||
logger.Infof("created legacy IndexDB snapshot of %q at %q in %.3f seconds", tb.path, dstDir, time.Since(startTime).Seconds())
|
||||
}
|
||||
@@ -14,9 +14,9 @@ const (
|
||||
tsidCacheFilename = "metricName_tsid"
|
||||
metricIDCacheFilename = "metricID_tsid"
|
||||
metricNameCacheFilename = "metricID_metricName"
|
||||
prevHourMetricIDsFilename = "prev_hour_metric_ids"
|
||||
currHourMetricIDsFilename = "curr_hour_metric_ids"
|
||||
nextDayMetricIDsFilename = "next_day_metric_ids_v2"
|
||||
prevHourMetricIDsFilename = "prev_hour_metric_ids_v2"
|
||||
currHourMetricIDsFilename = "curr_hour_metric_ids_v2"
|
||||
nextDayMetricIDsFilename = "next_day_metric_ids_v3"
|
||||
metricNameTrackerFilename = "metric_usage_tracker"
|
||||
)
|
||||
|
||||
|
||||
@@ -6,10 +6,8 @@ import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"path/filepath"
|
||||
"regexp"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
@@ -23,7 +21,6 @@ import (
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/lrucache"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
|
||||
@@ -74,12 +71,6 @@ const (
|
||||
|
||||
// indexDB represents an index db.
|
||||
type indexDB struct {
|
||||
// The number of references to indexDB struct.
|
||||
refCount atomic.Int32
|
||||
|
||||
// if the mustDrop is set to true, then the indexDB must be dropped after refCount reaches zero.
|
||||
mustDrop atomic.Bool
|
||||
|
||||
// The number of calls for date range searches.
|
||||
dateRangeSearchCalls atomic.Uint64
|
||||
|
||||
@@ -98,21 +89,24 @@ type indexDB struct {
|
||||
// The db must be automatically recovered after that.
|
||||
missingMetricNamesForMetricID atomic.Uint64
|
||||
|
||||
// minMissingTimestampByKey holds the minimum timestamps by index search key,
|
||||
// legacyMinMissingTimestampByKey holds the minimum timestamps by index search key,
|
||||
// which is missing in the given indexDB.
|
||||
// Key must be formed with marshalCommonPrefix function.
|
||||
//
|
||||
// This field is used at containsTimeRange() function only for the previous indexDB,
|
||||
// since this indexDB is readonly.
|
||||
// This field cannot be used for the current indexDB, since it may receive data
|
||||
// This field is used at legacyContainsTimeRange() function only for the
|
||||
// legacy indexDBs, since these indexDBs are readonly.
|
||||
// This field cannot be used for the partition indexDBs, since they may receive data
|
||||
// with bigger timestamps at any time.
|
||||
minMissingTimestampByKey map[string]int64
|
||||
// protects minMissingTimestampByKey
|
||||
minMissingTimestampByKeyLock sync.Mutex
|
||||
legacyMinMissingTimestampByKey map[string]int64
|
||||
// protects legacyMinMissingTimestampByKey
|
||||
legacyMinMissingTimestampByKeyLock sync.Mutex
|
||||
|
||||
// generation identifies the index generation ID
|
||||
// and is used for syncing items from different indexDBs
|
||||
generation uint64
|
||||
// id identifies the indexDB. It is used for in various caches to know which
|
||||
// indexDB contains a metricID and which does not.
|
||||
id uint64
|
||||
|
||||
// Time range covered by this IndexDB.
|
||||
tr TimeRange
|
||||
|
||||
name string
|
||||
tb *mergeset.Table
|
||||
@@ -123,10 +117,6 @@ type indexDB struct {
|
||||
|
||||
// noRegisterNewSeries indicates whether the indexDB receives new entries or
|
||||
// not.
|
||||
//
|
||||
// Note that setting this field to true won't disable registering new
|
||||
// index entries (should they arrive). It is solely used to decide whether
|
||||
// the containsTimeRange() optimization can be applied to the indexDB.
|
||||
noRegisterNewSeries atomic.Bool
|
||||
|
||||
// Cache for fast TagFilters -> MetricIDs lookup.
|
||||
@@ -151,6 +141,10 @@ type indexDB struct {
|
||||
// memory.
|
||||
dateMetricIDCache *dateMetricIDCache
|
||||
|
||||
// An inmemory set of deleted metricIDs.
|
||||
deletedMetricIDs atomic.Pointer[uint64set.Set]
|
||||
deletedMetricIDsUpdateLock sync.Mutex
|
||||
|
||||
indexSearchPool sync.Pool
|
||||
}
|
||||
|
||||
@@ -172,36 +166,27 @@ func getTagFiltersLoopsCacheSize() uint64 {
|
||||
return uint64(float64(memory.Allowed()) / 128)
|
||||
}
|
||||
|
||||
// mustOpenIndexDB opens index db from the given path.
|
||||
//
|
||||
// The last segment of the path should contain unique hex value which
|
||||
// will be then used as indexDB.generation
|
||||
func mustOpenIndexDB(path string, s *Storage, isReadOnly *atomic.Bool, noRegisterNewSeries bool) *indexDB {
|
||||
func mustOpenIndexDB(id uint64, tr TimeRange, name, path string, s *Storage, isReadOnly *atomic.Bool, noRegisterNewSeries bool) *indexDB {
|
||||
if s == nil {
|
||||
logger.Panicf("BUG: Storage must be non-nil")
|
||||
}
|
||||
|
||||
name := filepath.Base(path)
|
||||
gen, err := strconv.ParseUint(name, 16, 64)
|
||||
if err != nil {
|
||||
logger.Panicf("FATAL: cannot parse indexdb path %q: %s", path, err)
|
||||
logger.Panicf("BUG: Storage must not be nil")
|
||||
}
|
||||
|
||||
tfssCache := lrucache.NewCache(getTagFiltersCacheSize)
|
||||
tb := mergeset.MustOpenTable(path, dataFlushInterval, tfssCache.Reset, mergeTagToMetricIDsRows, isReadOnly)
|
||||
db := &indexDB{
|
||||
minMissingTimestampByKey: make(map[string]int64),
|
||||
generation: gen,
|
||||
name: name,
|
||||
tb: tb,
|
||||
s: s,
|
||||
tagFiltersToMetricIDsCache: tfssCache,
|
||||
loopsPerDateTagFilterCache: lrucache.NewCache(getTagFiltersLoopsCacheSize),
|
||||
metricIDCache: newMetricIDCache(),
|
||||
dateMetricIDCache: newDateMetricIDCache(),
|
||||
legacyMinMissingTimestampByKey: make(map[string]int64),
|
||||
id: id,
|
||||
tr: tr,
|
||||
name: name,
|
||||
tb: tb,
|
||||
s: s,
|
||||
tagFiltersToMetricIDsCache: tfssCache,
|
||||
loopsPerDateTagFilterCache: lrucache.NewCache(getTagFiltersLoopsCacheSize),
|
||||
metricIDCache: newMetricIDCache(),
|
||||
dateMetricIDCache: newDateMetricIDCache(),
|
||||
}
|
||||
db.noRegisterNewSeries.Store(noRegisterNewSeries)
|
||||
db.incRef()
|
||||
db.mustLoadDeletedMetricIDs()
|
||||
return db
|
||||
}
|
||||
|
||||
@@ -226,6 +211,8 @@ type IndexDBMetrics struct {
|
||||
DateMetricIDCacheSyncsCount uint64
|
||||
DateMetricIDCacheRotationsCount uint64
|
||||
|
||||
// Used by legacy indexDBs only.
|
||||
// See UpdateMetrics() in index_db_legacy.go
|
||||
IndexDBRefCount uint64
|
||||
|
||||
RecentHourMetricIDsSearchCalls uint64
|
||||
@@ -248,10 +235,6 @@ type IndexDBMetrics struct {
|
||||
mergeset.TableMetrics
|
||||
}
|
||||
|
||||
func (db *indexDB) scheduleToDrop() {
|
||||
db.mustDrop.Store(true)
|
||||
}
|
||||
|
||||
// UpdateMetrics updates m with metrics from the db.
|
||||
func (db *indexDB) UpdateMetrics(m *IndexDBMetrics) {
|
||||
// global index metrics
|
||||
@@ -293,8 +276,6 @@ func (db *indexDB) UpdateMetrics(m *IndexDBMetrics) {
|
||||
m.DateMetricIDCacheRotationsCount = dmcs.RotationsCount
|
||||
}
|
||||
|
||||
m.IndexDBRefCount += uint64(db.refCount.Load())
|
||||
|
||||
m.DateRangeSearchCalls += db.dateRangeSearchCalls.Load()
|
||||
m.DateRangeSearchHits += db.dateRangeSearchHits.Load()
|
||||
m.GlobalSearchCalls += db.globalSearchCalls.Load()
|
||||
@@ -307,27 +288,6 @@ func (db *indexDB) UpdateMetrics(m *IndexDBMetrics) {
|
||||
|
||||
// MustClose closes db.
|
||||
func (db *indexDB) MustClose() {
|
||||
rc := db.refCount.Load()
|
||||
if rc != 1 {
|
||||
logger.Fatalf("BUG: %q unexpected indexDB refCount: %d", db.name, rc)
|
||||
}
|
||||
db.decRef()
|
||||
}
|
||||
|
||||
func (db *indexDB) incRef() {
|
||||
db.refCount.Add(1)
|
||||
}
|
||||
|
||||
func (db *indexDB) decRef() {
|
||||
n := db.refCount.Add(-1)
|
||||
if n < 0 {
|
||||
logger.Panicf("BUG: %q negative refCount: %d", db.name, n)
|
||||
}
|
||||
if n > 0 {
|
||||
return
|
||||
}
|
||||
|
||||
tbPath := db.tb.Path()
|
||||
db.tb.MustClose()
|
||||
db.tb = nil
|
||||
db.s = nil
|
||||
@@ -342,14 +302,6 @@ func (db *indexDB) decRef() {
|
||||
db.loopsPerDateTagFilterCache = nil
|
||||
db.metricIDCache = nil
|
||||
db.dateMetricIDCache = nil
|
||||
|
||||
if !db.mustDrop.Load() {
|
||||
return
|
||||
}
|
||||
|
||||
logger.Infof("dropping indexDB %q", tbPath)
|
||||
fs.MustRemoveDir(tbPath)
|
||||
logger.Infof("indexDB %q has been dropped", tbPath)
|
||||
}
|
||||
|
||||
// getMetricIDsFromTagFiltersCache retrieves the set of metricIDs that
|
||||
@@ -474,6 +426,10 @@ func generateTSID(dst *TSID, mn *MetricName) {
|
||||
}
|
||||
|
||||
func (db *indexDB) createGlobalIndexes(tsid *TSID, mn *MetricName) {
|
||||
if db.noRegisterNewSeries.Load() {
|
||||
logger.Panicf("BUG: registration of new series is disabled for indexDB %q", db.name)
|
||||
}
|
||||
|
||||
// Add new metricID to cache.
|
||||
db.metricIDCache.Set(tsid.MetricID)
|
||||
|
||||
@@ -633,7 +589,7 @@ func (is *indexSearch) searchLabelNamesWithFiltersOnDate(qt *querytracer.Tracer,
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
dmis := is.db.getDeletedMetricIDs()
|
||||
loopsPaceLimiter := 0
|
||||
underscoreNameSeen := false
|
||||
nsPrefixExpected := byte(nsPrefixDateTagToMetricIDs)
|
||||
@@ -716,7 +672,7 @@ func (is *indexSearch) getLabelNamesForMetricIDs(qt *querytracer.Tracer, metricI
|
||||
lns["__name__"] = struct{}{}
|
||||
}
|
||||
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
dmis := is.db.getDeletedMetricIDs()
|
||||
|
||||
var mn MetricName
|
||||
foundLabelNames := 0
|
||||
@@ -909,7 +865,7 @@ func (is *indexSearch) searchLabelValuesOnDate(qt *querytracer.Tracer, labelName
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
dmis := is.db.getDeletedMetricIDs()
|
||||
loopsPaceLimiter := 0
|
||||
nsPrefixExpected := byte(nsPrefixDateTagToMetricIDs)
|
||||
if date == globalIndexDate {
|
||||
@@ -963,8 +919,7 @@ func (is *indexSearch) getLabelValuesForMetricIDs(qt *querytracer.Tracer, labelN
|
||||
}
|
||||
|
||||
lvs := make(map[string]struct{})
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
|
||||
dmis := is.db.getDeletedMetricIDs()
|
||||
var mn MetricName
|
||||
foundLabelValues := 0
|
||||
var buf []byte
|
||||
@@ -1090,7 +1045,7 @@ func (is *indexSearch) searchTagValueSuffixesForPrefix(nsPrefix byte, prefix []b
|
||||
kb := &is.kb
|
||||
ts := &is.ts
|
||||
mp := &is.mp
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
dmis := is.db.getDeletedMetricIDs()
|
||||
loopsPaceLimiter := 0
|
||||
ts.Seek(prefix)
|
||||
tvss := make(map[string]struct{})
|
||||
@@ -1350,7 +1305,7 @@ func (is *indexSearch) getTSDBStatus(qt *querytracer.Tracer, tfss []*TagFilters,
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
dmis := is.db.getDeletedMetricIDs()
|
||||
thSeriesCountByMetricName := newTopHeap(topN)
|
||||
thSeriesCountByLabelName := newTopHeap(topN)
|
||||
thSeriesCountByFocusLabelValue := newTopHeap(topN)
|
||||
@@ -1615,10 +1570,10 @@ func (db *indexDB) saveDeletedMetricIDs(metricIDs *uint64set.Set) {
|
||||
}
|
||||
|
||||
// atomically add deleted metricIDs to an inmemory map.
|
||||
db.s.updateDeletedMetricIDs(metricIDs)
|
||||
db.updateDeletedMetricIDs(metricIDs)
|
||||
|
||||
// Reset MetricName -> TSID cache, since it may contain deleted TSIDs.
|
||||
db.s.resetAndSaveTSIDCache()
|
||||
// Do not reset tsidCache (MetricName -> TSID), since a given TSID can be
|
||||
// deleted in one indexDB but still be used in another indexDB.
|
||||
|
||||
// Do not reset Storage's metricIDCache (MetricID -> TSID) and
|
||||
// metricNameCache (MetricID -> MetricName) since they must be used only
|
||||
@@ -1671,14 +1626,32 @@ func (db *indexDB) saveDeletedMetricIDs(metricIDs *uint64set.Set) {
|
||||
putIndexItems(items)
|
||||
}
|
||||
|
||||
func (db *indexDB) loadDeletedMetricIDs() (*uint64set.Set, error) {
|
||||
func (db *indexDB) getDeletedMetricIDs() *uint64set.Set {
|
||||
return db.deletedMetricIDs.Load()
|
||||
}
|
||||
|
||||
func (db *indexDB) setDeletedMetricIDs(dmis *uint64set.Set) {
|
||||
db.deletedMetricIDs.Store(dmis)
|
||||
}
|
||||
|
||||
func (db *indexDB) updateDeletedMetricIDs(metricIDs *uint64set.Set) {
|
||||
db.deletedMetricIDsUpdateLock.Lock()
|
||||
dmisOld := db.getDeletedMetricIDs()
|
||||
dmisNew := dmisOld.Clone()
|
||||
dmisNew.Union(metricIDs)
|
||||
db.setDeletedMetricIDs(dmisNew)
|
||||
db.deletedMetricIDsUpdateLock.Unlock()
|
||||
}
|
||||
|
||||
func (db *indexDB) mustLoadDeletedMetricIDs() {
|
||||
is := db.getIndexSearch(noDeadline)
|
||||
dmis, err := is.loadDeletedMetricIDs()
|
||||
db.putIndexSearch(is)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
logger.Panicf("FATAL: cannot load deleted metricIDs for indexDB %q: %v", db.name, err)
|
||||
return
|
||||
}
|
||||
return dmis, nil
|
||||
db.setDeletedMetricIDs(dmis)
|
||||
}
|
||||
|
||||
func (is *indexSearch) loadDeletedMetricIDs() (*uint64set.Set, error) {
|
||||
@@ -1890,8 +1863,9 @@ func (db *indexDB) SearchMetricNames(qt *querytracer.Tracer, tfss []*TagFilters,
|
||||
|
||||
var tagFiltersKeyBufPool bytesutil.ByteBufferPool
|
||||
|
||||
func (is *indexSearch) getTSIDByMetricName(dst *generationTSID, metricName []byte, date uint64) bool {
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
func (is *indexSearch) getTSIDByMetricName(dst *TSID, metricName []byte, date uint64) bool {
|
||||
dmis := is.db.getDeletedMetricIDs()
|
||||
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
|
||||
@@ -1910,18 +1884,17 @@ func (is *indexSearch) getTSIDByMetricName(dst *generationTSID, metricName []byt
|
||||
return false
|
||||
}
|
||||
v := ts.Item[len(kb.B):]
|
||||
tail, err := dst.TSID.Unmarshal(v)
|
||||
tail, err := dst.Unmarshal(v)
|
||||
if err != nil {
|
||||
logger.Panicf("FATAL: cannot unmarshal TSID: %s", err)
|
||||
}
|
||||
if len(tail) > 0 {
|
||||
logger.Panicf("FATAL: unexpected non-empty tail left after unmarshaling TSID: %X", tail)
|
||||
}
|
||||
if dmis.Has(dst.TSID.MetricID) {
|
||||
if dmis.Has(dst.MetricID) {
|
||||
// The dst is deleted. Continue searching.
|
||||
continue
|
||||
}
|
||||
dst.generation = is.db.generation
|
||||
// Found valid dst.
|
||||
return true
|
||||
}
|
||||
@@ -1964,7 +1937,8 @@ func (is *indexSearch) searchMetricName(dst []byte, metricID uint64) ([]byte, bo
|
||||
return dst, true
|
||||
}
|
||||
|
||||
func (is *indexSearch) containsTimeRange(tr TimeRange) bool {
|
||||
// TODO(@rtm0): Move to index_db_legacy.go
|
||||
func (is *indexSearch) legacyContainsTimeRange(tr TimeRange) bool {
|
||||
if tr == globalIndexTimeRange {
|
||||
return true
|
||||
}
|
||||
@@ -1974,8 +1948,6 @@ func (is *indexSearch) containsTimeRange(tr TimeRange) bool {
|
||||
// indexDB could register new time series - it is not safe to cache minMissingTimestamp
|
||||
return true
|
||||
}
|
||||
// The db corresponds to the previous indexDB, which is readonly.
|
||||
// So it is safe caching the minimum timestamp, which isn't covered by the db.
|
||||
|
||||
// use common prefix as a key for minMissingTimestamp
|
||||
// it's needed to properly track timestamps for cluster version
|
||||
@@ -1984,33 +1956,34 @@ func (is *indexSearch) containsTimeRange(tr TimeRange) bool {
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixDateToMetricID)
|
||||
key := kb.B
|
||||
|
||||
db.minMissingTimestampByKeyLock.Lock()
|
||||
minMissingTimestamp, ok := db.minMissingTimestampByKey[string(key)]
|
||||
db.minMissingTimestampByKeyLock.Unlock()
|
||||
db.legacyMinMissingTimestampByKeyLock.Lock()
|
||||
minMissingTimestamp, ok := db.legacyMinMissingTimestampByKey[string(key)]
|
||||
db.legacyMinMissingTimestampByKeyLock.Unlock()
|
||||
|
||||
if ok && tr.MinTimestamp >= minMissingTimestamp {
|
||||
return false
|
||||
}
|
||||
if is.containsTimeRangeSlowForPrefixBuf(kb, tr) {
|
||||
if is.legacyContainsTimeRangeSlow(kb, tr) {
|
||||
return true
|
||||
}
|
||||
|
||||
db.minMissingTimestampByKeyLock.Lock()
|
||||
minMissingTimestamp, ok = db.minMissingTimestampByKey[string(key)]
|
||||
db.legacyMinMissingTimestampByKeyLock.Lock()
|
||||
minMissingTimestamp, ok = db.legacyMinMissingTimestampByKey[string(key)]
|
||||
if !ok || tr.MinTimestamp < minMissingTimestamp {
|
||||
db.minMissingTimestampByKey[string(key)] = tr.MinTimestamp
|
||||
db.legacyMinMissingTimestampByKey[string(key)] = tr.MinTimestamp
|
||||
}
|
||||
db.minMissingTimestampByKeyLock.Unlock()
|
||||
db.legacyMinMissingTimestampByKeyLock.Unlock()
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
func (is *indexSearch) containsTimeRangeSlowForPrefixBuf(prefixBuf *bytesutil.ByteBuffer, tr TimeRange) bool {
|
||||
// TODO(@rtm0): Move to index_db_legacy.go
|
||||
func (is *indexSearch) legacyContainsTimeRangeSlow(prefixBuf *bytesutil.ByteBuffer, tr TimeRange) bool {
|
||||
ts := &is.ts
|
||||
|
||||
// Verify whether the tr.MinTimestamp is included into `ts` or is smaller than the minimum date stored in `ts`.
|
||||
// Do not check whether tr.MaxTimestamp is included into `ts` or is bigger than the max date stored in `ts` for performance reasons.
|
||||
// This means that containsTimeRangeSlow() can return true if `tr` is located below the min date stored in `ts`.
|
||||
// This means that this func can return true if `tr` is located below the min date stored in `ts`.
|
||||
// This is OK, since this case isn't encountered too much in practice.
|
||||
// The main practical case allows skipping searching in prev indexdb (`ts`) when `tr`
|
||||
// is located above the max date stored there.
|
||||
@@ -2301,7 +2274,7 @@ func (is *indexSearch) searchMetricIDs(qt *querytracer.Tracer, tfss []*TagFilter
|
||||
}
|
||||
|
||||
// Filter out deleted metricIDs.
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
dmis := is.db.getDeletedMetricIDs()
|
||||
metricIDs.Subtract(dmis)
|
||||
|
||||
return metricIDs, nil
|
||||
@@ -2320,9 +2293,7 @@ func (is *indexSearch) searchMetricIDsInternal(qt *querytracer.Tracer, tfss []*T
|
||||
|
||||
metricIDs := &uint64set.Set{}
|
||||
|
||||
// Always returns (true, nil) for zero time range used to indicate global
|
||||
// index search.
|
||||
if !is.containsTimeRange(tr) {
|
||||
if !is.legacyContainsTimeRange(tr) {
|
||||
qt.Printf("indexdb doesn't contain data for the given timeRange=%s", &tr)
|
||||
return metricIDs, nil
|
||||
}
|
||||
@@ -2814,6 +2785,10 @@ const (
|
||||
)
|
||||
|
||||
func (db *indexDB) createPerDayIndexes(date uint64, tsid *TSID, mn *MetricName) {
|
||||
if db.noRegisterNewSeries.Load() {
|
||||
logger.Panicf("BUG: registration of new series is disabled for indexDB %q", db.name)
|
||||
}
|
||||
|
||||
if db.s.disablePerDayIndex {
|
||||
return
|
||||
}
|
||||
|
||||
85
lib/storage/index_db_legacy.go
Normal file
85
lib/storage/index_db_legacy.go
Normal file
@@ -0,0 +1,85 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"math"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// legacyIndexDB is a wrapper around indexDB that provides reference counting
|
||||
type legacyIndexDB struct {
|
||||
// The number of references to legacyIndexDB struct.
|
||||
refCount atomic.Int32
|
||||
|
||||
// if the mustDrop is set to true, then the legacyIndexDB must be dropped after refCount reaches zero.
|
||||
mustDrop atomic.Bool
|
||||
|
||||
idb *indexDB
|
||||
}
|
||||
|
||||
func (db *legacyIndexDB) incRef() {
|
||||
db.refCount.Add(1)
|
||||
}
|
||||
|
||||
func (db *legacyIndexDB) decRef() {
|
||||
n := db.refCount.Add(-1)
|
||||
if n < 0 {
|
||||
logger.Panicf("BUG: %q negative refCount: %d", db.idb.name, n)
|
||||
}
|
||||
if n > 0 {
|
||||
return
|
||||
}
|
||||
|
||||
tbPath := db.idb.tb.Path()
|
||||
db.idb.MustClose()
|
||||
|
||||
if !db.mustDrop.Load() {
|
||||
return
|
||||
}
|
||||
|
||||
logger.Infof("dropping indexDB %q", tbPath)
|
||||
fs.MustRemoveDir(tbPath)
|
||||
logger.Infof("indexDB %q has been dropped", tbPath)
|
||||
}
|
||||
|
||||
func (db *legacyIndexDB) scheduleToDrop() {
|
||||
db.mustDrop.Store(true)
|
||||
}
|
||||
|
||||
func (db *legacyIndexDB) MustClose() {
|
||||
rc := db.refCount.Load()
|
||||
if rc != 1 {
|
||||
logger.Fatalf("BUG: %q unexpected legacy indexDB refCount: %d", db.idb.name, rc)
|
||||
}
|
||||
db.decRef()
|
||||
}
|
||||
|
||||
func (db *legacyIndexDB) UpdateMetrics(m *IndexDBMetrics) {
|
||||
db.idb.UpdateMetrics(m)
|
||||
m.IndexDBRefCount += uint64(db.refCount.Load())
|
||||
}
|
||||
|
||||
// mustOpenLegacyIndexDB opens legacy index db from the given path.
|
||||
//
|
||||
// The last segment of the path should contain unique hex value which
|
||||
// will be then used as indexDB.generation
|
||||
func mustOpenLegacyIndexDB(path string, s *Storage) *legacyIndexDB {
|
||||
name := filepath.Base(path)
|
||||
id, err := strconv.ParseUint(name, 16, 64)
|
||||
if err != nil {
|
||||
logger.Panicf("FATAL: cannot parse indexdb path %q: %s", path, err)
|
||||
}
|
||||
|
||||
tr := TimeRange{
|
||||
MinTimestamp: 0,
|
||||
MaxTimestamp: math.MaxInt64,
|
||||
}
|
||||
idb := mustOpenIndexDB(id, tr, name, path, s, &s.isReadOnly, true)
|
||||
legacyIDB := &legacyIndexDB{idb: idb}
|
||||
legacyIDB.incRef()
|
||||
return legacyIDB
|
||||
}
|
||||
136
lib/storage/index_db_legacy_test.go
Normal file
136
lib/storage/index_db_legacy_test.go
Normal file
@@ -0,0 +1,136 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestLegacyContainsTimeRange(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
const numMetrics = 10000
|
||||
trPrev := TimeRange{
|
||||
MinTimestamp: time.Date(2025, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 1, 15, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
trCurr := TimeRange{
|
||||
MinTimestamp: time.Date(2025, 1, 16, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 1, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
trPt := TimeRange{
|
||||
MinTimestamp: time.Date(2025, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 1, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
mrsPrev := testGenerateMetricRowsWithPrefix(rng, numMetrics, "legacy_prev", trPrev)
|
||||
mrsCurr := testGenerateMetricRowsWithPrefix(rng, numMetrics, "legacy_curr", trCurr)
|
||||
mrsPt := testGenerateMetricRowsWithPrefix(rng, numMetrics, "pt", trPt)
|
||||
|
||||
f := func(idb *indexDB, tr TimeRange, want bool) {
|
||||
t.Helper()
|
||||
is := idb.getIndexSearch(noDeadline)
|
||||
defer idb.putIndexSearch(is)
|
||||
|
||||
got := is.legacyContainsTimeRange(tr)
|
||||
|
||||
if got != want {
|
||||
t.Fatalf("legacyContainsTimeRange(%s) for index db %s returns unexpected result: got %t, want %t", tr.String(), idb.name, got, want)
|
||||
}
|
||||
}
|
||||
|
||||
// fill legacy index with data
|
||||
s := MustOpenStorage(t.Name(), OpenOptions{})
|
||||
s.AddRows(mrsPrev, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
s = mustConvertToLegacy(s)
|
||||
s.AddRows(mrsCurr, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
s = mustConvertToLegacy(s)
|
||||
// fill partitioned index with data
|
||||
s.AddRows(mrsPt, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
defer s.MustClose()
|
||||
|
||||
legacyIDBs := s.getLegacyIndexDBs()
|
||||
defer s.putLegacyIndexDBs(legacyIDBs)
|
||||
|
||||
ptws := s.tb.GetPartitions(trPt)
|
||||
defer s.tb.PutPartitions(ptws)
|
||||
if len(ptws) != 1 {
|
||||
t.Fatalf("unexpected number of partitions for one month time range %v: got %d, want 1", &trPt, len(ptws))
|
||||
}
|
||||
idb := ptws[0].pt.idb
|
||||
|
||||
var tr TimeRange
|
||||
|
||||
// Global index time range.
|
||||
tr = globalIndexTimeRange
|
||||
f(legacyIDBs.getIDBPrev(), tr, true)
|
||||
f(legacyIDBs.getIDBCurr(), tr, true)
|
||||
f(idb, tr, true)
|
||||
|
||||
// Fully before trPrev, trCurr, and trPt.
|
||||
tr = TimeRange{
|
||||
MinTimestamp: time.Date(2024, 12, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 12, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
f(legacyIDBs.getIDBPrev(), tr, true)
|
||||
f(legacyIDBs.getIDBCurr(), tr, true)
|
||||
f(idb, tr, true)
|
||||
|
||||
// Overlaps with trPrev and trPt on the left side, fully before trCurr.
|
||||
tr = TimeRange{
|
||||
MinTimestamp: time.Date(2024, 12, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 1, 7, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
f(legacyIDBs.getIDBPrev(), tr, true)
|
||||
f(legacyIDBs.getIDBCurr(), tr, true)
|
||||
f(idb, tr, true)
|
||||
|
||||
// Fully inside trPrev and trPt, fully before trCurr.
|
||||
tr = TimeRange{
|
||||
MinTimestamp: time.Date(2025, 1, 2, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 1, 7, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
f(legacyIDBs.getIDBPrev(), tr, true)
|
||||
f(legacyIDBs.getIDBCurr(), tr, true)
|
||||
f(idb, tr, true)
|
||||
|
||||
// Fully inside trPt, overlaps with trPrev on the right side and trCurr on
|
||||
// the left side.
|
||||
tr = TimeRange{
|
||||
MinTimestamp: time.Date(2025, 1, 7, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 1, 21, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
f(legacyIDBs.getIDBPrev(), tr, true)
|
||||
f(legacyIDBs.getIDBCurr(), tr, true)
|
||||
f(idb, tr, true)
|
||||
|
||||
// Fully inside trPt and trCurr, fully after trPrev.
|
||||
tr = TimeRange{
|
||||
MinTimestamp: time.Date(2025, 1, 18, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 1, 21, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
f(legacyIDBs.getIDBPrev(), tr, false)
|
||||
f(legacyIDBs.getIDBCurr(), tr, true)
|
||||
f(idb, tr, true)
|
||||
|
||||
// Overlaps with trPt and trCurr on the right side, fully after trPrev.
|
||||
tr = TimeRange{
|
||||
MinTimestamp: time.Date(2025, 1, 21, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 2, 21, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
f(legacyIDBs.getIDBPrev(), tr, false)
|
||||
f(legacyIDBs.getIDBCurr(), tr, true)
|
||||
f(idb, tr, true)
|
||||
|
||||
// fully after trPrev, trCurr, and trPt.
|
||||
tr = TimeRange{
|
||||
MinTimestamp: time.Date(2025, 2, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 3, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
f(legacyIDBs.getIDBPrev(), tr, false)
|
||||
f(legacyIDBs.getIDBCurr(), tr, false)
|
||||
f(idb, tr, true)
|
||||
}
|
||||
@@ -20,7 +20,6 @@ import (
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/mergeset"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/uint64set"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/workingsetcache"
|
||||
"github.com/VictoriaMetrics/fastcache"
|
||||
)
|
||||
|
||||
func TestTagFiltersToMetricIDsCache(t *testing.T) {
|
||||
@@ -33,14 +32,15 @@ func TestTagFiltersToMetricIDsCache(t *testing.T) {
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
defer s.MustClose()
|
||||
|
||||
idbPrev, idbCurr, idbNext := s.getIndexDBs()
|
||||
defer s.putIndexDBs(idbPrev, idbCurr, idbNext)
|
||||
ptw := s.tb.MustGetPartition(time.Now().UnixMilli())
|
||||
idb := ptw.pt.idb
|
||||
defer s.tb.PutPartition(ptw)
|
||||
|
||||
key := []byte("key")
|
||||
wantSet := &uint64set.Set{}
|
||||
wantSet.AddMulti(want)
|
||||
idbCurr.putMetricIDsToTagFiltersCache(nil, wantSet, key)
|
||||
gotSet, ok := idbCurr.getMetricIDsFromTagFiltersCache(nil, key)
|
||||
idb.putMetricIDsToTagFiltersCache(nil, wantSet, key)
|
||||
gotSet, ok := idb.getMetricIDsFromTagFiltersCache(nil, key)
|
||||
if !ok {
|
||||
t.Fatalf("expected metricIDs to be found in cache but they weren't: %v", want)
|
||||
}
|
||||
@@ -62,12 +62,13 @@ func TestTagFiltersToMetricIDsCache_EmptyMetricIDList(t *testing.T) {
|
||||
defer fs.MustRemoveDir(path)
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
defer s.MustClose()
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
defer s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
ptw := s.tb.MustGetPartition(time.Now().UnixMilli())
|
||||
idb := ptw.pt.idb
|
||||
defer s.tb.PutPartition(ptw)
|
||||
|
||||
key := []byte("key")
|
||||
idbCurr.putMetricIDsToTagFiltersCache(nil, nil, key)
|
||||
got, ok := idbCurr.getMetricIDsFromTagFiltersCache(nil, key)
|
||||
idb.putMetricIDsToTagFiltersCache(nil, nil, key)
|
||||
got, ok := idb.getMetricIDsFromTagFiltersCache(nil, key)
|
||||
if !ok {
|
||||
t.Fatalf("expected empty metricID list to be found in cache but it wasn't")
|
||||
}
|
||||
@@ -471,14 +472,15 @@ func TestRemoveDuplicateMetricIDs(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestIndexDBOpenClose(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
var s Storage
|
||||
tableName := nextIndexDBTableName()
|
||||
path := filepath.Join(t.Name(), "2025_01")
|
||||
for i := 0; i < 5; i++ {
|
||||
var isReadOnly atomic.Bool
|
||||
db := mustOpenIndexDB(tableName, &s, &isReadOnly, false)
|
||||
db := mustOpenIndexDB(123, TimeRange{}, "name", path, &s, &isReadOnly, false)
|
||||
db.MustClose()
|
||||
}
|
||||
fs.MustRemoveDir(tableName)
|
||||
}
|
||||
|
||||
func TestIndexDB(t *testing.T) {
|
||||
@@ -488,27 +490,28 @@ func TestIndexDB(t *testing.T) {
|
||||
t.Run("serial", func(t *testing.T) {
|
||||
const path = "TestIndexDB-serial"
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
mns, tsids, err := testIndexDBGetOrCreateTSIDByName(idbCurr, metricGroups, timestamp)
|
||||
ptw := s.tb.MustGetPartition(timestamp)
|
||||
db := ptw.pt.idb
|
||||
mns, tsids, err := testIndexDBGetOrCreateTSIDByName(db, metricGroups, timestamp)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
if err := testIndexDBCheckTSIDByName(idbCurr, mns, tsids, timestamp, false); err != nil {
|
||||
if err := testIndexDBCheckTSIDByName(db, mns, tsids, timestamp, false); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
|
||||
// Re-open the storage and verify it works as expected.
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
s.tb.PutPartition(ptw)
|
||||
s.MustClose()
|
||||
s = MustOpenStorage(path, OpenOptions{})
|
||||
|
||||
idbPrev, idbCurr = s.getPrevAndCurrIndexDBs()
|
||||
if err := testIndexDBCheckTSIDByName(idbCurr, mns, tsids, timestamp, false); err != nil {
|
||||
ptw = s.tb.MustGetPartition(timestamp)
|
||||
db = ptw.pt.idb
|
||||
if err := testIndexDBCheckTSIDByName(db, mns, tsids, timestamp, false); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
s.tb.PutPartition(ptw)
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
})
|
||||
@@ -516,17 +519,18 @@ func TestIndexDB(t *testing.T) {
|
||||
t.Run("concurrent", func(t *testing.T) {
|
||||
const path = "TestIndexDB-concurrent"
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
ptw := s.tb.MustGetPartition(timestamp)
|
||||
db := ptw.pt.idb
|
||||
|
||||
ch := make(chan error, 3)
|
||||
for i := 0; i < cap(ch); i++ {
|
||||
go func() {
|
||||
mns, tsid, err := testIndexDBGetOrCreateTSIDByName(idbCurr, metricGroups, timestamp)
|
||||
mns, tsid, err := testIndexDBGetOrCreateTSIDByName(db, metricGroups, timestamp)
|
||||
if err != nil {
|
||||
ch <- err
|
||||
return
|
||||
}
|
||||
if err := testIndexDBCheckTSIDByName(idbCurr, mns, tsid, timestamp, true); err != nil {
|
||||
if err := testIndexDBCheckTSIDByName(db, mns, tsid, timestamp, true); err != nil {
|
||||
ch <- err
|
||||
return
|
||||
}
|
||||
@@ -545,7 +549,7 @@ func TestIndexDB(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
s.tb.PutPartition(ptw)
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
})
|
||||
@@ -579,26 +583,26 @@ func testIndexDBGetOrCreateTSIDByName(db *indexDB, metricGroups int, timestamp i
|
||||
metricNameBuf = mn.Marshal(metricNameBuf[:0])
|
||||
|
||||
// Create tsid for the metricName.
|
||||
var genTSID generationTSID
|
||||
if !is.getTSIDByMetricName(&genTSID, metricNameBuf, date) {
|
||||
generateTSID(&genTSID.TSID, &mn)
|
||||
createAllIndexesForMetricName(db, &mn, &genTSID.TSID, date)
|
||||
var tsid TSID
|
||||
if !is.getTSIDByMetricName(&tsid, metricNameBuf, date) {
|
||||
generateTSID(&tsid, &mn)
|
||||
createAllIndexesForMetricName(db, &mn, &tsid, date)
|
||||
}
|
||||
|
||||
mns = append(mns, mn)
|
||||
tsids = append(tsids, genTSID.TSID)
|
||||
tsids = append(tsids, tsid)
|
||||
}
|
||||
|
||||
db.putIndexSearch(is)
|
||||
|
||||
// Flush index to disk, so it becomes visible for search
|
||||
db.s.DebugFlush()
|
||||
db.tb.DebugFlush()
|
||||
|
||||
return mns, tsids, nil
|
||||
}
|
||||
|
||||
func testIndexDBCheckTSIDByName(db *indexDB, mns []MetricName, tsids []TSID, timestamp int64, isConcurrent bool) error {
|
||||
timeseriesCounters := make(map[uint64]bool)
|
||||
var genTSID generationTSID
|
||||
var tsidLocal TSID
|
||||
var metricNameCopy []byte
|
||||
allLabelNames := make(map[string]bool)
|
||||
@@ -613,12 +617,11 @@ func testIndexDBCheckTSIDByName(db *indexDB, mns []MetricName, tsids []TSID, tim
|
||||
metricName := mn.Marshal(nil)
|
||||
|
||||
is := db.getIndexSearch(noDeadline)
|
||||
if !is.getTSIDByMetricName(&genTSID, metricName, uint64(timestamp)/msecPerDay) {
|
||||
if !is.getTSIDByMetricName(&tsidLocal, metricName, uint64(timestamp)/msecPerDay) {
|
||||
return fmt.Errorf("cannot obtain tsid #%d for mn %s", i, mn)
|
||||
}
|
||||
db.putIndexSearch(is)
|
||||
|
||||
tsidLocal = genTSID.TSID
|
||||
if isConcurrent {
|
||||
// Copy tsid.MetricID, since multiple TSIDs may match
|
||||
// the same mn in concurrent mode.
|
||||
@@ -1419,103 +1422,8 @@ func TestMatchTagFilters(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestIndexDBRepopulateAfterRotation(t *testing.T) {
|
||||
r := rand.New(rand.NewSource(1))
|
||||
path := "TestIndexRepopulateAfterRotation"
|
||||
opts := OpenOptions{
|
||||
Retention: retention31Days,
|
||||
MaxHourlySeries: 1e5,
|
||||
MaxDailySeries: 1e5,
|
||||
}
|
||||
s := MustOpenStorage(path, opts)
|
||||
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
if idbCurr.generation == 0 {
|
||||
t.Fatalf("expected indexDB generation to be not 0")
|
||||
}
|
||||
|
||||
const metricRowsN = 1000
|
||||
|
||||
currentDayTimestamp := (time.Now().UnixMilli() / msecPerDay) * msecPerDay
|
||||
timeMin := currentDayTimestamp - 24*3600*1000
|
||||
timeMax := currentDayTimestamp + 24*3600*1000
|
||||
mrs := testGenerateMetricRows(r, metricRowsN, timeMin, timeMax)
|
||||
s.AddRows(mrs, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
|
||||
// verify the storage contains rows.
|
||||
var m Metrics
|
||||
s.UpdateMetrics(&m)
|
||||
if rowsCount := m.TableMetrics.TotalRowsCount(); rowsCount < uint64(metricRowsN) {
|
||||
t.Fatalf("expecting at least %d rows in the table; got %d", metricRowsN, rowsCount)
|
||||
}
|
||||
|
||||
// check new series were registered in indexDB
|
||||
added := idbCurr.s.newTimeseriesCreated.Load()
|
||||
if added != metricRowsN {
|
||||
t.Fatalf("expected indexDB to contain %d rows; got %d", metricRowsN, added)
|
||||
}
|
||||
|
||||
// check new series were added to cache
|
||||
var cs fastcache.Stats
|
||||
s.tsidCache.UpdateStats(&cs)
|
||||
if cs.EntriesCount != metricRowsN {
|
||||
t.Fatalf("expected tsidCache to contain %d rows; got %d", metricRowsN, cs.EntriesCount)
|
||||
}
|
||||
|
||||
// check if cache entries do belong to current indexDB generation
|
||||
var genTSID generationTSID
|
||||
for _, mr := range mrs {
|
||||
s.getTSIDFromCache(&genTSID, mr.MetricNameRaw)
|
||||
if genTSID.generation != idbCurr.generation {
|
||||
t.Fatalf("expected all entries in tsidCache to have the same indexDB generation: %d;"+
|
||||
"got %d", idbCurr.generation, genTSID.generation)
|
||||
}
|
||||
}
|
||||
prevGeneration := idbCurr.generation
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
|
||||
// force index rotation
|
||||
s.mustRotateIndexDB(time.Now())
|
||||
|
||||
// check tsidCache wasn't reset after the rotation
|
||||
var cs2 fastcache.Stats
|
||||
s.tsidCache.UpdateStats(&cs2)
|
||||
if cs.EntriesCount != metricRowsN {
|
||||
t.Fatalf("expected tsidCache after rotation to contain %d rows; got %d", metricRowsN, cs2.EntriesCount)
|
||||
}
|
||||
|
||||
idbPrev, idbCurr = s.getPrevAndCurrIndexDBs()
|
||||
if idbCurr.generation == 0 {
|
||||
t.Fatalf("expected new indexDB generation to be not 0")
|
||||
}
|
||||
if idbCurr.generation == prevGeneration {
|
||||
t.Fatalf("expected new indexDB generation %d to be different from prev indexDB", idbCurr.generation)
|
||||
}
|
||||
|
||||
// Re-insert rows again and verify that all the entries belong to new generation
|
||||
s.AddRows(mrs, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
|
||||
for _, mr := range mrs {
|
||||
s.getTSIDFromCache(&genTSID, mr.MetricNameRaw)
|
||||
if genTSID.generation != idbCurr.generation {
|
||||
t.Fatalf("unexpected generation for data after rotation; got %d; want %d", genTSID.generation, idbCurr.generation)
|
||||
}
|
||||
}
|
||||
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
}
|
||||
|
||||
func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
const path = "TestSearchTSIDWithTimeRange"
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
|
||||
is := idbCurr.getIndexSearch(noDeadline)
|
||||
|
||||
// Create a bunch of per-day time series
|
||||
const days = 5
|
||||
const metricsPerDay = 1000
|
||||
@@ -1554,29 +1462,35 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
mn.sortTags()
|
||||
return mn
|
||||
}
|
||||
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
ptw := s.tb.MustGetPartition(timestamp)
|
||||
db := ptw.pt.idb
|
||||
is := db.getIndexSearch(noDeadline)
|
||||
|
||||
for day := 0; day < days; day++ {
|
||||
date := baseDate - uint64(day)
|
||||
var metricIDs uint64set.Set
|
||||
for metric := 0; metric < metricsPerDay; metric++ {
|
||||
mn := newMN("testMetric", day, metric)
|
||||
metricNameBuf = mn.Marshal(metricNameBuf[:0])
|
||||
var genTSID generationTSID
|
||||
if !is.getTSIDByMetricName(&genTSID, metricNameBuf, date) {
|
||||
generateTSID(&genTSID.TSID, &mn)
|
||||
createAllIndexesForMetricName(idbCurr, &mn, &genTSID.TSID, date)
|
||||
var tsid TSID
|
||||
if !is.getTSIDByMetricName(&tsid, metricNameBuf, date) {
|
||||
generateTSID(&tsid, &mn)
|
||||
createAllIndexesForMetricName(db, &mn, &tsid, date)
|
||||
}
|
||||
metricIDs.Add(genTSID.TSID.MetricID)
|
||||
metricIDs.Add(tsid.MetricID)
|
||||
}
|
||||
|
||||
allMetricIDs.Union(&metricIDs)
|
||||
perDayMetricIDs[date] = &metricIDs
|
||||
}
|
||||
idbCurr.putIndexSearch(is)
|
||||
db.putIndexSearch(is)
|
||||
|
||||
// Flush index to disk, so it becomes visible for search
|
||||
s.DebugFlush()
|
||||
db.tb.DebugFlush()
|
||||
|
||||
is2 := idbCurr.getIndexSearch(noDeadline)
|
||||
is2 := db.getIndexSearch(noDeadline)
|
||||
|
||||
// Check that all the metrics are found for all the days.
|
||||
for date := baseDate - days + 1; date <= baseDate; date++ {
|
||||
@@ -1597,10 +1511,10 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
if !allMetricIDs.Equal(metricIDs) {
|
||||
t.Fatalf("unexpected metricIDs found;\ngot\n%d\nwant\n%d", metricIDs.AppendTo(nil), allMetricIDs.AppendTo(nil))
|
||||
}
|
||||
idbCurr.putIndexSearch(is2)
|
||||
db.putIndexSearch(is2)
|
||||
|
||||
// add a metric that will be deleted shortly
|
||||
is3 := idbCurr.getIndexSearch(noDeadline)
|
||||
is3 := db.getIndexSearch(noDeadline)
|
||||
day := days
|
||||
date := baseDate - uint64(day)
|
||||
mn := newMN("deletedMetric", day, 999)
|
||||
@@ -1610,24 +1524,24 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
)
|
||||
mn.sortTags()
|
||||
metricNameBuf = mn.Marshal(metricNameBuf[:0])
|
||||
var genTSID generationTSID
|
||||
if !is3.getTSIDByMetricName(&genTSID, metricNameBuf, date) {
|
||||
generateTSID(&genTSID.TSID, &mn)
|
||||
createAllIndexesForMetricName(idbCurr, &mn, &genTSID.TSID, date)
|
||||
var tsid TSID
|
||||
if !is3.getTSIDByMetricName(&tsid, metricNameBuf, date) {
|
||||
generateTSID(&tsid, &mn)
|
||||
createAllIndexesForMetricName(db, &mn, &tsid, date)
|
||||
}
|
||||
// delete the added metric. It is expected it won't be returned during searches
|
||||
deletedSet := &uint64set.Set{}
|
||||
deletedSet.Add(genTSID.TSID.MetricID)
|
||||
s.setDeletedMetricIDs(deletedSet)
|
||||
idbCurr.putIndexSearch(is3)
|
||||
s.DebugFlush()
|
||||
deletedSet.Add(tsid.MetricID)
|
||||
db.setDeletedMetricIDs(deletedSet)
|
||||
db.putIndexSearch(is3)
|
||||
db.tb.DebugFlush()
|
||||
|
||||
// Check SearchLabelNames with the specified time range.
|
||||
tr := TimeRange{
|
||||
MinTimestamp: int64(timestamp) - msecPerDay,
|
||||
MaxTimestamp: int64(timestamp),
|
||||
MinTimestamp: timestamp - msecPerDay,
|
||||
MaxTimestamp: timestamp,
|
||||
}
|
||||
lns, err := idbCurr.SearchLabelNames(nil, nil, tr, 10000, 1e9, noDeadline)
|
||||
lns, err := db.SearchLabelNames(nil, nil, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelNames(timeRange=%s): %s", &tr, err)
|
||||
}
|
||||
@@ -1637,7 +1551,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelValues with the specified time range.
|
||||
lvs, err := idbCurr.SearchLabelValues(nil, "", nil, tr, 10000, 1e9, noDeadline)
|
||||
lvs, err := db.SearchLabelValues(nil, "", nil, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelValues(timeRange=%s): %s", &tr, err)
|
||||
}
|
||||
@@ -1666,10 +1580,10 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
// Perform a search within a day.
|
||||
// This should return the metrics for the day
|
||||
tr = TimeRange{
|
||||
MinTimestamp: int64(timestamp - 2*msecPerHour - 1),
|
||||
MaxTimestamp: int64(timestamp),
|
||||
MinTimestamp: timestamp - 2*msecPerHour - 1,
|
||||
MaxTimestamp: timestamp,
|
||||
}
|
||||
matchedTSIDs, err := idbCurr.SearchTSIDs(nil, []*TagFilters{tfs}, tr, 1e5, noDeadline)
|
||||
matchedTSIDs, err := db.SearchTSIDs(nil, []*TagFilters{tfs}, tr, 1e5, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error searching tsids: %v", err)
|
||||
}
|
||||
@@ -1678,7 +1592,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelNames with the specified filter.
|
||||
lns, err = idbCurr.SearchLabelNames(nil, []*TagFilters{tfs}, TimeRange{}, 10000, 1e9, noDeadline)
|
||||
lns, err = db.SearchLabelNames(nil, []*TagFilters{tfs}, TimeRange{}, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelNames(filters=%s): %s", tfs, err)
|
||||
}
|
||||
@@ -1688,7 +1602,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelNames with the specified filter and time range.
|
||||
lns, err = idbCurr.SearchLabelNames(nil, []*TagFilters{tfs}, tr, 10000, 1e9, noDeadline)
|
||||
lns, err = db.SearchLabelNames(nil, []*TagFilters{tfs}, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelNames(filters=%s, timeRange=%s): %s", tfs, &tr, err)
|
||||
}
|
||||
@@ -1698,7 +1612,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelNames with filters on metric name and time range.
|
||||
lns, err = idbCurr.SearchLabelNames(nil, []*TagFilters{tfsMetricName}, tr, 10000, 1e9, noDeadline)
|
||||
lns, err = db.SearchLabelNames(nil, []*TagFilters{tfsMetricName}, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelNames(filters=%s, timeRange=%s): %s", tfs, &tr, err)
|
||||
}
|
||||
@@ -1708,7 +1622,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelNames with filters on composite key and time range.
|
||||
lns, err = idbCurr.SearchLabelNames(nil, []*TagFilters{tfsComposite}, tr, 10000, 1e9, noDeadline)
|
||||
lns, err = db.SearchLabelNames(nil, []*TagFilters{tfsComposite}, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelNames(filters=%s, timeRange=%s): %s", tfs, &tr, err)
|
||||
}
|
||||
@@ -1718,7 +1632,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelValues with the specified filter.
|
||||
lvs, err = idbCurr.SearchLabelValues(nil, "", []*TagFilters{tfs}, TimeRange{}, 10000, 1e9, noDeadline)
|
||||
lvs, err = db.SearchLabelValues(nil, "", []*TagFilters{tfs}, TimeRange{}, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelValues(filters=%s): %s", tfs, err)
|
||||
}
|
||||
@@ -1728,7 +1642,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelValues with the specified filter and time range.
|
||||
lvs, err = idbCurr.SearchLabelValues(nil, "", []*TagFilters{tfs}, tr, 10000, 1e9, noDeadline)
|
||||
lvs, err = db.SearchLabelValues(nil, "", []*TagFilters{tfs}, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelValues(filters=%s, timeRange=%s): %s", tfs, &tr, err)
|
||||
}
|
||||
@@ -1738,7 +1652,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelValues with filters on metric name and time range.
|
||||
lvs, err = idbCurr.SearchLabelValues(nil, "", []*TagFilters{tfsMetricName}, tr, 10000, 1e9, noDeadline)
|
||||
lvs, err = db.SearchLabelValues(nil, "", []*TagFilters{tfsMetricName}, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelValues(filters=%s, timeRange=%s): %s", tfs, &tr, err)
|
||||
}
|
||||
@@ -1748,7 +1662,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check SearchLabelValues with filters on composite key and time range.
|
||||
lvs, err = idbCurr.SearchLabelValues(nil, "constant", []*TagFilters{tfsComposite}, tr, 10000, 1e9, noDeadline)
|
||||
lvs, err = db.SearchLabelValues(nil, "constant", []*TagFilters{tfsComposite}, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelValues(filters=%s, timeRange=%s): %s", tfs, &tr, err)
|
||||
}
|
||||
@@ -1760,11 +1674,11 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
|
||||
// Perform a search across all the days, should match all metrics
|
||||
tr = TimeRange{
|
||||
MinTimestamp: int64(timestamp - msecPerDay*days),
|
||||
MaxTimestamp: int64(timestamp),
|
||||
MinTimestamp: timestamp - msecPerDay*days,
|
||||
MaxTimestamp: timestamp,
|
||||
}
|
||||
|
||||
matchedTSIDs, err = idbCurr.SearchTSIDs(nil, []*TagFilters{tfs}, tr, 1e5, noDeadline)
|
||||
matchedTSIDs, err = db.SearchTSIDs(nil, []*TagFilters{tfs}, tr, 1e5, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error searching tsids: %v", err)
|
||||
}
|
||||
@@ -1773,7 +1687,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check GetTSDBStatus with nil filters.
|
||||
status, err := idbCurr.GetTSDBStatus(nil, nil, baseDate, "day", 5, 1e6, noDeadline)
|
||||
status, err := db.GetTSDBStatus(nil, nil, baseDate, "day", 5, 1e6, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error in GetTSDBStatus with nil filters: %s", err)
|
||||
}
|
||||
@@ -1887,7 +1801,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
if err := tfs.Add([]byte("day"), []byte("0"), false, false); err != nil {
|
||||
t.Fatalf("cannot add filter: %s", err)
|
||||
}
|
||||
status, err = idbCurr.GetTSDBStatus(nil, []*TagFilters{tfs}, baseDate, "", 5, 1e6, noDeadline)
|
||||
status, err = db.GetTSDBStatus(nil, []*TagFilters{tfs}, baseDate, "", 5, 1e6, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error in GetTSDBStatus: %s", err)
|
||||
}
|
||||
@@ -1913,7 +1827,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check GetTSDBStatus, which matches all the series on a global time range
|
||||
status, err = idbCurr.GetTSDBStatus(nil, nil, 0, "day", 5, 1e6, noDeadline)
|
||||
status, err = db.GetTSDBStatus(nil, nil, 0, "day", 5, 1e6, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error in GetTSDBStatus: %s", err)
|
||||
}
|
||||
@@ -1968,7 +1882,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
if err := tfs.Add([]byte("UniqueId"), []byte("0|1|3"), false, true); err != nil {
|
||||
t.Fatalf("cannot add filter: %s", err)
|
||||
}
|
||||
status, err = idbCurr.GetTSDBStatus(nil, []*TagFilters{tfs}, baseDate, "", 5, 1e6, noDeadline)
|
||||
status, err = db.GetTSDBStatus(nil, []*TagFilters{tfs}, baseDate, "", 5, 1e6, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error in GetTSDBStatus: %s", err)
|
||||
}
|
||||
@@ -1994,7 +1908,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
}
|
||||
|
||||
// Check GetTSDBStatus with non-nil filter on global time range, which matches only 15 series
|
||||
status, err = idbCurr.GetTSDBStatus(nil, []*TagFilters{tfs}, 0, "", 5, 1e6, noDeadline)
|
||||
status, err = db.GetTSDBStatus(nil, []*TagFilters{tfs}, 0, "", 5, 1e6, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error in GetTSDBStatus: %s", err)
|
||||
}
|
||||
@@ -2019,7 +1933,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
||||
t.Fatalf("unexpected TotalLabelValuePairs; got %d; want %d", status.TotalLabelValuePairs, expectedLabelValuePairs)
|
||||
}
|
||||
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
s.tb.PutPartition(ptw)
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
}
|
||||
@@ -2041,7 +1955,6 @@ func newTestStorage() *Storage {
|
||||
tsidCache: workingsetcache.New(1234),
|
||||
retentionMsecs: retentionMax.Milliseconds(),
|
||||
}
|
||||
s.setDeletedMetricIDs(&uint64set.Set{})
|
||||
return s
|
||||
}
|
||||
|
||||
@@ -2052,135 +1965,6 @@ func stopTestStorage(s *Storage) {
|
||||
fs.MustRemoveDir(s.cachePath)
|
||||
}
|
||||
|
||||
func TestSearchContainsTimeRange(t *testing.T) {
|
||||
path := t.Name()
|
||||
fs.MustRemoveDir(path)
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
|
||||
is := idbCurr.getIndexSearch(noDeadline)
|
||||
|
||||
// Create a bunch of per-day time series
|
||||
const (
|
||||
days = 6
|
||||
tenant2IngestionDay = 8
|
||||
metricsPerDay = 1000
|
||||
)
|
||||
rotationDay := time.Date(2019, time.October, 15, 5, 1, 0, 0, time.UTC)
|
||||
rotationMillis := uint64(rotationDay.UnixMilli())
|
||||
rotationDate := rotationMillis / msecPerDay
|
||||
var metricNameBuf []byte
|
||||
perDayMetricIDs := make(map[uint64]*uint64set.Set)
|
||||
labelNames := []string{
|
||||
"__name__", "constant", "day", "UniqueId", "some_unique_id",
|
||||
}
|
||||
|
||||
sort.Strings(labelNames)
|
||||
|
||||
newMN := func(name string, day, metric int) MetricName {
|
||||
var mn MetricName
|
||||
mn.MetricGroup = []byte(name)
|
||||
mn.AddTag(
|
||||
"constant",
|
||||
"const",
|
||||
)
|
||||
mn.AddTag(
|
||||
"day",
|
||||
fmt.Sprintf("%v", day),
|
||||
)
|
||||
mn.AddTag(
|
||||
"UniqueId",
|
||||
fmt.Sprintf("%v", metric),
|
||||
)
|
||||
mn.AddTag(
|
||||
"some_unique_id",
|
||||
fmt.Sprintf("%v", day),
|
||||
)
|
||||
mn.sortTags()
|
||||
return mn
|
||||
}
|
||||
|
||||
// ingest metrics for tenant 0:0
|
||||
for day := 0; day < days; day++ {
|
||||
date := rotationDate - uint64(day)
|
||||
|
||||
var metricIDs uint64set.Set
|
||||
for metric := range metricsPerDay {
|
||||
mn := newMN("testMetric", day, metric)
|
||||
metricNameBuf = mn.Marshal(metricNameBuf[:0])
|
||||
var genTSID generationTSID
|
||||
if !is.getTSIDByMetricName(&genTSID, metricNameBuf, date) {
|
||||
generateTSID(&genTSID.TSID, &mn)
|
||||
createAllIndexesForMetricName(idbCurr, &mn, &genTSID.TSID, date)
|
||||
}
|
||||
metricIDs.Add(genTSID.TSID.MetricID)
|
||||
}
|
||||
|
||||
perDayMetricIDs[date] = &metricIDs
|
||||
}
|
||||
idbCurr.putIndexSearch(is)
|
||||
|
||||
// Flush index to disk, so it becomes visible for search
|
||||
s.DebugFlush()
|
||||
|
||||
is2 := idbCurr.getIndexSearch(noDeadline)
|
||||
|
||||
// Check that all the metrics are found for all the days.
|
||||
for date := rotationDate - days + 1; date <= rotationDate; date++ {
|
||||
|
||||
metricIDs, err := is2.getMetricIDsForDate(date, metricsPerDay)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
if !perDayMetricIDs[date].Equal(metricIDs) {
|
||||
t.Fatalf("unexpected metricIDs found;\ngot\n%d\nwant\n%d", metricIDs.AppendTo(nil), perDayMetricIDs[date].AppendTo(nil))
|
||||
}
|
||||
}
|
||||
|
||||
idbCurr.putIndexSearch(is2)
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
|
||||
// rotate indexdb
|
||||
s.mustRotateIndexDB(rotationDay)
|
||||
idbPrev, idbNext := s.getPrevAndCurrIndexDBs()
|
||||
|
||||
// perform search for 0:0 tenant
|
||||
// results of previous search requests shouldn't affect it
|
||||
|
||||
isPrev := idbPrev.getIndexSearch(noDeadline)
|
||||
// search for range that covers prev indexDB for dates before ingestion
|
||||
tr := TimeRange{
|
||||
MinTimestamp: int64(rotationMillis - msecPerDay*(days)),
|
||||
MaxTimestamp: int64(rotationMillis),
|
||||
}
|
||||
if !isPrev.containsTimeRange(tr) {
|
||||
t.Fatalf("expected to have given time range at prev IndexDB")
|
||||
}
|
||||
|
||||
// search for range not exist at prev indexDB
|
||||
tr = TimeRange{
|
||||
MinTimestamp: int64(rotationMillis + msecPerDay*(days+4)),
|
||||
MaxTimestamp: int64(rotationMillis + msecPerDay*(days+2)),
|
||||
}
|
||||
if isPrev.containsTimeRange(tr) {
|
||||
t.Fatalf("not expected to have given time range at prev IndexDB")
|
||||
}
|
||||
key := isPrev.marshalCommonPrefix(nil, nsPrefixDateToMetricID)
|
||||
|
||||
idbPrev.minMissingTimestampByKeyLock.Lock()
|
||||
minMissingTimetamp := idbPrev.minMissingTimestampByKey[string(key)]
|
||||
idbPrev.minMissingTimestampByKeyLock.Unlock()
|
||||
|
||||
if minMissingTimetamp != tr.MinTimestamp {
|
||||
t.Fatalf("unexpected minMissingTimestamp for 0:0 tenant got %d, want %d", minMissingTimetamp, tr.MinTimestamp)
|
||||
}
|
||||
|
||||
idbPrev.putIndexSearch(isPrev)
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbNext)
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
}
|
||||
|
||||
func sortedSlice(m map[string]struct{}) []string {
|
||||
s := make([]string, 0, len(m))
|
||||
for k := range m {
|
||||
@@ -2190,19 +1974,19 @@ func sortedSlice(m map[string]struct{}) []string {
|
||||
return s
|
||||
}
|
||||
|
||||
func TestIndexSearchContainsTimeRange_Concurrent(t *testing.T) {
|
||||
func TestIndexSearchLegacyContainsTimeRange_Concurrent(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
// Create storage because indexDB depends on it.
|
||||
s := MustOpenStorage(filepath.Join(t.Name(), "storage"), OpenOptions{})
|
||||
defer s.MustClose()
|
||||
|
||||
idbName := nextIndexDBTableName()
|
||||
idbName := "test"
|
||||
idbPath := filepath.Join(t.Name(), indexdbDirname, idbName)
|
||||
var readOnly atomic.Bool
|
||||
readOnly.Store(true)
|
||||
noRegisterNewSeries := true
|
||||
idb := mustOpenIndexDB(idbPath, s, &readOnly, noRegisterNewSeries)
|
||||
idb := mustOpenIndexDB(123, TimeRange{}, idbName, idbPath, s, &readOnly, noRegisterNewSeries)
|
||||
defer idb.MustClose()
|
||||
|
||||
minTimestamp := time.Date(2000, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
@@ -2212,7 +1996,7 @@ func TestIndexSearchContainsTimeRange_Concurrent(t *testing.T) {
|
||||
wg.Add(1)
|
||||
go func(ts int64) {
|
||||
is := idb.getIndexSearch(noDeadline)
|
||||
_ = is.containsTimeRange(TimeRange{ts, ts})
|
||||
_ = is.legacyContainsTimeRange(TimeRange{ts, ts})
|
||||
idb.putIndexSearch(is)
|
||||
wg.Done()
|
||||
}(minTimestamp + msecPerDay*i)
|
||||
@@ -2220,7 +2004,7 @@ func TestIndexSearchContainsTimeRange_Concurrent(t *testing.T) {
|
||||
wg.Wait()
|
||||
|
||||
key := marshalCommonPrefix(nil, nsPrefixDateToMetricID)
|
||||
if got, want := idb.minMissingTimestampByKey[string(key)], minTimestamp; got != want {
|
||||
if got, want := idb.legacyMinMissingTimestampByKey[string(key)], minTimestamp; got != want {
|
||||
t.Fatalf("unexpected min timestamp: got %v, want %v", time.UnixMilli(got).UTC(), time.UnixMilli(want).UTC())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -42,7 +42,8 @@ func BenchmarkIndexDBAddTSIDs(b *testing.B) {
|
||||
const path = "BenchmarkIndexDBAddTSIDs"
|
||||
timestamp := time.Date(2025, 3, 17, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
ptw := s.tb.MustGetPartition(timestamp)
|
||||
db := ptw.pt.idb
|
||||
|
||||
const recordsPerLoop = 1e3
|
||||
|
||||
@@ -51,7 +52,7 @@ func BenchmarkIndexDBAddTSIDs(b *testing.B) {
|
||||
b.ResetTimer()
|
||||
b.RunParallel(func(pb *testing.PB) {
|
||||
var mn MetricName
|
||||
var genTSID generationTSID
|
||||
var tsid TSID
|
||||
|
||||
// The most common tags.
|
||||
mn.Tags = []Tag{
|
||||
@@ -65,18 +66,18 @@ func BenchmarkIndexDBAddTSIDs(b *testing.B) {
|
||||
|
||||
startOffset := 0
|
||||
for pb.Next() {
|
||||
benchmarkIndexDBAddTSIDs(idbCurr, &genTSID, &mn, timestamp, startOffset, recordsPerLoop)
|
||||
benchmarkIndexDBAddTSIDs(db, &tsid, &mn, timestamp, startOffset, recordsPerLoop)
|
||||
startOffset += recordsPerLoop
|
||||
}
|
||||
})
|
||||
b.StopTimer()
|
||||
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
s.tb.PutPartition(ptw)
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
}
|
||||
|
||||
func benchmarkIndexDBAddTSIDs(db *indexDB, genTSID *generationTSID, mn *MetricName, timestamp int64, startOffset, recordsPerLoop int) {
|
||||
func benchmarkIndexDBAddTSIDs(db *indexDB, tsid *TSID, mn *MetricName, timestamp int64, startOffset, recordsPerLoop int) {
|
||||
date := uint64(timestamp) / msecPerDay
|
||||
for i := 0; i < recordsPerLoop; i++ {
|
||||
mn.MetricGroup = strconv.AppendUint(mn.MetricGroup[:0], uint64(i+startOffset), 10)
|
||||
@@ -85,8 +86,8 @@ func benchmarkIndexDBAddTSIDs(db *indexDB, genTSID *generationTSID, mn *MetricNa
|
||||
}
|
||||
mn.sortTags()
|
||||
|
||||
generateTSID(&genTSID.TSID, mn)
|
||||
createAllIndexesForMetricName(db, mn, &genTSID.TSID, date)
|
||||
generateTSID(tsid, mn)
|
||||
createAllIndexesForMetricName(db, mn, tsid, date)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -96,11 +97,12 @@ func BenchmarkHeadPostingForMatchers(b *testing.B) {
|
||||
const path = "BenchmarkHeadPostingForMatchers"
|
||||
timestamp := int64(0)
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
ptw := s.tb.MustGetPartition(timestamp)
|
||||
db := ptw.pt.idb
|
||||
|
||||
// Fill the db with data as in https://github.com/prometheus/prometheus/blob/23c0299d85bfeb5d9b59e994861553a25ca578e5/tsdb/head_bench_test.go#L66
|
||||
var mn MetricName
|
||||
var genTSID generationTSID
|
||||
var tsid TSID
|
||||
date := uint64(timestamp) / msecPerDay
|
||||
addSeries := func(kvs ...string) {
|
||||
mn.Reset()
|
||||
@@ -108,8 +110,8 @@ func BenchmarkHeadPostingForMatchers(b *testing.B) {
|
||||
mn.AddTag(kvs[i], kvs[i+1])
|
||||
}
|
||||
mn.sortTags()
|
||||
generateTSID(&genTSID.TSID, &mn)
|
||||
createAllIndexesForMetricName(idbCurr, &mn, &genTSID.TSID, date)
|
||||
generateTSID(&tsid, &mn)
|
||||
createAllIndexesForMetricName(db, &mn, &tsid, date)
|
||||
}
|
||||
for n := 0; n < 10; n++ {
|
||||
ns := strconv.Itoa(n)
|
||||
@@ -134,9 +136,9 @@ func BenchmarkHeadPostingForMatchers(b *testing.B) {
|
||||
// index instead of per-day index.
|
||||
tr := globalIndexTimeRange
|
||||
for i := 0; i < b.N; i++ {
|
||||
is := idbCurr.getIndexSearch(noDeadline)
|
||||
is := db.getIndexSearch(noDeadline)
|
||||
metricIDs, err := is.searchMetricIDs(nil, tfss, tr, 2e9)
|
||||
idbCurr.putIndexSearch(is)
|
||||
db.putIndexSearch(is)
|
||||
if err != nil {
|
||||
b.Fatalf("unexpected error in searchMetricIDs: %s", err)
|
||||
}
|
||||
@@ -253,7 +255,7 @@ func BenchmarkHeadPostingForMatchers(b *testing.B) {
|
||||
benchSearch(b, tfs, 88889)
|
||||
})
|
||||
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
s.tb.PutPartition(ptw)
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
}
|
||||
@@ -262,7 +264,8 @@ func BenchmarkIndexDBGetTSIDs(b *testing.B) {
|
||||
const path = "BenchmarkIndexDBGetTSIDs"
|
||||
timestamp := time.Date(2025, 3, 17, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
s := MustOpenStorage(path, OpenOptions{})
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
ptw := s.tb.MustGetPartition(timestamp)
|
||||
db := ptw.pt.idb
|
||||
|
||||
const recordsPerLoop = 1000
|
||||
const recordsCount = 1e5
|
||||
@@ -277,38 +280,38 @@ func BenchmarkIndexDBGetTSIDs(b *testing.B) {
|
||||
}
|
||||
mn.sortTags()
|
||||
|
||||
var genTSID generationTSID
|
||||
var tsid TSID
|
||||
date := uint64(timestamp) / msecPerDay
|
||||
|
||||
for i := 0; i < recordsCount; i++ {
|
||||
generateTSID(&genTSID.TSID, &mn)
|
||||
createAllIndexesForMetricName(idbCurr, &mn, &genTSID.TSID, date)
|
||||
generateTSID(&tsid, &mn)
|
||||
createAllIndexesForMetricName(db, &mn, &tsid, date)
|
||||
}
|
||||
idbCurr.s.DebugFlush()
|
||||
db.s.DebugFlush()
|
||||
|
||||
b.SetBytes(recordsPerLoop)
|
||||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
b.RunParallel(func(pb *testing.PB) {
|
||||
var genTSIDLocal generationTSID
|
||||
var tsidLocal TSID
|
||||
var metricNameLocal []byte
|
||||
var mnLocal MetricName
|
||||
mnLocal.CopyFrom(&mn)
|
||||
mnLocal.sortTags()
|
||||
for pb.Next() {
|
||||
is := idbCurr.getIndexSearch(noDeadline)
|
||||
is := db.getIndexSearch(noDeadline)
|
||||
for i := 0; i < recordsPerLoop; i++ {
|
||||
metricNameLocal = mnLocal.Marshal(metricNameLocal[:0])
|
||||
if !is.getTSIDByMetricName(&genTSIDLocal, metricNameLocal, date) {
|
||||
if !is.getTSIDByMetricName(&tsidLocal, metricNameLocal, date) {
|
||||
panic(fmt.Errorf("cannot obtain tsid for row %d", i))
|
||||
}
|
||||
}
|
||||
idbCurr.putIndexSearch(is)
|
||||
db.putIndexSearch(is)
|
||||
}
|
||||
})
|
||||
b.StopTimer()
|
||||
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
s.tb.PutPartition(ptw)
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
}
|
||||
|
||||
@@ -4,14 +4,14 @@ import (
|
||||
"sync"
|
||||
)
|
||||
|
||||
// metricNameSearch is used for searching a metricName by a metricID in curr
|
||||
// and prev indexDBs. If useSparseCache is false the name is first searched in
|
||||
// metricNameCache and also stored in that cache when found in one of the
|
||||
// indexDBs.
|
||||
// metricNameSearch is used for searching a metricName by a metricID in
|
||||
// partition and legacy indexDBs. If useSparseCache is false the name is first
|
||||
// searched in metricNameCache and also stored in that cache when found in one
|
||||
// of the indexDBs.
|
||||
//
|
||||
// Most index search methods invoked only once per API call. For example, one
|
||||
// request to /api/v1/series results in one invocation of
|
||||
// Storage.SearchMetricNames() method. However, searching is metricName by
|
||||
// Storage.SearchMetricNames() method. However, searching a metricName by
|
||||
// metricID is done multiple times per API call. For example, data search
|
||||
// performs the metricName search for each data block (see search.go).
|
||||
//
|
||||
@@ -28,8 +28,9 @@ import (
|
||||
// search method is invoked (due do mutex locks).
|
||||
type metricNameSearch struct {
|
||||
storage *Storage
|
||||
idbPrev *indexDB
|
||||
idbCurr *indexDB
|
||||
ptws []*partitionWrapper
|
||||
idbs []*indexDB
|
||||
legacyIDBs *legacyIndexDBs
|
||||
useSparseCache bool
|
||||
}
|
||||
|
||||
@@ -43,21 +44,40 @@ func (s *metricNameSearch) search(dst []byte, metricID uint64) ([]byte, bool) {
|
||||
}
|
||||
}
|
||||
|
||||
dst, found := s.idbCurr.searchMetricName(dst, metricID, s.useSparseCache)
|
||||
if found {
|
||||
if !s.useSparseCache {
|
||||
s.storage.putMetricNameToCache(metricID, dst)
|
||||
var found bool
|
||||
|
||||
// This will be just one idb most of the time since a typical time range
|
||||
// fits within a single month.
|
||||
for _, idb := range s.idbs {
|
||||
dst, found = idb.searchMetricName(dst, metricID, s.useSparseCache)
|
||||
if found {
|
||||
if !s.useSparseCache {
|
||||
s.storage.putMetricNameToCache(metricID, dst)
|
||||
}
|
||||
return dst, true
|
||||
}
|
||||
return dst, true
|
||||
}
|
||||
|
||||
// Fallback to previous indexDB.
|
||||
dst, found = s.idbPrev.searchMetricName(dst, metricID, s.useSparseCache)
|
||||
if found {
|
||||
if !s.useSparseCache {
|
||||
s.storage.putMetricNameToCache(metricID, dst)
|
||||
// Fallback to current legacy indexDB.
|
||||
if idb := s.legacyIDBs.getIDBCurr(); idb != nil {
|
||||
dst, found = idb.searchMetricName(dst, metricID, s.useSparseCache)
|
||||
if found {
|
||||
if !s.useSparseCache {
|
||||
s.storage.putMetricNameToCache(metricID, dst)
|
||||
}
|
||||
return dst, true
|
||||
}
|
||||
}
|
||||
|
||||
// Fallback to previous legacy indexDB.
|
||||
if idb := s.legacyIDBs.getIDBPrev(); idb != nil {
|
||||
dst, found = idb.searchMetricName(dst, metricID, s.useSparseCache)
|
||||
if found {
|
||||
if !s.useSparseCache {
|
||||
s.storage.putMetricNameToCache(metricID, dst)
|
||||
}
|
||||
return dst, true
|
||||
}
|
||||
return dst, true
|
||||
}
|
||||
|
||||
// Not deleting metricID if no corresponding metricName has been found
|
||||
@@ -74,19 +94,25 @@ var mnsPool = &sync.Pool{
|
||||
},
|
||||
}
|
||||
|
||||
func getMetricNameSearch(storage *Storage, useSparseCache bool) *metricNameSearch {
|
||||
func getMetricNameSearch(storage *Storage, tr TimeRange, useSparseCache bool) *metricNameSearch {
|
||||
s := mnsPool.Get().(*metricNameSearch)
|
||||
s.storage = storage
|
||||
s.idbPrev, s.idbCurr = storage.getPrevAndCurrIndexDBs()
|
||||
s.ptws = storage.tb.GetPartitions(tr)
|
||||
for _, ptw := range s.ptws {
|
||||
s.idbs = append(s.idbs, ptw.pt.idb)
|
||||
}
|
||||
s.legacyIDBs = storage.getLegacyIndexDBs()
|
||||
s.useSparseCache = useSparseCache
|
||||
return s
|
||||
}
|
||||
|
||||
func putMetricNameSearch(s *metricNameSearch) {
|
||||
s.storage.putPrevAndCurrIndexDBs(s.idbPrev, s.idbCurr)
|
||||
s.storage.tb.PutPartitions(s.ptws)
|
||||
s.storage.putLegacyIndexDBs(s.legacyIDBs)
|
||||
s.storage = nil
|
||||
s.idbPrev = nil
|
||||
s.idbCurr = nil
|
||||
s.ptws = nil
|
||||
s.idbs = nil
|
||||
s.legacyIDBs = nil
|
||||
s.useSparseCache = false
|
||||
mnsPool.Put(s)
|
||||
}
|
||||
|
||||
@@ -99,7 +99,11 @@ type partition struct {
|
||||
// the path to directory with bigParts.
|
||||
bigPartsPath string
|
||||
|
||||
// the path to directory with IndexDB parts.
|
||||
indexDBPartsPath string
|
||||
|
||||
// The parent storage.
|
||||
// TODO(@rtm0): Do not depend on Storage, pass only what is required.
|
||||
s *Storage
|
||||
|
||||
// Name is the name of the partition in the form YYYY_MM.
|
||||
@@ -127,6 +131,9 @@ type partition struct {
|
||||
// Contains file-based parts with big number of items, which are visible for search.
|
||||
bigParts []*partWrapper
|
||||
|
||||
// Contains the inverted index for the data stored in this partition.
|
||||
idb *indexDB
|
||||
|
||||
// stopCh is used for notifying all the background workers to stop.
|
||||
//
|
||||
// It must be closed under partsLock in order to prevent from calling wg.Add()
|
||||
@@ -193,14 +200,19 @@ func (pw *partWrapper) decRef() {
|
||||
}
|
||||
|
||||
// mustCreatePartition creates new partition for the given timestamp and the given paths to small and big partitions.
|
||||
func mustCreatePartition(timestamp int64, smallPartitionsPath, bigPartitionsPath string, s *Storage) *partition {
|
||||
func mustCreatePartition(timestamp int64, smallPartitionsPath, bigPartitionsPath, indexDBPath string, s *Storage) *partition {
|
||||
var tr TimeRange
|
||||
tr.fromPartitionTimestamp(timestamp)
|
||||
name := timestampToPartitionName(timestamp)
|
||||
|
||||
smallPartsPath := filepath.Join(filepath.Clean(smallPartitionsPath), name)
|
||||
bigPartsPath := filepath.Join(filepath.Clean(bigPartitionsPath), name)
|
||||
logger.Infof("creating a partition %q with smallPartsPath=%q, bigPartsPath=%q", name, smallPartsPath, bigPartsPath)
|
||||
indexDBPartsPath := filepath.Join(filepath.Clean(indexDBPath), name)
|
||||
logger.Infof("creating a partition %q with smallPartsPath=%q, bigPartsPath=%q, indexDBPartsPath=%q", name, smallPartsPath, bigPartsPath, indexDBPartsPath)
|
||||
|
||||
fs.MustMkdirFailIfExist(smallPartsPath)
|
||||
fs.MustMkdirFailIfExist(bigPartsPath)
|
||||
fs.MustMkdirFailIfExist(indexDBPartsPath)
|
||||
|
||||
// Create parts.json file. Since we are creating a new partition, there
|
||||
// will be no parts, i.e. the smallPartsPath and bigPartPath dirs will be
|
||||
@@ -208,13 +220,11 @@ func mustCreatePartition(timestamp int64, smallPartitionsPath, bigPartitionsPath
|
||||
// there will be panic.
|
||||
mustWritePartNames(nil, nil, smallPartsPath)
|
||||
|
||||
var tr TimeRange
|
||||
tr.fromPartitionTimestamp(timestamp)
|
||||
|
||||
pt := newPartition(name, smallPartsPath, bigPartsPath, tr, s)
|
||||
pt := newPartition(name, smallPartsPath, bigPartsPath, indexDBPartsPath, tr, s)
|
||||
|
||||
fs.MustSyncPathAndParentDir(smallPartsPath)
|
||||
fs.MustSyncPathAndParentDir(bigPartsPath)
|
||||
fs.MustSyncPathAndParentDir(indexDBPartsPath)
|
||||
|
||||
pt.startBackgroundWorkers()
|
||||
|
||||
@@ -238,21 +248,24 @@ func (pt *partition) startBackgroundWorkers() {
|
||||
//
|
||||
// The pt must be detached from table before calling pt.Drop.
|
||||
func (pt *partition) Drop() {
|
||||
logger.Infof("dropping partition %q at smallPartsPath=%q, bigPartsPath=%q", pt.name, pt.smallPartsPath, pt.bigPartsPath)
|
||||
logger.Infof("dropping partition %q at smallPartsPath=%q, bigPartsPath=%q, indexDBPartsPath=%q", pt.name, pt.smallPartsPath, pt.bigPartsPath, pt.indexDBPartsPath)
|
||||
|
||||
fs.MustRemoveDir(pt.smallPartsPath)
|
||||
fs.MustRemoveDir(pt.bigPartsPath)
|
||||
fs.MustRemoveDir(pt.indexDBPartsPath)
|
||||
logger.Infof("partition %q has been dropped", pt.name)
|
||||
}
|
||||
|
||||
// mustOpenPartition opens the existing partition from the given paths.
|
||||
func mustOpenPartition(smallPartsPath, bigPartsPath string, s *Storage) *partition {
|
||||
func mustOpenPartition(smallPartsPath, bigPartsPath, indexDBPartsPath string, s *Storage) *partition {
|
||||
smallPartsPath = filepath.Clean(smallPartsPath)
|
||||
bigPartsPath = filepath.Clean(bigPartsPath)
|
||||
indexDBPartsPath = filepath.Clean(indexDBPartsPath)
|
||||
|
||||
// Create paths to parts if they are missing.
|
||||
fs.MustMkdirIfNotExist(smallPartsPath)
|
||||
fs.MustMkdirIfNotExist(bigPartsPath)
|
||||
fs.MustMkdirIfNotExist(indexDBPartsPath)
|
||||
|
||||
name := filepath.Base(smallPartsPath)
|
||||
var tr TimeRange
|
||||
@@ -262,6 +275,9 @@ func mustOpenPartition(smallPartsPath, bigPartsPath string, s *Storage) *partiti
|
||||
if !strings.HasSuffix(bigPartsPath, name) {
|
||||
logger.Panicf("FATAL: partition name in bigPartsPath %q doesn't match smallPartsPath %q; want %q", bigPartsPath, smallPartsPath, name)
|
||||
}
|
||||
if !strings.HasSuffix(indexDBPartsPath, name) {
|
||||
logger.Panicf("FATAL: partition name in indexDBPartsPath %q doesn't match smallPartsPath %q; want %q", indexDBPartsPath, smallPartsPath, name)
|
||||
}
|
||||
|
||||
partsFile := filepath.Join(smallPartsPath, partsFilename)
|
||||
partNamesSmall, partNamesBig := mustReadPartNames(partsFile, smallPartsPath, bigPartsPath)
|
||||
@@ -276,7 +292,7 @@ func mustOpenPartition(smallPartsPath, bigPartsPath string, s *Storage) *partiti
|
||||
mustWritePartNames(smallParts, bigParts, smallPartsPath)
|
||||
}
|
||||
|
||||
pt := newPartition(name, smallPartsPath, bigPartsPath, tr, s)
|
||||
pt := newPartition(name, smallPartsPath, bigPartsPath, indexDBPartsPath, tr, s)
|
||||
pt.smallParts = smallParts
|
||||
pt.bigParts = bigParts
|
||||
|
||||
@@ -288,17 +304,23 @@ func mustOpenPartition(smallPartsPath, bigPartsPath string, s *Storage) *partiti
|
||||
return pt
|
||||
}
|
||||
|
||||
func newPartition(name, smallPartsPath, bigPartsPath string, tr TimeRange, s *Storage) *partition {
|
||||
func newPartition(name, smallPartsPath, bigPartsPath, indexDBPartsPath string, tr TimeRange, s *Storage) *partition {
|
||||
id := uint64(tr.MinTimestamp)
|
||||
idb := mustOpenIndexDB(id, tr, name, indexDBPartsPath, s, &s.isReadOnly, false)
|
||||
|
||||
p := &partition{
|
||||
name: name,
|
||||
smallPartsPath: smallPartsPath,
|
||||
bigPartsPath: bigPartsPath,
|
||||
tr: tr,
|
||||
s: s,
|
||||
stopCh: make(chan struct{}),
|
||||
name: name,
|
||||
smallPartsPath: smallPartsPath,
|
||||
bigPartsPath: bigPartsPath,
|
||||
indexDBPartsPath: indexDBPartsPath,
|
||||
tr: tr,
|
||||
s: s,
|
||||
idb: idb,
|
||||
stopCh: make(chan struct{}),
|
||||
}
|
||||
p.mergeIdx.Store(uint64(time.Now().UnixNano()))
|
||||
p.rawRows.init()
|
||||
|
||||
return p
|
||||
}
|
||||
|
||||
@@ -350,6 +372,8 @@ type partitionMetrics struct {
|
||||
|
||||
ScheduledDownsamplingPartitions uint64
|
||||
ScheduledDownsamplingPartitionsSize uint64
|
||||
|
||||
IndexDBMetrics IndexDBMetrics
|
||||
}
|
||||
|
||||
// TotalRowsCount returns total number of rows in tm.
|
||||
@@ -426,6 +450,8 @@ func (pt *partition) UpdateMetrics(m *partitionMetrics) {
|
||||
m.InmemoryRowsDeleted += pt.inmemoryRowsDeleted.Load()
|
||||
m.SmallRowsDeleted += pt.smallRowsDeleted.Load()
|
||||
m.BigRowsDeleted += pt.bigRowsDeleted.Load()
|
||||
|
||||
pt.idb.UpdateMetrics(&m.IndexDBMetrics)
|
||||
}
|
||||
|
||||
// AddRows adds the given rows to the partition pt.
|
||||
@@ -634,6 +660,7 @@ func (pt *partition) NotifyReadWriteMode() {
|
||||
pt.startInmemoryPartsMergers()
|
||||
pt.startSmallPartsMergers()
|
||||
pt.startBigPartsMergers()
|
||||
pt.idb.tb.NotifyReadWriteMode()
|
||||
}
|
||||
|
||||
func (pt *partition) inmemoryPartsMerger() {
|
||||
@@ -962,13 +989,18 @@ func (pt *partition) MustClose() {
|
||||
logger.Panicf("BUG: unexpected non-zero refCount: %d", refCount)
|
||||
}
|
||||
}
|
||||
|
||||
idb := pt.idb
|
||||
pt.idb = nil
|
||||
idb.MustClose()
|
||||
}
|
||||
|
||||
// DebugFlush flushes pending raw data rows of this partition so they
|
||||
// DebugFlush flushes pending raw index and data rows of this partition so they
|
||||
// become visible to search.
|
||||
//
|
||||
// This function is for debug purposes only.
|
||||
func (pt *partition) DebugFlush() {
|
||||
pt.idb.tb.DebugFlush()
|
||||
pt.flushPendingRows(true)
|
||||
}
|
||||
|
||||
@@ -1600,7 +1632,7 @@ func (pt *partition) mergePartsInternal(dstPartPath string, bsw *blockStreamWrit
|
||||
retentionDeadline := currentTimestamp - pt.s.retentionMsecs
|
||||
activeMerges.Add(1)
|
||||
_ = useSparseCache // unused in OSS version.
|
||||
dmis := pt.s.getDeletedMetricIDs()
|
||||
dmis := pt.idb.getDeletedMetricIDs()
|
||||
err := mergeBlockStreams(&ph, bsw, bsrs, stopCh, dmis, retentionDeadline, rowsMerged, rowsDeleted)
|
||||
activeMerges.Add(-1)
|
||||
mergesCount.Add(1)
|
||||
@@ -1994,8 +2026,8 @@ func mustOpenParts(partsFile, path string, partNames []string) []*partWrapper {
|
||||
// MustCreateSnapshotAt creates pt snapshot at the given smallPath and bigPath dirs.
|
||||
//
|
||||
// Snapshot is created using linux hard links, so it is usually created very quickly.
|
||||
func (pt *partition) MustCreateSnapshotAt(smallPath, bigPath string) {
|
||||
logger.Infof("creating partition snapshot of %q and %q...", pt.smallPartsPath, pt.bigPartsPath)
|
||||
func (pt *partition) MustCreateSnapshotAt(smallPath, bigPath, indexDBPath string) {
|
||||
logger.Infof("creating partition snapshot of %q, %q, and %q...", pt.smallPartsPath, pt.bigPartsPath, pt.indexDBPartsPath)
|
||||
startTime := time.Now()
|
||||
|
||||
// Flush inmemory data to disk.
|
||||
@@ -2025,8 +2057,10 @@ func (pt *partition) MustCreateSnapshotAt(smallPath, bigPath string) {
|
||||
fs.MustSyncPathAndParentDir(smallPath)
|
||||
fs.MustSyncPathAndParentDir(bigPath)
|
||||
|
||||
logger.Infof("created partition snapshot of %q and %q at %q and %q in %.3f seconds",
|
||||
pt.smallPartsPath, pt.bigPartsPath, smallPath, bigPath, time.Since(startTime).Seconds())
|
||||
pt.idb.tb.MustCreateSnapshotAt(indexDBPath)
|
||||
|
||||
logger.Infof("created partition snapshot of %q, %q, and %q at %q, %q, and %q in %.3f seconds",
|
||||
pt.smallPartsPath, pt.bigPartsPath, pt.indexDBPartsPath, smallPath, bigPath, indexDBPath, time.Since(startTime).Seconds())
|
||||
}
|
||||
|
||||
// mustCreateSnapshot creates a snapshot from srcDir to dstDir.
|
||||
|
||||
@@ -81,12 +81,29 @@ func (pts *partitionSearch) Init(pt *partition, tsids []TSID, tr TimeRange) {
|
||||
return
|
||||
}
|
||||
|
||||
filteredTSIDs := tsids
|
||||
deletedMetricsIDs := pt.idb.getDeletedMetricIDs()
|
||||
if deletedMetricsIDs.Len() > 0 {
|
||||
filteredTSIDs = make([]TSID, 0, len(tsids))
|
||||
for _, tsid := range tsids {
|
||||
if !deletedMetricsIDs.Has(tsid.MetricID) {
|
||||
filteredTSIDs = append(filteredTSIDs, tsid)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if len(filteredTSIDs) == 0 {
|
||||
// Fast path - zero tsids.
|
||||
pts.err = io.EOF
|
||||
return
|
||||
}
|
||||
|
||||
pts.pws = pt.GetParts(pts.pws[:0], true)
|
||||
|
||||
// Initialize psPool.
|
||||
pts.psPool = slicesutil.SetLength(pts.psPool, len(pts.pws))
|
||||
for i, pw := range pts.pws {
|
||||
pts.psPool[i].Init(pw.p, tsids, tr)
|
||||
pts.psPool[i].Init(pw.p, filteredTSIDs, tr)
|
||||
}
|
||||
|
||||
// Initialize the psHeap.
|
||||
|
||||
@@ -173,6 +173,7 @@ func testPartitionSearchEx(t *testing.T, ptt int64, tr TimeRange, partsCount, ma
|
||||
pt := testCreatePartition(t, ptt, strg)
|
||||
smallPartsPath := pt.smallPartsPath
|
||||
bigPartsPath := pt.bigPartsPath
|
||||
indexDBPartsPath := pt.indexDBPartsPath
|
||||
for _, rows := range rowss {
|
||||
pt.AddRows(rows)
|
||||
|
||||
@@ -183,7 +184,7 @@ func testPartitionSearchEx(t *testing.T, ptt int64, tr TimeRange, partsCount, ma
|
||||
pt.MustClose()
|
||||
|
||||
// Open the created partition and test search on it.
|
||||
pt = mustOpenPartition(smallPartsPath, bigPartsPath, strg)
|
||||
pt = mustOpenPartition(smallPartsPath, bigPartsPath, indexDBPartsPath, strg)
|
||||
testPartitionSearch(t, pt, tsids, tr, rbsExpected, rowsCountExpected)
|
||||
pt.MustClose()
|
||||
stopTestStorage(strg)
|
||||
|
||||
@@ -199,7 +199,8 @@ func testCreatePartition(t *testing.T, timestamp int64, s *Storage) *partition {
|
||||
t.Helper()
|
||||
small := filepath.Join(t.Name(), smallDirname)
|
||||
big := filepath.Join(t.Name(), bigDirname)
|
||||
return mustCreatePartition(timestamp, small, big, s)
|
||||
indexdb := filepath.Join(t.Name(), indexdbDirname)
|
||||
return mustCreatePartition(timestamp, small, big, indexdb, s)
|
||||
}
|
||||
|
||||
func TestMustCreatePartition(t *testing.T) {
|
||||
@@ -214,9 +215,13 @@ func TestMustCreatePartition(t *testing.T) {
|
||||
if fs.IsPathExist(bigPath) {
|
||||
t.Errorf("big partition directory must not exist: %s", bigPath)
|
||||
}
|
||||
indexDBPath := filepath.Join(t.Name(), "indexdb")
|
||||
if fs.IsPathExist(indexDBPath) {
|
||||
t.Errorf("indexdb parition directory must not exist: %s", indexDBPath)
|
||||
}
|
||||
s := &Storage{}
|
||||
|
||||
got := mustCreatePartition(ts, smallPath, bigPath, s)
|
||||
got := mustCreatePartition(ts, smallPath, bigPath, indexDBPath, s)
|
||||
defer got.MustClose()
|
||||
|
||||
wantSmallPartsPath := filepath.Join(smallPath, "2025_03")
|
||||
@@ -233,6 +238,14 @@ func TestMustCreatePartition(t *testing.T) {
|
||||
if !fs.IsPathExist(wantBigPartsPath) {
|
||||
t.Errorf("big parts directory hasn't been created: %s", wantBigPartsPath)
|
||||
}
|
||||
wantIndexDBPartsPath := filepath.Join(indexDBPath, "2025_03")
|
||||
if got.indexDBPartsPath != wantIndexDBPartsPath {
|
||||
t.Errorf("unexpected indexDB parts path: got %s, want %s", got.indexDBPartsPath, wantIndexDBPartsPath)
|
||||
}
|
||||
if !fs.IsPathExist(wantIndexDBPartsPath) {
|
||||
t.Errorf("indexDB parts directory hasn't been created: %s", wantIndexDBPartsPath)
|
||||
}
|
||||
|
||||
wantStorage := s
|
||||
if got.s != wantStorage {
|
||||
t.Errorf("unexpected storage: got %v, want %v", got.s, wantStorage)
|
||||
@@ -248,7 +261,6 @@ func TestMustCreatePartition(t *testing.T) {
|
||||
if got.tr != wantTR {
|
||||
t.Errorf("unexpected time range: got %v, want %v", &got.tr, &wantTR)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func TestMustOpenPartition(t *testing.T) {
|
||||
@@ -256,10 +268,11 @@ func TestMustOpenPartition(t *testing.T) {
|
||||
|
||||
smallPartsPath := filepath.Join(t.Name(), "small", "2025_03")
|
||||
bigPartsPath := filepath.Join(t.Name(), "big", "2025_03")
|
||||
indexDBPartsPath := filepath.Join(t.Name(), "indexdb", "2025_03")
|
||||
|
||||
s := &Storage{}
|
||||
|
||||
got := mustOpenPartition(smallPartsPath, bigPartsPath, s)
|
||||
got := mustOpenPartition(smallPartsPath, bigPartsPath, indexDBPartsPath, s)
|
||||
defer got.MustClose()
|
||||
|
||||
if got.smallPartsPath != smallPartsPath {
|
||||
@@ -274,6 +287,12 @@ func TestMustOpenPartition(t *testing.T) {
|
||||
if !fs.IsPathExist(bigPartsPath) {
|
||||
t.Errorf("big parts directory hasn't been created: %s", bigPartsPath)
|
||||
}
|
||||
if got.indexDBPartsPath != indexDBPartsPath {
|
||||
t.Errorf("unexpected indexDB parts path: got %s, want %s", got.indexDBPartsPath, indexDBPartsPath)
|
||||
}
|
||||
if !fs.IsPathExist(indexDBPartsPath) {
|
||||
t.Errorf("indexDB parts directory hasn't been created: %s", indexDBPartsPath)
|
||||
}
|
||||
if got.s != s {
|
||||
t.Errorf("unexpected storage: got %v, want %v", got.s, s)
|
||||
}
|
||||
@@ -296,15 +315,16 @@ func TestMustOpenPartition_invalidPartitionName(t *testing.T) {
|
||||
|
||||
smallPartsPath := filepath.Join(t.Name(), "small", "2025_03_invalid")
|
||||
bigPartsPath := filepath.Join(t.Name(), "big", "2025_03_invalid")
|
||||
indexDBPartsPath := filepath.Join(t.Name(), "indexdb", "2025_03_invalid")
|
||||
|
||||
defer func() {
|
||||
if err := recover(); err == nil {
|
||||
t.Fatalf("expected panic on invalid partition name in smallPartsPath but it did not happen: %v", smallPartsPath)
|
||||
t.Fatalf("expected panic on invalid partition name in smallPartsPath but it did not happen: %q", smallPartsPath)
|
||||
}
|
||||
}()
|
||||
|
||||
s := &Storage{}
|
||||
_ = mustOpenPartition(smallPartsPath, bigPartsPath, s)
|
||||
_ = mustOpenPartition(smallPartsPath, bigPartsPath, indexDBPartsPath, s)
|
||||
|
||||
}
|
||||
|
||||
@@ -313,13 +333,14 @@ func TestMustOpenPartition_smallAndBigPartsPathsAreNotTheSame(t *testing.T) {
|
||||
|
||||
smallPartsPath := filepath.Join(t.Name(), "small", "2025_03")
|
||||
bigPartsPath := filepath.Join(t.Name(), "big", "2025_04")
|
||||
indexDBPartsPath := filepath.Join(t.Name(), "indexDB", "2025_04")
|
||||
|
||||
defer func() {
|
||||
if err := recover(); err == nil {
|
||||
t.Fatalf("expected panic on different partition name in smallPartsPath=%v and bigPartsPath=%v but it did not happen", smallPartsPath, bigPartsPath)
|
||||
t.Fatalf("expected panic on different partition name in smallPartsPath=%q and bigPartsPath=%q indexDBPartsPath=%q but it did not happen", smallPartsPath, bigPartsPath, indexDBPartsPath)
|
||||
}
|
||||
}()
|
||||
|
||||
s := &Storage{}
|
||||
_ = mustOpenPartition(smallPartsPath, bigPartsPath, s)
|
||||
|
||||
_ = mustOpenPartition(smallPartsPath, bigPartsPath, indexDBPartsPath, s)
|
||||
}
|
||||
|
||||
@@ -151,18 +151,16 @@ func (s *Search) reset() {
|
||||
//
|
||||
// Init returns the upper bound on the number of found time series.
|
||||
func (s *Search) Init(qt *querytracer.Tracer, storage *Storage, tfss []*TagFilters, tr TimeRange, maxMetrics int, deadline uint64) int {
|
||||
qt = qt.NewChild("init series search: filters=%s, timeRange=%s", tfss, &tr)
|
||||
qt = qt.NewChild("init series search: filters=%s, timeRange=%s, maxMetrics=%d", tfss, &tr, maxMetrics)
|
||||
defer qt.Done()
|
||||
|
||||
dataTR := tr
|
||||
|
||||
if s.needClosing {
|
||||
logger.Panicf("BUG: missing MustClose call before the next call to Init")
|
||||
}
|
||||
retentionDeadline := int64(fasttime.UnixTimestamp()*1e3) - storage.retentionMsecs
|
||||
|
||||
s.reset()
|
||||
s.mns = getMetricNameSearch(storage, false)
|
||||
s.mns = getMetricNameSearch(storage, tr, false)
|
||||
s.retentionDeadline = retentionDeadline
|
||||
s.metricsTracker = storage.metricsTracker
|
||||
s.tr = tr
|
||||
@@ -175,7 +173,7 @@ func (s *Search) Init(qt *querytracer.Tracer, storage *Storage, tfss []*TagFilte
|
||||
// It is ok to call Init on non-nil err.
|
||||
// Init must be called before returning because it will fail
|
||||
// on Search.MustClose otherwise.
|
||||
s.ts.Init(storage.tb, tsids, dataTR)
|
||||
s.ts.Init(storage.tb, tsids, tr)
|
||||
qt.Printf("search for parts with data for %d series", len(tsids))
|
||||
if err != nil {
|
||||
s.err = err
|
||||
|
||||
@@ -4,6 +4,7 @@ package storage
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"slices"
|
||||
"testing"
|
||||
"testing/synctest"
|
||||
"time"
|
||||
@@ -12,22 +13,37 @@ import (
|
||||
func TestSearch_metricNamesIndifferentIndexDBs(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
synctest.Test(t, func(t *testing.T) {
|
||||
const numMetrics = 10
|
||||
synctest.Run(func() {
|
||||
const numSeries = 10
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Now().UnixMilli(),
|
||||
MaxTimestamp: time.Now().Add(23 * time.Hour).UnixMilli(),
|
||||
}
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
mrs := testGenerateMetricRowsWithPrefix(rng, numMetrics, "metric", tr)
|
||||
mrsPrev := testGenerateMetricRowsWithPrefix(rng, numSeries, "legacy_prev", tr)
|
||||
mrsCurr := testGenerateMetricRowsWithPrefix(rng, numSeries, "legacy_curr", tr)
|
||||
mrsPt := testGenerateMetricRowsWithPrefix(rng, numSeries, "pt", tr)
|
||||
mrs := slices.Concat(mrsPrev, mrsCurr, mrsPt)
|
||||
s := MustOpenStorage(t.Name(), OpenOptions{})
|
||||
defer s.MustClose()
|
||||
s.AddRows(mrs[:numMetrics/2], defaultPrecisionBits)
|
||||
// Rotate the indexDB to ensure that the index for the entire dataset is
|
||||
// split across prev and curr indexDBs.
|
||||
s.mustRotateIndexDB(time.Now())
|
||||
s.AddRows(mrs[numMetrics/2:], defaultPrecisionBits)
|
||||
s.AddRows(mrsPrev, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
// Advance the time a bit before converting to legacy so that the
|
||||
// storage could use a different timestamp for a legacy idb.
|
||||
time.Sleep(time.Second)
|
||||
s = mustConvertToLegacy(s)
|
||||
s.AddRows(mrsCurr, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
// Advance the time a bit before converting to legacy so that the
|
||||
// storage could use a different timestamp for a legacy idb.
|
||||
time.Sleep(time.Second)
|
||||
// Convert second time to have two legacy idbs (prev and curr)
|
||||
s = mustConvertToLegacy(s)
|
||||
// Advance the time a bit before converting to legacy so that the
|
||||
// storage could use a different timestamp for data and pt index parts.
|
||||
time.Sleep(time.Second)
|
||||
s.AddRows(mrsPt, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
defer s.MustClose()
|
||||
|
||||
tfs := NewTagFilters()
|
||||
if err := tfs.Add(nil, []byte(".*"), false, true); err != nil {
|
||||
@@ -45,7 +61,7 @@ func TestSearch_metricNamesIndifferentIndexDBs(t *testing.T) {
|
||||
|
||||
var m Metrics
|
||||
s.UpdateMetrics(&m)
|
||||
if got, want := m.IndexDBMetrics.MissingTSIDsForMetricID, uint64(0); got != want {
|
||||
if got, want := m.TableMetrics.IndexDBMetrics.MissingTSIDsForMetricID, uint64(0); got != want {
|
||||
t.Fatalf("unexpected MissingTSIDsForMetricID count: got %d, want %d", got, want)
|
||||
}
|
||||
|
||||
@@ -63,7 +79,7 @@ func TestSearch_metricNamesIndifferentIndexDBs(t *testing.T) {
|
||||
}
|
||||
|
||||
s.UpdateMetrics(&m)
|
||||
if got, want := m.IndexDBMetrics.MissingTSIDsForMetricID, uint64(0); got != want {
|
||||
if got, want := m.TableMetrics.IndexDBMetrics.MissingTSIDsForMetricID, uint64(0); got != want {
|
||||
t.Fatalf("unexpected MissingTSIDsForMetricID count: got %d, want %d", got, want)
|
||||
}
|
||||
|
||||
@@ -77,7 +93,7 @@ func TestSearch_metricNamesIndifferentIndexDBs(t *testing.T) {
|
||||
}
|
||||
|
||||
s.UpdateMetrics(&m)
|
||||
if got, want := m.IndexDBMetrics.MissingTSIDsForMetricID, uint64(0); got != want {
|
||||
if got, want := m.TableMetrics.IndexDBMetrics.MissingTSIDsForMetricID, uint64(0); got != want {
|
||||
t.Fatalf("unexpected MissingTSIDsForMetricID count: got %d, want %d", got, want)
|
||||
}
|
||||
})
|
||||
|
||||
@@ -166,10 +166,7 @@ func TestSearch_VariousTimeRanges(t *testing.T) {
|
||||
|
||||
s := MustOpenStorage(t.Name(), OpenOptions{})
|
||||
defer s.MustClose()
|
||||
s.AddRows(mrs[:numMetrics/2], defaultPrecisionBits)
|
||||
// Rotate the indexDB to ensure that the search operation covers both current and prev indexDBs.
|
||||
s.mustRotateIndexDB(time.Now())
|
||||
s.AddRows(mrs[numMetrics/2:], defaultPrecisionBits)
|
||||
s.AddRows(mrs, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
|
||||
if err := testSearchInternal(s, tr, mrs); err != nil {
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
229
lib/storage/storage_legacy.go
Normal file
229
lib/storage/storage_legacy.go
Normal file
@@ -0,0 +1,229 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"path/filepath"
|
||||
"time"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/querytracer"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/uint64set"
|
||||
)
|
||||
|
||||
type legacyIndexDBs struct {
|
||||
idbPrev *legacyIndexDB
|
||||
idbCurr *legacyIndexDB
|
||||
}
|
||||
|
||||
func (dbs *legacyIndexDBs) incRef() {
|
||||
if dbs == nil {
|
||||
// No legacy indexDBs, nothing to increment reference count.
|
||||
return
|
||||
}
|
||||
|
||||
if dbs.idbPrev != nil {
|
||||
dbs.idbPrev.incRef()
|
||||
}
|
||||
if dbs.idbCurr != nil {
|
||||
dbs.idbCurr.incRef()
|
||||
}
|
||||
}
|
||||
|
||||
func (dbs *legacyIndexDBs) decRef() {
|
||||
if dbs == nil {
|
||||
// No legacy indexDBs, nothing to decrement reference count.
|
||||
return
|
||||
}
|
||||
|
||||
if dbs.idbPrev != nil {
|
||||
dbs.idbPrev.decRef()
|
||||
}
|
||||
if dbs.idbCurr != nil {
|
||||
dbs.idbCurr.decRef()
|
||||
}
|
||||
}
|
||||
|
||||
func (dbs *legacyIndexDBs) appendTo(dst []*indexDB) []*indexDB {
|
||||
if dbs == nil {
|
||||
// No legacy indexDBs, nothing to append.
|
||||
return dst
|
||||
}
|
||||
|
||||
if dbs.idbPrev != nil {
|
||||
dst = append(dst, dbs.idbPrev.idb)
|
||||
}
|
||||
if dbs.idbCurr != nil {
|
||||
dst = append(dst, dbs.idbCurr.idb)
|
||||
}
|
||||
return dst
|
||||
}
|
||||
|
||||
func (dbs *legacyIndexDBs) getIDBPrev() *indexDB {
|
||||
if dbs == nil || dbs.idbPrev == nil {
|
||||
return nil
|
||||
}
|
||||
return dbs.idbPrev.idb
|
||||
}
|
||||
|
||||
func (dbs *legacyIndexDBs) getIDBCurr() *indexDB {
|
||||
if dbs == nil || dbs.idbCurr == nil {
|
||||
return nil
|
||||
}
|
||||
return dbs.idbCurr.idb
|
||||
}
|
||||
|
||||
func (s *Storage) hasLegacyIndexDBs() bool {
|
||||
return s.legacyIndexDBs.Load() != nil
|
||||
}
|
||||
|
||||
func (s *Storage) getLegacyIndexDBs() *legacyIndexDBs {
|
||||
legacyIDBs := s.legacyIndexDBs.Load()
|
||||
legacyIDBs.incRef()
|
||||
return legacyIDBs
|
||||
}
|
||||
|
||||
func (s *Storage) putLegacyIndexDBs(legacyIDBs *legacyIndexDBs) {
|
||||
legacyIDBs.decRef()
|
||||
}
|
||||
|
||||
func (s *Storage) legacyCreateSnapshot(snapshotName, srcDir, dstDir string) {
|
||||
legacyIDBs := s.getLegacyIndexDBs()
|
||||
defer s.putLegacyIndexDBs(legacyIDBs)
|
||||
|
||||
if legacyIDBs == nil {
|
||||
return
|
||||
}
|
||||
|
||||
idbSnapshot := filepath.Join(srcDir, indexdbDirname, snapshotsDirname, snapshotName)
|
||||
if idbPrev := legacyIDBs.getIDBPrev(); idbPrev != nil {
|
||||
prevSnapshot := filepath.Join(idbSnapshot, idbPrev.name)
|
||||
idbPrev.tb.LegacyMustCreateSnapshotAt(prevSnapshot)
|
||||
}
|
||||
if idbCurr := legacyIDBs.getIDBCurr(); idbCurr != nil {
|
||||
currSnapshot := filepath.Join(idbSnapshot, idbCurr.name)
|
||||
idbCurr.tb.LegacyMustCreateSnapshotAt(currSnapshot)
|
||||
}
|
||||
dstIdbDir := filepath.Join(dstDir, indexdbDirname)
|
||||
fs.MustSymlinkRelative(idbSnapshot, dstIdbDir)
|
||||
}
|
||||
|
||||
func (s *Storage) legacyMustRotateIndexDB(currentTime time.Time) {
|
||||
legacyIDBs := s.getLegacyIndexDBs()
|
||||
defer s.putLegacyIndexDBs(legacyIDBs)
|
||||
|
||||
if legacyIDBs == nil {
|
||||
// No legacy indexDBs, nothing to rotate.
|
||||
return
|
||||
}
|
||||
|
||||
legacyIDBs.idbPrev.scheduleToDrop()
|
||||
legacyIDBs.idbPrev.decRef()
|
||||
|
||||
var rotatedLegacyIDBs *legacyIndexDBs
|
||||
|
||||
if legacyIDBs.idbCurr != nil {
|
||||
rotatedLegacyIDBs = &legacyIndexDBs{
|
||||
idbPrev: legacyIDBs.idbCurr,
|
||||
}
|
||||
}
|
||||
s.legacyIndexDBs.Store(rotatedLegacyIDBs)
|
||||
|
||||
// Update nextRotationTimestamp
|
||||
nextRotationTimestamp := currentTime.Unix() + s.retentionMsecs/1000
|
||||
s.legacyNextRotationTimestamp.Store(nextRotationTimestamp)
|
||||
}
|
||||
|
||||
func (s *Storage) legacyDeleteSeries(qt *querytracer.Tracer, tfss []*TagFilters, maxMetrics int) (*uint64set.Set, error) {
|
||||
legacyIDBs := s.getLegacyIndexDBs()
|
||||
defer s.putLegacyIndexDBs(legacyIDBs)
|
||||
|
||||
if legacyIDBs == nil {
|
||||
// No legacy indexDBs, nothing to delete.
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
all := &uint64set.Set{}
|
||||
|
||||
if idbPrev := legacyIDBs.getIDBPrev(); idbPrev != nil {
|
||||
qt.Printf("start deleting from previous legacy indexDB")
|
||||
dmis, err := idbPrev.DeleteSeries(qt, tfss, maxMetrics)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
qt.Printf("deleted %d metricIDs from previous legacy indexDB", dmis.Len())
|
||||
all.UnionMayOwn(dmis)
|
||||
}
|
||||
|
||||
if idbCurr := legacyIDBs.getIDBCurr(); idbCurr != nil {
|
||||
qt.Printf("start deleting from current legacy indexDB")
|
||||
dmis, err := idbCurr.DeleteSeries(qt, tfss, maxMetrics)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
qt.Printf("deleted %d metricIDs from current legacy indexDB", dmis.Len())
|
||||
all.UnionMayOwn(dmis)
|
||||
}
|
||||
|
||||
return all, nil
|
||||
}
|
||||
|
||||
func (s *Storage) legacyDebugFlush() {
|
||||
legacyIDBs := s.getLegacyIndexDBs()
|
||||
defer s.putLegacyIndexDBs(legacyIDBs)
|
||||
|
||||
if legacyIDBs == nil {
|
||||
return
|
||||
}
|
||||
|
||||
if idbPrev := legacyIDBs.getIDBPrev(); idbPrev != nil {
|
||||
idbPrev.tb.DebugFlush()
|
||||
}
|
||||
if idbCurr := legacyIDBs.getIDBCurr(); idbCurr != nil {
|
||||
idbCurr.tb.DebugFlush()
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Storage) legacyNotifyReadWriteMode() {
|
||||
legacyIDBs := s.getLegacyIndexDBs()
|
||||
defer s.putLegacyIndexDBs(legacyIDBs)
|
||||
|
||||
if legacyIDBs == nil {
|
||||
return
|
||||
}
|
||||
|
||||
if idbPrev := legacyIDBs.getIDBPrev(); idbPrev != nil {
|
||||
idbPrev.tb.NotifyReadWriteMode()
|
||||
}
|
||||
if idbCurr := legacyIDBs.getIDBCurr(); idbCurr != nil {
|
||||
idbCurr.tb.NotifyReadWriteMode()
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Storage) legacyUpdateMetrics(m *Metrics) {
|
||||
legacyIDBs := s.getLegacyIndexDBs()
|
||||
defer s.putLegacyIndexDBs(legacyIDBs)
|
||||
|
||||
if legacyIDBs == nil {
|
||||
return
|
||||
}
|
||||
|
||||
if legacyIDBs.idbPrev != nil {
|
||||
legacyIDBs.idbPrev.UpdateMetrics(&m.TableMetrics.IndexDBMetrics)
|
||||
}
|
||||
if legacyIDBs.idbCurr != nil {
|
||||
legacyIDBs.idbCurr.UpdateMetrics(&m.TableMetrics.IndexDBMetrics)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Storage) legacyMustCloseIndexDBs() {
|
||||
legacyIDBs := s.legacyIndexDBs.Load()
|
||||
if legacyIDBs == nil {
|
||||
return
|
||||
}
|
||||
|
||||
if legacyIDBs.idbPrev != nil {
|
||||
legacyIDBs.idbPrev.MustClose()
|
||||
}
|
||||
if legacyIDBs.idbCurr != nil {
|
||||
legacyIDBs.idbCurr.MustClose()
|
||||
}
|
||||
}
|
||||
815
lib/storage/storage_legacy_test.go
Normal file
815
lib/storage/storage_legacy_test.go
Normal file
@@ -0,0 +1,815 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"math/rand"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"slices"
|
||||
"sort"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
)
|
||||
|
||||
func TestLegacyStorage_SearchMetricNames(t *testing.T) {
|
||||
genData := func(numMetrics int, prefix string, tr TimeRange) ([]MetricRow, []string) {
|
||||
mrs := make([]MetricRow, numMetrics)
|
||||
want := make([]string, numMetrics)
|
||||
for i := range numMetrics {
|
||||
name := fmt.Sprintf("%s_metric_%03d", prefix, i)
|
||||
mn := MetricName{
|
||||
MetricGroup: []byte(name),
|
||||
}
|
||||
mrs[i].MetricNameRaw = mn.marshalRaw(nil)
|
||||
mrs[i].Timestamp = rand.Int63n(tr.MaxTimestamp-tr.MinTimestamp) + tr.MinTimestamp
|
||||
mrs[i].Value = float64(i)
|
||||
want[i] = name
|
||||
}
|
||||
return mrs, want
|
||||
}
|
||||
const numMetrics = 1000
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Date(2023, 6, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 5, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
legacyData, wantLegacy := genData(numMetrics, "legacy", tr)
|
||||
newData, wantNew := genData(numMetrics, "new", tr)
|
||||
wantNew = append(wantNew, wantLegacy...)
|
||||
slices.Sort(wantNew)
|
||||
|
||||
assertSearchResults := func(s *Storage, want []string) {
|
||||
t.Helper()
|
||||
tfsAll := NewTagFilters()
|
||||
if err := tfsAll.Add([]byte("__name__"), []byte(".*"), false, true); err != nil {
|
||||
t.Fatalf("unexpected error in TagFilters.Add: %v", err)
|
||||
}
|
||||
tfssAll := []*TagFilters{tfsAll}
|
||||
got, err := s.SearchMetricNames(nil, tfssAll, tr, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("SearchMetricNames() failed unexpectedly: %v", err)
|
||||
}
|
||||
for i, name := range got {
|
||||
var mn MetricName
|
||||
if err := mn.UnmarshalString(name); err != nil {
|
||||
t.Fatalf("Could not unmarshal metric name %q: %v", name, err)
|
||||
}
|
||||
got[i] = string(mn.MetricGroup)
|
||||
}
|
||||
slices.Sort(got)
|
||||
if diff := cmp.Diff(want, got); diff != "" {
|
||||
t.Fatalf("unexpected metric names (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
|
||||
assertLegacyData := func(s *Storage) {
|
||||
assertSearchResults(s, wantLegacy)
|
||||
}
|
||||
assertNewData := func(s *Storage) {
|
||||
assertSearchResults(s, wantNew)
|
||||
}
|
||||
testSearchOpWithLegacyIndexDBs(t, legacyData, newData, assertLegacyData, assertNewData)
|
||||
}
|
||||
|
||||
func TestLegacyStorage_SearchLabelNames(t *testing.T) {
|
||||
genData := func(numMetrics int, prefix string, tr TimeRange) ([]MetricRow, []string) {
|
||||
mrs := make([]MetricRow, numMetrics)
|
||||
want := make([]string, numMetrics)
|
||||
for i := range numMetrics {
|
||||
name := fmt.Sprintf("%s_label_%03d", prefix, i)
|
||||
mn := MetricName{
|
||||
MetricGroup: []byte("metric"),
|
||||
Tags: []Tag{
|
||||
{[]byte(name), []byte("value")},
|
||||
},
|
||||
}
|
||||
mrs[i].MetricNameRaw = mn.marshalRaw(nil)
|
||||
mrs[i].Timestamp = rand.Int63n(tr.MaxTimestamp-tr.MinTimestamp) + tr.MinTimestamp
|
||||
mrs[i].Value = float64(i)
|
||||
want[i] = name
|
||||
}
|
||||
return mrs, want
|
||||
}
|
||||
const numMetrics = 1000
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Date(2023, 6, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 5, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
legacyData, wantLegacy := genData(numMetrics, "legacy", tr)
|
||||
newData, wantNew := genData(numMetrics, "new", tr)
|
||||
wantNew = append(wantNew, wantLegacy...)
|
||||
|
||||
assertSearchResults := func(s *Storage, want []string) {
|
||||
t.Helper()
|
||||
got, err := s.SearchLabelNames(nil, nil, tr, 1e9, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("SearchLabelNames() failed unexpectedly: %v", err)
|
||||
}
|
||||
slices.Sort(got)
|
||||
if diff := cmp.Diff(want, got); diff != "" {
|
||||
t.Fatalf("unexpected label names (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
|
||||
assertLegacyData := func(s *Storage) {
|
||||
want := append(wantLegacy, "__name__")
|
||||
slices.Sort(want)
|
||||
assertSearchResults(s, want)
|
||||
}
|
||||
assertNewData := func(s *Storage) {
|
||||
want := append(wantNew, "__name__")
|
||||
slices.Sort(want)
|
||||
assertSearchResults(s, want)
|
||||
}
|
||||
testSearchOpWithLegacyIndexDBs(t, legacyData, newData, assertLegacyData, assertNewData)
|
||||
}
|
||||
|
||||
func TestLegacyStorage_SearchLabelValues(t *testing.T) {
|
||||
genData := func(numMetrics int, prefix string, tr TimeRange) ([]MetricRow, []string) {
|
||||
mrs := make([]MetricRow, numMetrics)
|
||||
want := make([]string, numMetrics)
|
||||
for i := range numMetrics {
|
||||
value := fmt.Sprintf("%s_value_%03d", prefix, i)
|
||||
mn := MetricName{
|
||||
MetricGroup: []byte("metric"),
|
||||
Tags: []Tag{
|
||||
{[]byte("label"), []byte(value)},
|
||||
},
|
||||
}
|
||||
mrs[i].MetricNameRaw = mn.marshalRaw(nil)
|
||||
mrs[i].Timestamp = rand.Int63n(tr.MaxTimestamp-tr.MinTimestamp) + tr.MinTimestamp
|
||||
mrs[i].Value = float64(i)
|
||||
want[i] = value
|
||||
}
|
||||
return mrs, want
|
||||
}
|
||||
const numMetrics = 1000
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Date(2023, 6, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 5, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
legacyData, wantLegacy := genData(numMetrics, "legacy", tr)
|
||||
newData, wantNew := genData(numMetrics, "new", tr)
|
||||
wantNew = append(wantNew, wantLegacy...)
|
||||
slices.Sort(wantNew)
|
||||
|
||||
assertSearchResults := func(s *Storage, want []string) {
|
||||
t.Helper()
|
||||
got, err := s.SearchLabelValues(nil, "label", nil, tr, 1e9, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("SearchLabelValues() failed unexpectedly: %v", err)
|
||||
}
|
||||
slices.Sort(got)
|
||||
if diff := cmp.Diff(want, got); diff != "" {
|
||||
t.Fatalf("unexpected label values (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
|
||||
assertLegacyData := func(s *Storage) {
|
||||
t.Helper()
|
||||
assertSearchResults(s, wantLegacy)
|
||||
}
|
||||
assertNewData := func(s *Storage) {
|
||||
t.Helper()
|
||||
assertSearchResults(s, wantNew)
|
||||
}
|
||||
testSearchOpWithLegacyIndexDBs(t, legacyData, newData, assertLegacyData, assertNewData)
|
||||
}
|
||||
|
||||
func TestLegacyStorage_SearchTagValueSuffixes(t *testing.T) {
|
||||
genData := func(numMetrics int, prefix string, tr TimeRange) ([]MetricRow, []string) {
|
||||
mrs := make([]MetricRow, numMetrics)
|
||||
want := make([]string, numMetrics)
|
||||
for i := range numMetrics {
|
||||
name := fmt.Sprintf("%s_metric_%03d", prefix, i)
|
||||
mn := MetricName{
|
||||
MetricGroup: []byte("prefix." + name),
|
||||
}
|
||||
mrs[i].MetricNameRaw = mn.marshalRaw(nil)
|
||||
mrs[i].Timestamp = rand.Int63n(tr.MaxTimestamp-tr.MinTimestamp) + tr.MinTimestamp
|
||||
mrs[i].Value = float64(i)
|
||||
want[i] = name
|
||||
}
|
||||
return mrs, want
|
||||
}
|
||||
const numMetrics = 1000
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Date(2023, 6, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 5, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
legacyData, wantLegacy := genData(numMetrics, "legacy", tr)
|
||||
newData, wantNew := genData(numMetrics, "new", tr)
|
||||
wantNew = append(wantNew, wantLegacy...)
|
||||
slices.Sort(wantNew)
|
||||
|
||||
assertSearchResults := func(s *Storage, want []string) {
|
||||
t.Helper()
|
||||
got, err := s.SearchTagValueSuffixes(nil, tr, "", "prefix.", '.', 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("SearchTagValueSuffixes() failed unexpectedly: %v", err)
|
||||
}
|
||||
slices.Sort(got)
|
||||
|
||||
if diff := cmp.Diff(want, got); diff != "" {
|
||||
t.Errorf("unexpected tag value suffixes (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
|
||||
assertLegacyData := func(s *Storage) {
|
||||
t.Helper()
|
||||
assertSearchResults(s, wantLegacy)
|
||||
}
|
||||
assertNewData := func(s *Storage) {
|
||||
t.Helper()
|
||||
assertSearchResults(s, wantNew)
|
||||
}
|
||||
testSearchOpWithLegacyIndexDBs(t, legacyData, newData, assertLegacyData, assertNewData)
|
||||
}
|
||||
|
||||
func TestLegacyStorage_SearchGraphitePaths(t *testing.T) {
|
||||
genData := func(numMetrics int, prefix string, tr TimeRange) ([]MetricRow, []string) {
|
||||
mrs := make([]MetricRow, numMetrics)
|
||||
want := make([]string, numMetrics)
|
||||
for i := range numMetrics {
|
||||
name := fmt.Sprintf("prefix.%s_metric_%03d", prefix, i)
|
||||
mn := MetricName{
|
||||
MetricGroup: []byte(name),
|
||||
}
|
||||
mrs[i].MetricNameRaw = mn.marshalRaw(nil)
|
||||
mrs[i].Timestamp = rand.Int63n(tr.MaxTimestamp-tr.MinTimestamp) + tr.MinTimestamp
|
||||
mrs[i].Value = float64(i)
|
||||
want[i] = name
|
||||
}
|
||||
return mrs, want
|
||||
}
|
||||
const numMetrics = 1000
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Date(2023, 6, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 5, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
legacyData, wantLegacy := genData(numMetrics, "legacy", tr)
|
||||
newData, wantNew := genData(numMetrics, "new", tr)
|
||||
wantNew = append(wantNew, wantLegacy...)
|
||||
slices.Sort(wantNew)
|
||||
|
||||
assertSearchResults := func(s *Storage, want []string) {
|
||||
t.Helper()
|
||||
got, err := s.SearchGraphitePaths(nil, tr, []byte("*.*"), 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("SearchTagGraphitePaths() failed unexpectedly: %v", err)
|
||||
}
|
||||
slices.Sort(got)
|
||||
|
||||
if diff := cmp.Diff(want, got); diff != "" {
|
||||
t.Errorf("unexpected graphite paths (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
|
||||
assertLegacyData := func(s *Storage) {
|
||||
t.Helper()
|
||||
assertSearchResults(s, wantLegacy)
|
||||
}
|
||||
assertNewData := func(s *Storage) {
|
||||
t.Helper()
|
||||
assertSearchResults(s, wantNew)
|
||||
}
|
||||
testSearchOpWithLegacyIndexDBs(t, legacyData, newData, assertLegacyData, assertNewData)
|
||||
}
|
||||
|
||||
func TestLegacyStorage_Search(t *testing.T) {
|
||||
genData := func(numMetrics int, prefix string, tr TimeRange) []MetricRow {
|
||||
mrs := make([]MetricRow, numMetrics)
|
||||
for i := range numMetrics {
|
||||
name := fmt.Sprintf("%s_metric_%03d", prefix, i)
|
||||
mn := MetricName{
|
||||
MetricGroup: []byte(name),
|
||||
}
|
||||
mrs[i].MetricNameRaw = mn.marshalRaw(nil)
|
||||
mrs[i].Timestamp = rand.Int63n(tr.MaxTimestamp-tr.MinTimestamp) + tr.MinTimestamp
|
||||
mrs[i].Value = float64(i)
|
||||
}
|
||||
return mrs
|
||||
}
|
||||
const numMetrics = 1000
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Date(2023, 6, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 5, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
legacyData := genData(numMetrics, "legacy", tr)
|
||||
newData := genData(numMetrics, "new", tr)
|
||||
|
||||
assertSearchResults := func(s *Storage, want []MetricRow) {
|
||||
tfsAll := NewTagFilters()
|
||||
if err := tfsAll.Add([]byte("__name__"), []byte(".*"), false, true); err != nil {
|
||||
t.Fatalf("unexpected error in TagFilters.Add: %v", err)
|
||||
}
|
||||
if err := testAssertSearchResult(s, tr, tfsAll, want); err != nil {
|
||||
t.Fatalf("unexpected search results: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
assertLegacyData := func(s *Storage) {
|
||||
t.Helper()
|
||||
want := legacyData
|
||||
assertSearchResults(s, want)
|
||||
}
|
||||
assertNewData := func(s *Storage) {
|
||||
t.Helper()
|
||||
want := slices.Concat(legacyData, newData)
|
||||
assertSearchResults(s, want)
|
||||
}
|
||||
testSearchOpWithLegacyIndexDBs(t, legacyData, newData, assertLegacyData, assertNewData)
|
||||
}
|
||||
|
||||
func TestLegacyStorage_GetSeriesCount(t *testing.T) {
|
||||
const numMetrics = 1000
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Date(2023, 6, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 5, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
legacyData := testGenerateMetricRowsWithPrefix(rng, numMetrics, "legacy", tr)
|
||||
newData := testGenerateMetricRowsWithPrefix(rng, numMetrics, "new", tr)
|
||||
|
||||
assertSearchResults := func(s *Storage, want uint64) {
|
||||
t.Helper()
|
||||
got, err := s.GetSeriesCount(noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("GetSeriesCount() failed unexpectedly: %v", err)
|
||||
}
|
||||
if got != want {
|
||||
t.Fatalf("unexpected metric count: got %d, want %d", got, want)
|
||||
}
|
||||
}
|
||||
|
||||
assertLegacyData := func(s *Storage) {
|
||||
t.Helper()
|
||||
want := uint64(len(legacyData))
|
||||
assertSearchResults(s, want)
|
||||
}
|
||||
assertNewData := func(s *Storage) {
|
||||
t.Helper()
|
||||
want := uint64(len(legacyData) + len(newData))
|
||||
assertSearchResults(s, want)
|
||||
}
|
||||
testSearchOpWithLegacyIndexDBs(t, legacyData, newData, assertLegacyData, assertNewData)
|
||||
}
|
||||
|
||||
func TestLegacyStorage_DeleteSeries(t *testing.T) {
|
||||
const numMetrics = 1000
|
||||
tr := TimeRange{
|
||||
MinTimestamp: time.Date(2023, 6, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2024, 5, 31, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
legacyData := testGenerateMetricRowsWithPrefix(rng, numMetrics, "legacy", tr)
|
||||
newData := testGenerateMetricRowsWithPrefix(rng, numMetrics, "new", tr)
|
||||
tfsAll := NewTagFilters()
|
||||
if err := tfsAll.Add([]byte("__name__"), []byte(".*"), false, true); err != nil {
|
||||
t.Fatalf("unexpected error in TagFilters.Add: %v", err)
|
||||
}
|
||||
tfssAll := []*TagFilters{tfsAll}
|
||||
|
||||
assertSeriesCount := func(s *Storage, want int) {
|
||||
t.Helper()
|
||||
got, err := s.SearchMetricNames(nil, tfssAll, tr, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("SearchMetricNames() failed unexpectedly: %v", err)
|
||||
}
|
||||
if len(got) != want {
|
||||
t.Fatalf("unexpected metric count: got %d, want %d", len(got), want)
|
||||
}
|
||||
}
|
||||
|
||||
assertLegacyData := func(s *Storage) {
|
||||
t.Helper()
|
||||
want := len(legacyData)
|
||||
assertSeriesCount(s, want)
|
||||
}
|
||||
assertNewData := func(s *Storage) {
|
||||
t.Helper()
|
||||
want := len(legacyData) + len(newData)
|
||||
assertSeriesCount(s, want)
|
||||
|
||||
got, err := s.DeleteSeries(nil, tfssAll, 1e9)
|
||||
if err != nil {
|
||||
t.Fatalf("DeleteSeries() failed unexpectedly: %v", err)
|
||||
}
|
||||
if got != want {
|
||||
t.Fatalf("Unexpected number of deleted series: got %d, want %d", got, want)
|
||||
}
|
||||
|
||||
assertSeriesCount(s, 0)
|
||||
}
|
||||
testSearchOpWithLegacyIndexDBs(t, legacyData, newData, assertLegacyData, assertNewData)
|
||||
}
|
||||
|
||||
// testSearchWithLegacyIndexDBs a search operation when the index data
|
||||
// is located both partition and legacy indexDBs.
|
||||
func testSearchOpWithLegacyIndexDBs(t *testing.T, legacyData, newData []MetricRow, assertLegacyData, assertNewData func(s *Storage)) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
s := MustOpenStorage(t.Name(), OpenOptions{})
|
||||
s.AddRows(legacyData, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
assertLegacyData(s)
|
||||
|
||||
s = mustConvertToLegacy(s)
|
||||
assertLegacyData(s)
|
||||
|
||||
s.AddRows(newData, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
assertNewData(s)
|
||||
s.MustClose()
|
||||
}
|
||||
|
||||
func TestLegacyStorageSnapshots_CreateListDelete(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
const numRows = 10000
|
||||
minTimestamp := time.Date(2024, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
maxTimestamp := time.Date(2024, 2, 29, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
mrs := testGenerateMetricRows(rng, numRows, minTimestamp, maxTimestamp)
|
||||
|
||||
root := t.Name()
|
||||
s := MustOpenStorage(root, OpenOptions{})
|
||||
s.AddRows(mrs, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
// Convert to legacy 2 times in order to have both prev and curr legacy idbs.
|
||||
s = mustConvertToLegacy(s)
|
||||
s.AddRows(mrs, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
s = mustConvertToLegacy(s)
|
||||
defer s.MustClose()
|
||||
s.AddRows(mrs, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
|
||||
var (
|
||||
data = filepath.Join(root, dataDirname)
|
||||
smallData = filepath.Join(data, smallDirname)
|
||||
bigData = filepath.Join(data, bigDirname)
|
||||
indexData = filepath.Join(data, indexdbDirname)
|
||||
smallSnapshots = filepath.Join(smallData, snapshotsDirname)
|
||||
bigSnapshots = filepath.Join(bigData, snapshotsDirname)
|
||||
indexSnapshots = filepath.Join(indexData, snapshotsDirname)
|
||||
legacyIndexData = filepath.Join(root, indexdbDirname)
|
||||
legacyIndexSnapshots = filepath.Join(legacyIndexData, snapshotsDirname)
|
||||
)
|
||||
|
||||
snapshot1Name := s.MustCreateSnapshot()
|
||||
assertListSnapshots := func(want []string) {
|
||||
got := s.MustListSnapshots()
|
||||
if diff := cmp.Diff(want, got); diff != "" {
|
||||
t.Fatalf("unexpected snapshot list (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
assertListSnapshots([]string{snapshot1Name})
|
||||
|
||||
var (
|
||||
snapshot1 = filepath.Join(root, snapshotsDirname, snapshot1Name)
|
||||
smallSnapshot1 = filepath.Join(smallSnapshots, snapshot1Name)
|
||||
smallSymlink1 = filepath.Join(snapshot1, dataDirname, smallDirname)
|
||||
bigSnapshot1 = filepath.Join(bigSnapshots, snapshot1Name)
|
||||
bigSymlink1 = filepath.Join(snapshot1, dataDirname, bigDirname)
|
||||
indexSnapshot1 = filepath.Join(indexSnapshots, snapshot1Name)
|
||||
indexSymlink1 = filepath.Join(snapshot1, dataDirname, indexdbDirname)
|
||||
legacyIndexSnapshot1 = filepath.Join(legacyIndexSnapshots, snapshot1Name)
|
||||
legacyIndexSymlink1 = filepath.Join(snapshot1, indexdbDirname)
|
||||
)
|
||||
|
||||
// Check snapshot1 dir entries
|
||||
assertDirEntries := func(srcDir, snapshotDir string, excludePath ...string) {
|
||||
t.Helper()
|
||||
dataDirEntries := testListDirEntries(t, srcDir, excludePath...)
|
||||
snapshotDirEntries := testListDirEntries(t, snapshotDir)
|
||||
if diff := cmp.Diff(dataDirEntries, snapshotDirEntries); diff != "" {
|
||||
t.Fatalf("unexpected snapshot dir entries (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
assertDirEntries(smallData, smallSnapshot1, smallSnapshots)
|
||||
assertDirEntries(bigData, bigSnapshot1, bigSnapshots)
|
||||
assertDirEntries(indexData, indexSnapshot1, indexSnapshots)
|
||||
assertDirEntries(legacyIndexData, legacyIndexSnapshot1, legacyIndexSnapshots)
|
||||
|
||||
// Check snapshot1 symlinks
|
||||
assertSymlink := func(symlink string, wantRealpath string) {
|
||||
t.Helper()
|
||||
gotRealpath, err := filepath.EvalSymlinks(symlink)
|
||||
if err != nil {
|
||||
t.Fatalf("Could not evaluate symlink %q: %v", symlink, err)
|
||||
}
|
||||
if gotRealpath != wantRealpath {
|
||||
t.Fatalf("unexpected realpath for symlink %q: got %q, want %q", symlink, gotRealpath, wantRealpath)
|
||||
}
|
||||
}
|
||||
assertSymlink(bigSymlink1, bigSnapshot1)
|
||||
assertSymlink(smallSymlink1, smallSnapshot1)
|
||||
assertSymlink(indexSymlink1, indexSnapshot1)
|
||||
assertSymlink(legacyIndexSymlink1, legacyIndexSnapshot1)
|
||||
|
||||
// Rotate indexdb. Only one legacy indexDB must remain.
|
||||
s.legacyMustRotateIndexDB(time.Now().UTC())
|
||||
|
||||
// Create snapshot2
|
||||
snapshot2Name := s.MustCreateSnapshot()
|
||||
assertListSnapshots([]string{snapshot1Name, snapshot2Name})
|
||||
|
||||
var (
|
||||
snapshot2 = filepath.Join(root, snapshotsDirname, snapshot2Name)
|
||||
smallSnapshot2 = filepath.Join(smallSnapshots, snapshot2Name)
|
||||
smallSymlink2 = filepath.Join(snapshot2, dataDirname, smallDirname)
|
||||
bigSnapshot2 = filepath.Join(bigSnapshots, snapshot2Name)
|
||||
bigSymlink2 = filepath.Join(snapshot2, dataDirname, bigDirname)
|
||||
indexSnapshot2 = filepath.Join(indexSnapshots, snapshot2Name)
|
||||
indexSymlink2 = filepath.Join(snapshot2, dataDirname, indexdbDirname)
|
||||
legacyIndexSnapshot2 = filepath.Join(legacyIndexSnapshots, snapshot2Name)
|
||||
legacyIndexSymlink2 = filepath.Join(snapshot2, indexdbDirname)
|
||||
)
|
||||
|
||||
// Check snapshot2 dir entries
|
||||
assertDirEntries(smallData, smallSnapshot2, smallSnapshots)
|
||||
assertDirEntries(bigData, bigSnapshot2, bigSnapshots)
|
||||
assertDirEntries(indexData, indexSnapshot2, indexSnapshots)
|
||||
assertDirEntries(legacyIndexData, legacyIndexSnapshot2, legacyIndexSnapshots)
|
||||
|
||||
// Check snapshot2 symlinks
|
||||
assertSymlink(bigSymlink2, bigSnapshot2)
|
||||
assertSymlink(smallSymlink2, smallSnapshot2)
|
||||
assertSymlink(indexSymlink2, indexSnapshot2)
|
||||
assertSymlink(legacyIndexSymlink2, legacyIndexSnapshot2)
|
||||
|
||||
// Rotate indexdb once again. There shouldn't be any legacy indexDBs left.
|
||||
s.legacyMustRotateIndexDB(time.Now().UTC())
|
||||
|
||||
// Create snapshot3
|
||||
snapshot3Name := s.MustCreateSnapshot()
|
||||
assertListSnapshots([]string{snapshot1Name, snapshot2Name, snapshot3Name})
|
||||
|
||||
var (
|
||||
snapshot3 = filepath.Join(root, snapshotsDirname, snapshot3Name)
|
||||
smallSnapshot3 = filepath.Join(smallSnapshots, snapshot3Name)
|
||||
smallSymlink3 = filepath.Join(snapshot3, dataDirname, smallDirname)
|
||||
bigSnapshot3 = filepath.Join(bigSnapshots, snapshot3Name)
|
||||
bigSymlink3 = filepath.Join(snapshot3, dataDirname, bigDirname)
|
||||
indexSnapshot3 = filepath.Join(indexSnapshots, snapshot3Name)
|
||||
indexSymlink3 = filepath.Join(snapshot3, dataDirname, indexdbDirname)
|
||||
legacyIndexSnapshot3 = filepath.Join(legacyIndexSnapshots, snapshot3Name)
|
||||
legacyIndexSymlink3 = filepath.Join(snapshot3, indexdbDirname)
|
||||
)
|
||||
|
||||
assertPathDoesNotExist := func(path string) {
|
||||
t.Helper()
|
||||
if fs.IsPathExist(path) {
|
||||
t.Fatalf("path was not expected to exist: %q", path)
|
||||
}
|
||||
}
|
||||
|
||||
// Check snapshot3 dir entries
|
||||
assertDirEntries(smallData, smallSnapshot3, smallSnapshots)
|
||||
assertDirEntries(bigData, bigSnapshot3, bigSnapshots)
|
||||
assertDirEntries(indexData, indexSnapshot3, indexSnapshots)
|
||||
assertPathDoesNotExist(legacyIndexSnapshot3)
|
||||
|
||||
// Check snapshot3 symlinks
|
||||
assertSymlink(bigSymlink3, bigSnapshot3)
|
||||
assertSymlink(smallSymlink3, smallSnapshot3)
|
||||
assertSymlink(indexSymlink3, indexSnapshot3)
|
||||
assertPathDoesNotExist(legacyIndexSymlink3)
|
||||
|
||||
// Check snapshot deletion.
|
||||
for _, name := range []string{snapshot1Name, snapshot2Name, snapshot3Name} {
|
||||
if err := s.DeleteSnapshot(name); err != nil {
|
||||
t.Fatalf("could not delete snapshot %q: %v", name, err)
|
||||
}
|
||||
}
|
||||
assertListSnapshots([]string{})
|
||||
assertPathDoesNotExist(snapshot1)
|
||||
assertPathDoesNotExist(snapshot2)
|
||||
assertPathDoesNotExist(snapshot3)
|
||||
assertPathDoesNotExist(bigSnapshot1)
|
||||
assertPathDoesNotExist(bigSnapshot2)
|
||||
assertPathDoesNotExist(bigSnapshot3)
|
||||
assertPathDoesNotExist(smallSnapshot1)
|
||||
assertPathDoesNotExist(smallSnapshot2)
|
||||
assertPathDoesNotExist(smallSnapshot3)
|
||||
assertPathDoesNotExist(indexSnapshot1)
|
||||
assertPathDoesNotExist(indexSnapshot2)
|
||||
assertPathDoesNotExist(indexSnapshot3)
|
||||
assertPathDoesNotExist(legacyIndexSnapshot1)
|
||||
assertPathDoesNotExist(legacyIndexSnapshot2)
|
||||
}
|
||||
|
||||
func TestStorageConvertToLegacy(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
assertMetricNames := func(s *Storage, tr TimeRange, wantMRs []MetricRow) {
|
||||
t.Helper()
|
||||
tfs := NewTagFilters()
|
||||
if err := tfs.Add([]byte("__name__"), []byte(".*"), false, true); err != nil {
|
||||
t.Fatalf("unexpected error in TagFilters.Add: %v", err)
|
||||
}
|
||||
got, err := s.SearchMetricNames(nil, []*TagFilters{tfs}, tr, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("SearchMetricNames() failed unexpectedly: %v", err)
|
||||
}
|
||||
var mn MetricName
|
||||
for i, name := range got {
|
||||
if err := mn.UnmarshalString(name); err != nil {
|
||||
t.Fatalf("could not unmarshal metric name %q: %v", name, err)
|
||||
}
|
||||
got[i] = string(mn.MetricGroup)
|
||||
}
|
||||
slices.Sort(got)
|
||||
want := make([]string, len(wantMRs))
|
||||
for i, mr := range wantMRs {
|
||||
if err := mn.UnmarshalRaw(mr.MetricNameRaw); err != nil {
|
||||
t.Fatalf("could not unmarshal raw metric name %v: %v", mr.MetricNameRaw, err)
|
||||
}
|
||||
want[i] = string(mn.MetricGroup)
|
||||
}
|
||||
slices.Sort(want)
|
||||
if diff := cmp.Diff(want, got); diff != "" {
|
||||
t.Fatalf("unexpected metric names (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
const numSeries = 10
|
||||
tr1 := TimeRange{
|
||||
MinTimestamp: time.Date(2025, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli(),
|
||||
MaxTimestamp: time.Date(2025, 1, 1, 23, 59, 59, 999_999_999, time.UTC).UnixMilli(),
|
||||
}
|
||||
tr2 := TimeRange{
|
||||
MinTimestamp: tr1.MinTimestamp + msecPerDay,
|
||||
MaxTimestamp: tr1.MaxTimestamp + msecPerDay,
|
||||
}
|
||||
tr3 := TimeRange{
|
||||
MinTimestamp: tr2.MinTimestamp + msecPerDay,
|
||||
MaxTimestamp: tr2.MaxTimestamp + msecPerDay,
|
||||
}
|
||||
tr4 := TimeRange{
|
||||
MinTimestamp: tr3.MinTimestamp + msecPerDay,
|
||||
MaxTimestamp: tr3.MaxTimestamp + msecPerDay,
|
||||
}
|
||||
trAll := TimeRange{
|
||||
MinTimestamp: tr1.MinTimestamp,
|
||||
MaxTimestamp: tr4.MaxTimestamp,
|
||||
}
|
||||
mrs1 := testGenerateMetricRowsWithPrefix(rng, numSeries, "generation1", tr1)
|
||||
mrs2 := testGenerateMetricRowsWithPrefix(rng, numSeries, "generation2", tr2)
|
||||
mrs3 := testGenerateMetricRowsWithPrefix(rng, numSeries, "generation3", tr3)
|
||||
mrs4 := testGenerateMetricRowsWithPrefix(rng, numSeries, "generation4", tr4)
|
||||
|
||||
s := MustOpenStorage(t.Name(), OpenOptions{})
|
||||
s.AddRows(mrs1, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
s = mustConvertToLegacy(s)
|
||||
assertMetricNames(s, trAll, mrs1)
|
||||
|
||||
s.AddRows(mrs2, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
s = mustConvertToLegacy(s)
|
||||
assertMetricNames(s, trAll, slices.Concat(mrs1, mrs2))
|
||||
|
||||
s.AddRows(mrs3, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
s = mustConvertToLegacy(s)
|
||||
assertMetricNames(s, trAll, slices.Concat(mrs2, mrs3))
|
||||
|
||||
s.AddRows(mrs4, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
s = mustConvertToLegacy(s)
|
||||
assertMetricNames(s, trAll, slices.Concat(mrs3, mrs4))
|
||||
|
||||
s.MustClose()
|
||||
}
|
||||
|
||||
// mustConvertToLegacy converts the storage partition indexDBs into a
|
||||
// legacy indexDB. The original partition indexDBs are removed.
|
||||
//
|
||||
// Each invocation of this function will a new legacy indexDB in
|
||||
// storageDataPath/indexdb dir. The function will keep only 2 most recent
|
||||
// indexDBs under that path.
|
||||
//
|
||||
// The function also deteles all persistent caches.
|
||||
func mustConvertToLegacy(s *Storage) *Storage {
|
||||
// Stop storage, move legacy idbs to tmp dir, delete all caches,
|
||||
// re-open storage with pt index only.
|
||||
storageDataPath := s.path
|
||||
s.MustClose()
|
||||
legacyIDBsPathOrig := filepath.Join(s.path, indexdbDirname)
|
||||
fs.MustMkdirIfNotExist(legacyIDBsPathOrig)
|
||||
legacyIDBsPathTmp := filepath.Join(s.path, "indexdb-legacy")
|
||||
if err := os.Rename(legacyIDBsPathOrig, legacyIDBsPathTmp); err != nil {
|
||||
panic(fmt.Sprintf("could not rename %q to %q: %v", legacyIDBsPathOrig, legacyIDBsPathTmp, err))
|
||||
}
|
||||
fs.MustRemoveDir(filepath.Join(storageDataPath, cacheDirname))
|
||||
s = MustOpenStorage(storageDataPath, OpenOptions{})
|
||||
|
||||
legacyIDBID := uint64(time.Now().UnixNano())
|
||||
legacyIDBName := fmt.Sprintf("%016X", legacyIDBID)
|
||||
legacyIDBPath := filepath.Join(legacyIDBsPathTmp, legacyIDBName)
|
||||
fs.MustMkdirFailIfExist(legacyIDBPath)
|
||||
legacyIDBPartsFile := filepath.Join(legacyIDBPath, partsFilename)
|
||||
fs.MustWriteAtomic(legacyIDBPartsFile, []byte("[]"), true)
|
||||
legacyIDBTimeRange := TimeRange{
|
||||
MinTimestamp: 0,
|
||||
MaxTimestamp: math.MaxInt64,
|
||||
}
|
||||
var isReadOnly atomic.Bool
|
||||
isReadOnly.Store(false)
|
||||
legacyIDB := mustOpenIndexDB(legacyIDBID, legacyIDBTimeRange, legacyIDBName, legacyIDBPath, s, &isReadOnly, false)
|
||||
|
||||
// Read index items from the partition indexDBs and write them to the legacy
|
||||
// indexDB.
|
||||
|
||||
ptws := s.tb.GetPartitions(legacyIDBTimeRange)
|
||||
tfsAll := NewTagFilters()
|
||||
if err := tfsAll.Add([]byte("__name__"), []byte(".*"), false, true); err != nil {
|
||||
panic(fmt.Sprintf("unexpected error in TagFilters.Add: %v", err))
|
||||
}
|
||||
tfssAll := []*TagFilters{tfsAll}
|
||||
seenGlobalIndexEntries := make(map[uint64]bool)
|
||||
type dateMetricID struct {
|
||||
date uint64
|
||||
metricID uint64
|
||||
}
|
||||
seenPerDayIndexEntries := make(map[dateMetricID]bool)
|
||||
for _, ptw := range ptws {
|
||||
idb := ptw.pt.idb
|
||||
for ts := idb.tr.MinTimestamp; ts < idb.tr.MaxTimestamp; ts += msecPerDay {
|
||||
day := TimeRange{
|
||||
MinTimestamp: ts,
|
||||
MaxTimestamp: ts + msecPerDay - 1,
|
||||
}
|
||||
date := uint64(ts / msecPerDay)
|
||||
tsids, err := idb.SearchTSIDs(nil, tfssAll, day, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("could not get TSIDs: %v", err))
|
||||
}
|
||||
for _, tsid := range tsids {
|
||||
metricID := tsid.MetricID
|
||||
mnBytes, ok := idb.searchMetricName(nil, metricID, false)
|
||||
if !ok {
|
||||
panic(fmt.Sprintf("could not get metric name for metricID %d", metricID))
|
||||
}
|
||||
var mn MetricName
|
||||
if err := mn.Unmarshal(mnBytes); err != nil {
|
||||
panic(fmt.Sprintf("Could not unmarshal metric name from bytes %q: %v", string(mnBytes), err))
|
||||
}
|
||||
if !seenGlobalIndexEntries[metricID] {
|
||||
legacyIDB.createGlobalIndexes(&tsid, &mn)
|
||||
seenGlobalIndexEntries[metricID] = true
|
||||
}
|
||||
dateMetricID := dateMetricID{
|
||||
date: date,
|
||||
metricID: metricID,
|
||||
}
|
||||
if !seenPerDayIndexEntries[dateMetricID] {
|
||||
legacyIDB.createPerDayIndexes(date, &tsid, &mn)
|
||||
seenPerDayIndexEntries[dateMetricID] = true
|
||||
}
|
||||
}
|
||||
}
|
||||
is := idb.getIndexSearch(noDeadline)
|
||||
dmis, err := is.loadDeletedMetricIDs()
|
||||
idb.putIndexSearch(is)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("cannot load deleted metricIDs for indexDB %q: %v", idb.name, err))
|
||||
}
|
||||
legacyIDB.saveDeletedMetricIDs(dmis)
|
||||
}
|
||||
|
||||
s.tb.PutPartitions(ptws)
|
||||
legacyIDB.MustClose()
|
||||
|
||||
// Stop storage, delete partition idbs, remove caches, move legacy idb dir
|
||||
// to its original location, keep only 2 recent legacy idbs.
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(filepath.Join(storageDataPath, dataDirname, indexdbDirname))
|
||||
fs.MustRemoveDir(filepath.Join(storageDataPath, cacheDirname))
|
||||
if err := os.Rename(legacyIDBsPathTmp, legacyIDBsPathOrig); err != nil {
|
||||
panic(fmt.Sprintf("could not rename %q to %q: %v", legacyIDBsPathTmp, legacyIDBsPathOrig, err))
|
||||
}
|
||||
entries := fs.MustReadDir(legacyIDBsPathOrig)
|
||||
var names []string
|
||||
for _, e := range entries {
|
||||
names = append(names, e.Name())
|
||||
}
|
||||
sort.Slice(names, func(i, j int) bool {
|
||||
return names[i] < names[j]
|
||||
})
|
||||
if len(names) > 2 {
|
||||
for _, name := range names[:len(names)-2] {
|
||||
p := filepath.Join(legacyIDBsPathOrig, name)
|
||||
fs.MustRemoveDir(p)
|
||||
}
|
||||
}
|
||||
|
||||
return MustOpenStorage(s.path, OpenOptions{})
|
||||
}
|
||||
@@ -27,8 +27,9 @@ func TestStorageSearchTSIDs_CorruptedIndex(t *testing.T) {
|
||||
}
|
||||
const numMetrics = 10
|
||||
date := uint64(tr.MinTimestamp) / msecPerDay
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
defer s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
ptw := s.tb.MustGetPartition(tr.MinTimestamp)
|
||||
idb := ptw.pt.idb
|
||||
defer s.tb.PutPartition(ptw)
|
||||
var wantMetricIDs []uint64
|
||||
|
||||
// Simulate corrupted index by not creating nsPrefixMetricIDToTSID
|
||||
@@ -44,9 +45,9 @@ func TestStorageSearchTSIDs_CorruptedIndex(t *testing.T) {
|
||||
skipMetricIDToTSID: true,
|
||||
})
|
||||
|
||||
idbCurr.tb.AddItems(ii.Items)
|
||||
idb.tb.AddItems(ii.Items)
|
||||
}
|
||||
idbCurr.tb.DebugFlush()
|
||||
idb.tb.DebugFlush()
|
||||
|
||||
tfsAll := NewTagFilters()
|
||||
if err := tfsAll.Add([]byte("__name__"), []byte(".*"), false, true); err != nil {
|
||||
@@ -55,7 +56,7 @@ func TestStorageSearchTSIDs_CorruptedIndex(t *testing.T) {
|
||||
tfssAll := []*TagFilters{tfsAll}
|
||||
|
||||
searchMetricIDs := func() []uint64 {
|
||||
metricIDs, err := idbCurr.searchMetricIDs(nil, tfssAll, tr, 1e9, noDeadline)
|
||||
metricIDs, err := idb.searchMetricIDs(nil, tfssAll, tr, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("searchMetricIDs() failed unexpectedly: %v", err))
|
||||
}
|
||||
@@ -89,7 +90,7 @@ func TestStorageSearchTSIDs_CorruptedIndex(t *testing.T) {
|
||||
// is not incremented yet.
|
||||
var m Metrics
|
||||
s.UpdateMetrics(&m)
|
||||
if got, want := m.IndexDBMetrics.MissingTSIDsForMetricID, uint64(0); got != want {
|
||||
if got, want := m.TableMetrics.IndexDBMetrics.MissingTSIDsForMetricID, uint64(0); got != want {
|
||||
t.Fatalf("unexpected MissingTSIDsForMetricID: got %d, want %d", got, want)
|
||||
}
|
||||
|
||||
@@ -108,7 +109,7 @@ func TestStorageSearchTSIDs_CorruptedIndex(t *testing.T) {
|
||||
// Ensure the metric that counts metricIDs for which no TSIDs were found
|
||||
// is incremented after the metricID deletion.
|
||||
s.UpdateMetrics(&m)
|
||||
if got, want := m.IndexDBMetrics.MissingTSIDsForMetricID, uint64(numMetrics); got != want {
|
||||
if got, want := m.TableMetrics.IndexDBMetrics.MissingTSIDsForMetricID, uint64(numMetrics); got != want {
|
||||
t.Fatalf("unexpected MissingTSIDsForMetricID: got %d, want %d", got, want)
|
||||
}
|
||||
})
|
||||
@@ -128,8 +129,9 @@ func TestStorageSearchMetricNames_CorruptedIndex(t *testing.T) {
|
||||
}
|
||||
const numMetrics = 10
|
||||
date := uint64(tr.MinTimestamp) / msecPerDay
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
defer s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
ptw := s.tb.MustGetPartition(tr.MinTimestamp)
|
||||
idb := ptw.pt.idb
|
||||
defer s.tb.PutPartition(ptw)
|
||||
var wantMetricIDs []uint64
|
||||
|
||||
// Simulate corrupted index by not creating nsPrefixMetricIDToMetricName
|
||||
@@ -145,9 +147,9 @@ func TestStorageSearchMetricNames_CorruptedIndex(t *testing.T) {
|
||||
skipMetricIDToMetricName: true,
|
||||
})
|
||||
|
||||
idbCurr.tb.AddItems(ii.Items)
|
||||
idb.tb.AddItems(ii.Items)
|
||||
}
|
||||
idbCurr.tb.DebugFlush()
|
||||
idb.tb.DebugFlush()
|
||||
|
||||
tfsAll := NewTagFilters()
|
||||
if err := tfsAll.Add([]byte("__name__"), []byte(".*"), false, true); err != nil {
|
||||
@@ -156,7 +158,7 @@ func TestStorageSearchMetricNames_CorruptedIndex(t *testing.T) {
|
||||
tfssAll := []*TagFilters{tfsAll}
|
||||
|
||||
searchMetricIDs := func() []uint64 {
|
||||
metricIDs, err := idbCurr.searchMetricIDs(nil, tfssAll, tr, 1e9, noDeadline)
|
||||
metricIDs, err := idb.searchMetricIDs(nil, tfssAll, tr, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("searchMetricIDs() failed unexpectedly: %v", err))
|
||||
}
|
||||
@@ -190,7 +192,7 @@ func TestStorageSearchMetricNames_CorruptedIndex(t *testing.T) {
|
||||
// were found is not incremented yet.
|
||||
var m Metrics
|
||||
s.UpdateMetrics(&m)
|
||||
if got, want := m.IndexDBMetrics.MissingMetricNamesForMetricID, uint64(0); got != want {
|
||||
if got, want := m.TableMetrics.IndexDBMetrics.MissingMetricNamesForMetricID, uint64(0); got != want {
|
||||
t.Fatalf("unexpected MissingMetricNamesForMetricID: got %d, want %d", got, want)
|
||||
}
|
||||
|
||||
@@ -209,7 +211,7 @@ func TestStorageSearchMetricNames_CorruptedIndex(t *testing.T) {
|
||||
// Ensure the metric that counts metricIDs for which no metric names
|
||||
// were found is incremented after the metricID deletion.
|
||||
s.UpdateMetrics(&m)
|
||||
if got, want := m.IndexDBMetrics.MissingMetricNamesForMetricID, uint64(numMetrics); got != want {
|
||||
if got, want := m.TableMetrics.IndexDBMetrics.MissingMetricNamesForMetricID, uint64(numMetrics); got != want {
|
||||
t.Fatalf("unexpected MissingMetricNamesForMetricID: got %d, want %d", got, want)
|
||||
}
|
||||
})
|
||||
@@ -283,89 +285,99 @@ func testCreateIndexItems(date uint64, tsid *TSID, mn *MetricName, opts testInde
|
||||
}
|
||||
|
||||
func TestStorageRotateIndexDBPrefill(t *testing.T) {
|
||||
f := func(opts OpenOptions, prefillStart time.Duration) {
|
||||
defer testRemoveAll(t)
|
||||
defer testRemoveAll(t)
|
||||
f := func(t *testing.T, opts OpenOptions, prefillStart time.Duration) {
|
||||
t.Helper()
|
||||
|
||||
synctest.Test(t, func(t *testing.T) {
|
||||
// Align start time to 05:00 in order to have 23h before the next rotation cycle at 04:00 next morning.
|
||||
time.Sleep(time.Hour * 5)
|
||||
|
||||
nextRotationTime := time.Now().Add(time.Hour * 23).Truncate(time.Hour)
|
||||
synctest.Run(func() {
|
||||
// Prefill of the next partition indexDB happens during the
|
||||
// (nextMonth-prefillStart, nextMonth] time interval.
|
||||
// Advance current time right before the the beginning of that interval.
|
||||
ct := time.Now().UTC()
|
||||
nextMonth := time.Date(ct.Year(), ct.Month()+1, 1, 0, 0, 0, 0, time.UTC)
|
||||
time.Sleep(nextMonth.Sub(ct.Add(prefillStart)))
|
||||
|
||||
s := MustOpenStorage(t.Name(), opts)
|
||||
defer s.MustClose()
|
||||
// first rotation cycle in 4 hours due to synctest start time of 00:00:00
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
ct := time.Now()
|
||||
tr := TimeRange{
|
||||
MinTimestamp: ct.Add(time.Hour).UnixMilli(),
|
||||
MaxTimestamp: ct.Add(time.Hour * 24).UnixMilli(),
|
||||
}
|
||||
|
||||
const numSeries = 1000
|
||||
|
||||
mrs := testGenerateMetricRowsWithPrefix(rng, numSeries, "metric.", tr)
|
||||
s.AddRows(mrs, 1)
|
||||
s.DebugFlush()
|
||||
createdSeries := s.newTimeseriesCreated.Load()
|
||||
if createdSeries != numSeries {
|
||||
t.Fatalf("unexpected number of created series (-%d;+%d)", numSeries, createdSeries)
|
||||
addRows := func() {
|
||||
t.Helper()
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
ct := time.Now().UTC()
|
||||
tr := TimeRange{
|
||||
MinTimestamp: ct.Add(-prefillStart).UnixMilli(),
|
||||
MaxTimestamp: ct.UnixMilli(),
|
||||
}
|
||||
mrs := testGenerateMetricRowsWithPrefix(rng, numSeries, "metric.", tr)
|
||||
s.AddRows(mrs, 1)
|
||||
s.DebugFlush()
|
||||
}
|
||||
|
||||
// Sleep until a minute before the prefill start time,
|
||||
// then verify that no timeseries have been pre-created yet.
|
||||
time.Sleep(time.Hour*23 - prefillStart - 1*time.Minute)
|
||||
s.AddRows(mrs, 1)
|
||||
s.DebugFlush()
|
||||
preCreated := s.timeseriesPreCreated.Load()
|
||||
if preCreated != 0 {
|
||||
t.Fatalf("expected no timeseries to be re-created, got: %d", preCreated)
|
||||
// Insert metrics into the empty storage right before the prefill
|
||||
// interval starts.
|
||||
addRows()
|
||||
if got, want := s.newTimeseriesCreated.Load(), uint64(numSeries); got != want {
|
||||
t.Fatalf("unexpected number of new timeseries: got %d, want %d", got, want)
|
||||
}
|
||||
if got, want := s.timeseriesPreCreated.Load(), uint64(0); got != want {
|
||||
t.Fatalf("unexpected number of pre-created timeseries: got %d, want %d", got, want)
|
||||
}
|
||||
|
||||
// Sleep until half of the prefill rotation interval has elapsed,
|
||||
// Sleep until half of the prefill interval has elapsed,
|
||||
// then verify that some time series have been pre-created.
|
||||
time.Sleep(prefillStart / 2)
|
||||
s.AddRows(mrs, 1)
|
||||
s.DebugFlush()
|
||||
preCreated = s.timeseriesPreCreated.Load()
|
||||
if preCreated == 0 {
|
||||
t.Fatalf("expected some timeseries to be re-created, got: %d", preCreated)
|
||||
addRows()
|
||||
if got, want := s.timeseriesPreCreated.Load(), uint64(0); got <= want {
|
||||
t.Fatalf("unexpected number of pre-created timeseries: got %d, want > %d", got, want)
|
||||
}
|
||||
|
||||
// Sleep until a minute before the index rotation,
|
||||
// verify that almost all time series have been pre-created.
|
||||
time.Sleep(nextRotationTime.Sub(time.Now().Add(time.Minute)))
|
||||
s.AddRows(mrs, 1)
|
||||
s.DebugFlush()
|
||||
preCreated = s.timeseriesPreCreated.Load()
|
||||
if preCreated == 0 || preCreated < numSeries/2 {
|
||||
t.Fatalf("expected more than 50 percent of timeseries to be re-created, got: %d", preCreated)
|
||||
// Sleep until a minute before the next partition transition, verify
|
||||
// that almost all time series have been pre-created.
|
||||
ct = time.Now().UTC()
|
||||
time.Sleep(nextMonth.Sub(ct.Add(time.Minute)))
|
||||
addRows()
|
||||
if got, want := s.timeseriesPreCreated.Load(), uint64(numSeries/2); got <= want {
|
||||
t.Fatalf("unexpected number of pre-created timeseries: got %d, want > %d", got, want)
|
||||
}
|
||||
|
||||
// Sleep until the rotation is over, verify that the rest of time series have been re-created
|
||||
time.Sleep(time.Hour)
|
||||
s.AddRows(mrs, 1)
|
||||
s.DebugFlush()
|
||||
createdSeries, reCreated, rePopulated := s.newTimeseriesCreated.Load(), s.timeseriesPreCreated.Load(), s.timeseriesRepopulated.Load()
|
||||
if createdSeries != numSeries {
|
||||
t.Fatalf("unexpected number of created series (-%d;+%d)", numSeries, createdSeries)
|
||||
}
|
||||
if reCreated+rePopulated != numSeries {
|
||||
t.Fatalf("unexpected number of re-created=%d and re-populated=%d series, want sum to be equal to %d", numSeries, createdSeries, numSeries)
|
||||
// Align the time with the start of the next month.
|
||||
time.Sleep(time.Minute)
|
||||
// Sleep until the transition to the next partition is over, verify
|
||||
// that the rest of time series have been re-created
|
||||
time.Sleep(prefillStart)
|
||||
newCreated := s.newTimeseriesCreated.Load()
|
||||
addRows()
|
||||
newCreated = s.newTimeseriesCreated.Load() - newCreated
|
||||
// If jump in time is bigger than 1h, the tsidCache will be cleared
|
||||
// and therefore the metrics will not be repopulated. Instead, new
|
||||
// metrics will be created.
|
||||
preCreated, repopulated := s.timeseriesPreCreated.Load(), s.timeseriesRepopulated.Load()
|
||||
if preCreated+repopulated+newCreated != numSeries {
|
||||
t.Fatalf("unexpected number of pre-populated, repopulated, and new timeseries: got %d + %d + %d, want %d", preCreated, repopulated, newCreated, numSeries)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// Test the default prefill start duration, see -storage.idbPrefillStart flag:
|
||||
// VictoriaMetrics starts prefill indexDB at 3 A.M UTC, while indexDB rotates at 4 A.M UTC.
|
||||
f(OpenOptions{Retention: time.Hour * 24, IDBPrefillStart: time.Hour}, time.Hour)
|
||||
|
||||
// Zero IDBPrefillStart option should fallback to 1 hour prefill start:
|
||||
f(OpenOptions{Retention: time.Hour * 24, IDBPrefillStart: 0}, time.Hour)
|
||||
|
||||
// Test a custom prefill duration: 2h:
|
||||
// VictoriaMetrics starts prefill indexDB at 2 A.M UTC, while indexDB rotates at 4 A.M UTC.
|
||||
f(OpenOptions{Retention: time.Hour * 24, IDBPrefillStart: 2 * time.Hour}, 2*time.Hour)
|
||||
// Verify an interval that is shorter than one hour.
|
||||
t.Run("30m", func(t *testing.T) {
|
||||
f(t, OpenOptions{IDBPrefillStart: 30 * time.Minute}, 30*time.Minute)
|
||||
})
|
||||
// Verify 1h inteval (which is also the default).
|
||||
// tsidCache will be cleared because it will have two cache rotations (one
|
||||
// every 30 mins). This means that once the new month starts the timeseries
|
||||
// that waren't pre-populated will be re-created instead of being
|
||||
// re-populated.
|
||||
t.Run("default", func(t *testing.T) {
|
||||
f(t, OpenOptions{IDBPrefillStart: 0}, time.Hour)
|
||||
})
|
||||
t.Run("1h", func(t *testing.T) {
|
||||
f(t, OpenOptions{IDBPrefillStart: time.Hour}, time.Hour)
|
||||
})
|
||||
// Vefiry 2h interval. Same here, the tsidCache will be cleared.
|
||||
t.Run("2h", func(t *testing.T) {
|
||||
f(t, OpenOptions{IDBPrefillStart: 2 * time.Hour}, 2*time.Hour)
|
||||
})
|
||||
}
|
||||
|
||||
// TestStorageAddRows_nextDayIndexPrefill tests gradual creation of per-day
|
||||
@@ -538,11 +550,11 @@ func TestStorageAddRows_nextDayIndexPrefill(t *testing.T) {
|
||||
func TestStorageMustLoadNextDayMetricIDs(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
assertNextDayMetricIDs := func(t *testing.T, gotNextDayMetricIDs *nextDayMetricIDs, wantGen, wantDate uint64, wantLen int) {
|
||||
assertNextDayMetricIDs := func(t *testing.T, gotNextDayMetricIDs *nextDayMetricIDs, wantIDBID, wantDate uint64, wantLen int) {
|
||||
t.Helper()
|
||||
|
||||
if got, want := gotNextDayMetricIDs.generation, wantGen; got != want {
|
||||
t.Fatalf("unexpected nextDayMetricIDs idb generation: got %d, want %d", got, want)
|
||||
if got, want := gotNextDayMetricIDs.idbID, wantIDBID; got != want {
|
||||
t.Fatalf("unexpected nextDayMetricIDs idb id: got %d, want %d", got, want)
|
||||
}
|
||||
if got, want := gotNextDayMetricIDs.date, wantDate; got != want {
|
||||
t.Fatalf("unexpected nextDayMetricIDs date: got %d, want %d", got, want)
|
||||
@@ -556,13 +568,13 @@ func TestStorageMustLoadNextDayMetricIDs(t *testing.T) {
|
||||
// synctest starts at 2000-01-01T00:00:00Z.
|
||||
// Advance time to 23:30 to enable next day prefill.
|
||||
time.Sleep(23*time.Hour + 30*time.Minute) // 2000-01-01T23:30:00Z
|
||||
date := uint64(time.Now().UnixMilli() / msecPerDay)
|
||||
date := uint64(time.Now().UnixMilli()) / msecPerDay
|
||||
|
||||
const numSeries = 1000
|
||||
s := MustOpenStorage(t.Name(), OpenOptions{})
|
||||
idbPrev, idbCurr := s.getPrevAndCurrIndexDBs()
|
||||
genCurr := idbCurr.generation
|
||||
s.putPrevAndCurrIndexDBs(idbPrev, idbCurr)
|
||||
ptw := s.tb.MustGetPartition(time.Now().UnixMilli())
|
||||
idbID := ptw.pt.idb.id
|
||||
s.tb.PutPartition(ptw)
|
||||
|
||||
rng := rand.New(rand.NewSource(1))
|
||||
mrs := testGenerateMetricRowsWithPrefix(rng, numSeries, "metric", TimeRange{
|
||||
@@ -579,7 +591,7 @@ func TestStorageMustLoadNextDayMetricIDs(t *testing.T) {
|
||||
numNextDayMetricIDs := s.pendingNextDayMetricIDs.Len()
|
||||
// But not in the nextDayMetricIDs cache. The pending metrics will be
|
||||
// moved to it by a bg process a few seconds later.
|
||||
assertNextDayMetricIDs(t, s.nextDayMetricIDs.Load(), genCurr, date, 0)
|
||||
assertNextDayMetricIDs(t, s.nextDayMetricIDs.Load(), idbID, date, 0)
|
||||
|
||||
// Wait for nextDayMetricIDs cache to populate.
|
||||
time.Sleep(15 * time.Second)
|
||||
@@ -592,7 +604,7 @@ func TestStorageMustLoadNextDayMetricIDs(t *testing.T) {
|
||||
}
|
||||
// While the actual cache, must contain the exact number of metricIDs
|
||||
// that once were pending.
|
||||
assertNextDayMetricIDs(t, s.nextDayMetricIDs.Load(), genCurr, date, numNextDayMetricIDs)
|
||||
assertNextDayMetricIDs(t, s.nextDayMetricIDs.Load(), idbID, date, numNextDayMetricIDs)
|
||||
|
||||
// Close the storage to persist nextDayMetricIDs cache to a file.
|
||||
s.MustClose()
|
||||
@@ -602,18 +614,19 @@ func TestStorageMustLoadNextDayMetricIDs(t *testing.T) {
|
||||
if got := s.pendingNextDayMetricIDs.Len(); got != 0 {
|
||||
t.Fatalf("unexpected pendingNextDayMetricIDs count: got %d, want 0", got)
|
||||
}
|
||||
assertNextDayMetricIDs(t, s.nextDayMetricIDs.Load(), genCurr, date, numNextDayMetricIDs)
|
||||
|
||||
// Try loading the cache file contents for a different indexDB.
|
||||
genOther := genCurr + 1
|
||||
gotNextDayMetricIDs := s.mustLoadNextDayMetricIDs(genOther, date)
|
||||
assertNextDayMetricIDs(t, gotNextDayMetricIDs, genOther, date, 0)
|
||||
|
||||
// Try loading the cache file contents for a different date.
|
||||
dateOther := date + 1
|
||||
gotNextDayMetricIDs = s.mustLoadNextDayMetricIDs(genCurr, dateOther)
|
||||
assertNextDayMetricIDs(t, gotNextDayMetricIDs, genCurr, dateOther, 0)
|
||||
assertNextDayMetricIDs(t, s.nextDayMetricIDs.Load(), idbID, date, numNextDayMetricIDs)
|
||||
s.MustClose()
|
||||
|
||||
// Advance the time by one day and open the storage.
|
||||
// Since the current date and the date in the cache file do not match,
|
||||
// nothing will be loaded into cache.
|
||||
time.Sleep(24 * time.Hour)
|
||||
date = uint64(time.Now().UnixMilli()) / msecPerDay
|
||||
s = MustOpenStorage(t.Name(), OpenOptions{})
|
||||
if got := s.pendingNextDayMetricIDs.Len(); got != 0 {
|
||||
t.Fatalf("unexpected pendingNextDayMetricIDs count: got %d, want 0", got)
|
||||
}
|
||||
assertNextDayMetricIDs(t, s.nextDayMetricIDs.Load(), idbID, date, 0)
|
||||
s.MustClose()
|
||||
})
|
||||
}
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -273,7 +273,6 @@ func BenchmarkStorageInsertWithAndWithoutPerDayIndex(b *testing.B) {
|
||||
var (
|
||||
rowsAddedTotal int
|
||||
dataSize int64
|
||||
indexSize int64
|
||||
)
|
||||
|
||||
path := b.Name()
|
||||
@@ -295,7 +294,6 @@ func BenchmarkStorageInsertWithAndWithoutPerDayIndex(b *testing.B) {
|
||||
|
||||
rowsAddedTotal = numBatches * numRowsPerBatch
|
||||
dataSize = benchmarkDirSize(path + "/data")
|
||||
indexSize = benchmarkDirSize(path + "/indexdb")
|
||||
|
||||
s.MustClose()
|
||||
fs.MustRemoveDir(path)
|
||||
@@ -303,7 +301,6 @@ func BenchmarkStorageInsertWithAndWithoutPerDayIndex(b *testing.B) {
|
||||
|
||||
b.ReportMetric(float64(rowsAddedTotal)/float64(b.Elapsed().Seconds()), "rows/s")
|
||||
b.ReportMetric(float64(dataSize)/(1024*1024), "data-MiB")
|
||||
b.ReportMetric(float64(indexSize)/(1024*1024), "indexdb-MiB")
|
||||
}
|
||||
|
||||
b.Run("HighChurnRate/perDayIndexes", func(b *testing.B) {
|
||||
@@ -392,7 +389,7 @@ type dataConfig struct {
|
||||
type searchFunc func(b *testing.B, s *Storage, tr TimeRange, mrs []MetricRow)
|
||||
|
||||
// splitFunc split the test data between prev and curr indexDBs.
|
||||
type splitFunc func(total dataConfig) (prev, curr dataConfig)
|
||||
type splitFunc func(total dataConfig) (prev, curr, pt dataConfig)
|
||||
|
||||
// benchmarkSearch implements the core logic of benchmark of a search operation.
|
||||
//
|
||||
@@ -455,11 +452,13 @@ func benchmarkSearch(b *testing.B, dataConfig dataConfig, split splitFunc, searc
|
||||
|
||||
}
|
||||
|
||||
cfgPrev, cfgCurr := split(dataConfig)
|
||||
cfgPrev, cfgCurr, cfgPt := split(dataConfig)
|
||||
mrsToDeletePrev := genRows(cfgPrev.numDeletedSeries, "prev", cfgPrev.tr)
|
||||
mrsToDeleteCurr := genRows(cfgCurr.numDeletedSeries, "curr", cfgCurr.tr)
|
||||
mrsToDeletePt := genRows(cfgPt.numDeletedSeries, "pt", cfgPt.tr)
|
||||
mrsPrev := genRows(cfgPrev.numSeries, "prev", cfgPrev.tr)
|
||||
mrsCurr := genRows(cfgCurr.numSeries, "curr", cfgCurr.tr)
|
||||
mrsPt := genRows(cfgPt.numSeries, "pt", cfgPt.tr)
|
||||
|
||||
s := MustOpenStorage(b.Name(), OpenOptions{})
|
||||
s.AddRows(mrsToDeletePrev, defaultPrecisionBits)
|
||||
@@ -468,8 +467,7 @@ func benchmarkSearch(b *testing.B, dataConfig dataConfig, split splitFunc, searc
|
||||
s.DebugFlush()
|
||||
s.AddRows(mrsPrev, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
|
||||
s.mustRotateIndexDB(time.Now())
|
||||
s = mustConvertToLegacy(s)
|
||||
|
||||
s.AddRows(mrsToDeleteCurr, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
@@ -477,8 +475,16 @@ func benchmarkSearch(b *testing.B, dataConfig dataConfig, split splitFunc, searc
|
||||
s.DebugFlush()
|
||||
s.AddRows(mrsCurr, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
s = mustConvertToLegacy(s)
|
||||
|
||||
mrs := slices.Concat(mrsPrev, mrsCurr)
|
||||
s.AddRows(mrsToDeletePt, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
deleteSeries(s, "pt", cfgPt.numDeletedSeries)
|
||||
s.DebugFlush()
|
||||
s.AddRows(mrsPt, defaultPrecisionBits)
|
||||
s.DebugFlush()
|
||||
|
||||
mrs := slices.Concat(mrsPrev, mrsCurr, mrsPt)
|
||||
search(b, s, dataConfig.tr, mrs)
|
||||
|
||||
s.MustClose()
|
||||
@@ -691,36 +697,38 @@ func isGraphite(op searchFunc) bool {
|
||||
|
||||
// indexConfigs holds the index configurations for which BenchmarkSearch() will
|
||||
// perform the measurements.
|
||||
var indexConfigs = []splitFunc{prevOnly, currOnly, prevCurr}
|
||||
var indexConfigNames = []string{"PrevOnly", "CurrOnly", "PrevCurr"}
|
||||
var indexConfigs = []splitFunc{prevOnly, currOnly, prevCurr, ptOnly, prevPt, currPt, prevCurrPt}
|
||||
var indexConfigNames = []string{"PrevOnly", "CurrOnly", "PrevCurr", "PtOnly", "PrevPt", "CurrPt", "PrevCurrPt"}
|
||||
|
||||
// prevOnly is an index config func that puts all index data into prev indexDB.
|
||||
// No index data goes to curr indexDB.
|
||||
// prevOnly is an index config func that puts all index data into legacy prev
|
||||
// indexDB. No index data goes to legacy curr indexDB or pt indexDBs.
|
||||
//
|
||||
// This config corresponds to a state when indexDBs have just been rotated.
|
||||
// I.e. most of the index entries are in the prev indexDB.
|
||||
func prevOnly(total dataConfig) (prev, curr dataConfig) {
|
||||
// This config corresponds to a state when the deployment has switched to pt
|
||||
// index right after legacy indexDBs have just been rotated. I.e. most of the
|
||||
// index entries are in the prev indexDB.
|
||||
func prevOnly(total dataConfig) (prev, curr, pt dataConfig) {
|
||||
prev = total
|
||||
return prev, curr
|
||||
return prev, curr, pt
|
||||
}
|
||||
|
||||
// currOnly is an index config func that puts all index data into curr
|
||||
// indexDB. No index data goes to prev indexDB.
|
||||
// currOnly is an index config func that puts all index data into legacy curr
|
||||
// indexDB. No index data goes to legacy prev indexDB or pt indexDBs.
|
||||
//
|
||||
// This config corresponds to a state when indexDBs haven't been rotated yet or
|
||||
// rotated long time ago. I.e. most of the index entries are in the curr
|
||||
// indexDB.
|
||||
func currOnly(total dataConfig) (prev, curr dataConfig) {
|
||||
// This config corresponds to a state when the deployment has switched to pt
|
||||
// index before legacy indexDB rotation or the rotation has happened long time
|
||||
// ago. I.e. most of the index entries are in the curr indexDB.
|
||||
func currOnly(total dataConfig) (prev, curr, pt dataConfig) {
|
||||
curr = total
|
||||
return prev, curr
|
||||
return prev, curr, pt
|
||||
}
|
||||
|
||||
// prevCurr is an index config func that splits index data evenly between
|
||||
// prev and curr indexDBs.
|
||||
// prev and curr legacy indexDBs. No data goes to pt indexDBs.
|
||||
//
|
||||
// This config corresponds to a state when the indexDB rotation has happened
|
||||
// some time ago. I.e. index entries are in both prev and curr indexDBs.
|
||||
func prevCurr(total dataConfig) (prev, curr dataConfig) {
|
||||
// This config corresponds to a state when the the deployment has switched to pt
|
||||
// index some significant time after legacy indexDB rotation. I.e. index entries
|
||||
// are in both prev and curr legacy indexDBs.
|
||||
func prevCurr(total dataConfig) (prev, curr, pt dataConfig) {
|
||||
prev.numSeries = total.numSeries / 2
|
||||
prev.numDeletedSeries = total.numDeletedSeries / 2
|
||||
prev.tr.MinTimestamp = total.tr.MinTimestamp
|
||||
@@ -731,7 +739,82 @@ func prevCurr(total dataConfig) (prev, curr dataConfig) {
|
||||
curr.tr.MinTimestamp = prev.tr.MaxTimestamp + 1
|
||||
curr.tr.MaxTimestamp = total.tr.MaxTimestamp
|
||||
|
||||
return prev, curr
|
||||
return prev, curr, pt
|
||||
}
|
||||
|
||||
// ptOnly is an index config func that puts all index data into pt indexDBs. No
|
||||
// index data goes to prev or curr legacy indexDBs.
|
||||
//
|
||||
// This config corresponds to a state when a fresh deployment has started with
|
||||
// pt index right away.I.e. all the index entries are in the pt indexDBs.
|
||||
func ptOnly(total dataConfig) (prev, curr, pt dataConfig) {
|
||||
pt = total
|
||||
return prev, curr, pt
|
||||
}
|
||||
|
||||
// prevPt is an index config func that splits index data evenly between
|
||||
// prev legacy indexDB and pt indexDBs. No data goes to curr legacy indexDB.
|
||||
//
|
||||
// This config corresponds to a state when the the deployment has switched to pt
|
||||
// index right after legacy indexDB rotation and continued to work for some
|
||||
// time.
|
||||
func prevPt(total dataConfig) (prev, curr, pt dataConfig) {
|
||||
prev.numSeries = total.numSeries / 2
|
||||
prev.numDeletedSeries = total.numDeletedSeries / 2
|
||||
prev.tr.MinTimestamp = total.tr.MinTimestamp
|
||||
prev.tr.MaxTimestamp = total.tr.MinTimestamp + (total.tr.MaxTimestamp-total.tr.MinTimestamp)/2
|
||||
|
||||
pt.numSeries = total.numSeries - prev.numSeries
|
||||
pt.numDeletedSeries = total.numDeletedSeries - prev.numDeletedSeries
|
||||
pt.tr.MinTimestamp = prev.tr.MaxTimestamp + 1
|
||||
pt.tr.MaxTimestamp = total.tr.MaxTimestamp
|
||||
|
||||
return prev, curr, pt
|
||||
}
|
||||
|
||||
// currPt is an index config func that splits index data evenly between
|
||||
// curr legacy indexDB and pt indexDBs. No data goes to prev legacy indexDB.
|
||||
//
|
||||
// This config corresponds to a state when the the deployment has switched to pt
|
||||
// index right before legacy indexDB rotation and continued to work for some
|
||||
// time.
|
||||
func currPt(total dataConfig) (prev, curr, pt dataConfig) {
|
||||
curr.numSeries = total.numSeries / 2
|
||||
curr.numDeletedSeries = total.numDeletedSeries / 2
|
||||
curr.tr.MinTimestamp = total.tr.MinTimestamp
|
||||
curr.tr.MaxTimestamp = total.tr.MinTimestamp + (total.tr.MaxTimestamp-total.tr.MinTimestamp)/2
|
||||
|
||||
pt.numSeries = total.numSeries - curr.numSeries
|
||||
pt.numDeletedSeries = total.numDeletedSeries - curr.numDeletedSeries
|
||||
pt.tr.MinTimestamp = curr.tr.MaxTimestamp + 1
|
||||
pt.tr.MaxTimestamp = total.tr.MaxTimestamp
|
||||
|
||||
return prev, curr, pt
|
||||
}
|
||||
|
||||
// prevCurrPt is an index config func that splits index data evenly between
|
||||
// prev and curr legacy indexDBs and pt indexDBs.
|
||||
//
|
||||
// This config corresponds to a state when the the deployment has switched to pt
|
||||
// index right some time after legacy indexDB rotation and continued to work for
|
||||
// some time.
|
||||
func prevCurrPt(total dataConfig) (prev, curr, pt dataConfig) {
|
||||
prev.numSeries = total.numSeries / 3
|
||||
prev.numDeletedSeries = total.numDeletedSeries / 3
|
||||
prev.tr.MinTimestamp = total.tr.MinTimestamp
|
||||
prev.tr.MaxTimestamp = total.tr.MinTimestamp + (total.tr.MaxTimestamp-total.tr.MinTimestamp)/3
|
||||
|
||||
curr.numSeries = prev.numSeries
|
||||
curr.numDeletedSeries = prev.numDeletedSeries
|
||||
curr.tr.MinTimestamp = prev.tr.MaxTimestamp + 1
|
||||
curr.tr.MaxTimestamp = prev.tr.MaxTimestamp + (total.tr.MaxTimestamp-total.tr.MinTimestamp)/3
|
||||
|
||||
pt.numSeries = total.numSeries - prev.numSeries - curr.numSeries
|
||||
pt.numDeletedSeries = total.numDeletedSeries - prev.numDeletedSeries - curr.numDeletedSeries
|
||||
pt.tr.MinTimestamp = curr.tr.MaxTimestamp + 1
|
||||
pt.tr.MaxTimestamp = total.tr.MaxTimestamp
|
||||
|
||||
return prev, curr, pt
|
||||
}
|
||||
|
||||
// dataConfigFunc generates a collection of data configs. For example, various
|
||||
|
||||
@@ -28,7 +28,9 @@ type table struct {
|
||||
path string
|
||||
smallPartitionsPath string
|
||||
bigPartitionsPath string
|
||||
indexDBPath string
|
||||
|
||||
// TODO(@rtm0): Do not depend on Storage.
|
||||
s *Storage
|
||||
|
||||
ptws []*partitionWrapper
|
||||
@@ -105,8 +107,11 @@ func mustOpenTable(path string, s *Storage) *table {
|
||||
bigSnapshotsPath := filepath.Join(bigPartitionsPath, snapshotsDirname)
|
||||
fs.MustMkdirIfNotExist(bigSnapshotsPath)
|
||||
|
||||
indexDBPath := filepath.Join(path, indexdbDirname)
|
||||
fs.MustMkdirIfNotExist(indexDBPath)
|
||||
|
||||
// Open partitions.
|
||||
pts := mustOpenPartitions(smallPartitionsPath, bigPartitionsPath, s)
|
||||
pts := mustOpenPartitions(smallPartitionsPath, bigPartitionsPath, indexDBPath, s)
|
||||
|
||||
// Make sure all the directories inside the path are properly synced.
|
||||
fs.MustSyncPathAndParentDir(path)
|
||||
@@ -115,6 +120,7 @@ func mustOpenTable(path string, s *Storage) *table {
|
||||
path: path,
|
||||
smallPartitionsPath: smallPartitionsPath,
|
||||
bigPartitionsPath: bigPartitionsPath,
|
||||
indexDBPath: indexDBPath,
|
||||
s: s,
|
||||
|
||||
stopCh: make(chan struct{}),
|
||||
@@ -127,12 +133,13 @@ func mustOpenTable(path string, s *Storage) *table {
|
||||
return tb
|
||||
}
|
||||
|
||||
// MustCreateSnapshot creates tb snapshot and returns paths to small and big parts of it.
|
||||
func (tb *table) MustCreateSnapshot(snapshotName string) (string, string) {
|
||||
// MustCreateSnapshot creates tb snapshot and returns paths to small parts, big
|
||||
// parts, and indexdb.
|
||||
func (tb *table) MustCreateSnapshot(snapshotName string) (string, string, string) {
|
||||
logger.Infof("creating table snapshot of %q...", tb.path)
|
||||
startTime := time.Now()
|
||||
|
||||
ptws := tb.GetPartitions(nil)
|
||||
ptws := tb.GetAllPartitions(nil)
|
||||
defer tb.PutPartitions(ptws)
|
||||
|
||||
dstSmallDir := filepath.Join(tb.path, smallDirname, snapshotsDirname, snapshotName)
|
||||
@@ -141,17 +148,22 @@ func (tb *table) MustCreateSnapshot(snapshotName string) (string, string) {
|
||||
dstBigDir := filepath.Join(tb.path, bigDirname, snapshotsDirname, snapshotName)
|
||||
fs.MustMkdirFailIfExist(dstBigDir)
|
||||
|
||||
dstIndexDBDir := filepath.Join(tb.path, indexdbDirname, snapshotsDirname, snapshotName)
|
||||
fs.MustMkdirFailIfExist(dstIndexDBDir)
|
||||
|
||||
for _, ptw := range ptws {
|
||||
smallPath := filepath.Join(dstSmallDir, ptw.pt.name)
|
||||
bigPath := filepath.Join(dstBigDir, ptw.pt.name)
|
||||
ptw.pt.MustCreateSnapshotAt(smallPath, bigPath)
|
||||
indexDBPath := filepath.Join(dstIndexDBDir, ptw.pt.name)
|
||||
ptw.pt.MustCreateSnapshotAt(smallPath, bigPath, indexDBPath)
|
||||
}
|
||||
|
||||
fs.MustSyncPathAndParentDir(dstSmallDir)
|
||||
fs.MustSyncPathAndParentDir(dstBigDir)
|
||||
fs.MustSyncPathAndParentDir(dstIndexDBDir)
|
||||
|
||||
logger.Infof("created table snapshot for %q at (%q, %q) in %.3f seconds", tb.path, dstSmallDir, dstBigDir, time.Since(startTime).Seconds())
|
||||
return dstSmallDir, dstBigDir
|
||||
logger.Infof("created table snapshot for %q at (%q, %q, %q) in %.3f seconds", tb.path, dstSmallDir, dstBigDir, dstIndexDBDir, time.Since(startTime).Seconds())
|
||||
return dstSmallDir, dstBigDir, dstIndexDBDir
|
||||
}
|
||||
|
||||
// MustDeleteSnapshot deletes snapshot with the given snapshotName.
|
||||
@@ -160,14 +172,24 @@ func (tb *table) MustDeleteSnapshot(snapshotName string) {
|
||||
fs.MustRemoveDir(smallDir)
|
||||
bigDir := filepath.Join(tb.path, bigDirname, snapshotsDirname, snapshotName)
|
||||
fs.MustRemoveDir(bigDir)
|
||||
indexDBDir := filepath.Join(tb.path, indexdbDirname, snapshotsDirname, snapshotName)
|
||||
fs.MustRemoveDir(indexDBDir)
|
||||
}
|
||||
|
||||
func (tb *table) addPartitionLocked(pt *partition) {
|
||||
_ = tb.addPartitionWrapperLocked(pt)
|
||||
// It is expected that the caller of this method will eventually decrement
|
||||
// the pt refCount.
|
||||
}
|
||||
|
||||
func (tb *table) addPartitionWrapperLocked(pt *partition) *partitionWrapper {
|
||||
ptw := &partitionWrapper{
|
||||
pt: pt,
|
||||
}
|
||||
ptw.incRef()
|
||||
|
||||
tb.ptws = append(tb.ptws, ptw)
|
||||
return ptw
|
||||
}
|
||||
|
||||
// MustClose closes the table.
|
||||
@@ -194,12 +216,12 @@ func (tb *table) MustClose() {
|
||||
}
|
||||
}
|
||||
|
||||
// DebugFlush flushes all pending raw data rows, so they become
|
||||
// DebugFlush flushes all pending raw index and data rows, so they become
|
||||
// visible to search.
|
||||
//
|
||||
// This function is for debug purposes only.
|
||||
func (tb *table) DebugFlush() {
|
||||
ptws := tb.GetPartitions(nil)
|
||||
ptws := tb.GetAllPartitions(nil)
|
||||
defer tb.PutPartitions(ptws)
|
||||
|
||||
for _, ptw := range ptws {
|
||||
@@ -229,7 +251,7 @@ type TableMetrics struct {
|
||||
|
||||
// UpdateMetrics updates m with metrics from tb.
|
||||
func (tb *table) UpdateMetrics(m *TableMetrics) {
|
||||
ptws := tb.GetPartitions(nil)
|
||||
ptws := tb.GetAllPartitions(nil)
|
||||
defer tb.PutPartitions(ptws)
|
||||
|
||||
for _, ptw := range ptws {
|
||||
@@ -253,7 +275,7 @@ func (tb *table) UpdateMetrics(m *TableMetrics) {
|
||||
//
|
||||
// Partitions are merged sequentially in order to reduce load on the system.
|
||||
func (tb *table) ForceMergePartitions(partitionNamePrefix string) error {
|
||||
ptws := tb.GetPartitions(nil)
|
||||
ptws := tb.GetAllPartitions(nil)
|
||||
defer tb.PutPartitions(ptws)
|
||||
|
||||
tb.forceMergeWG.Add(1)
|
||||
@@ -283,7 +305,7 @@ func (tb *table) MustAddRows(rows []rawRow) {
|
||||
ptwsX := getPartitionWrappers()
|
||||
defer putPartitionWrappers(ptwsX)
|
||||
|
||||
ptwsX.a = tb.GetPartitions(ptwsX.a[:0])
|
||||
ptwsX.a = tb.GetAllPartitions(ptwsX.a[:0])
|
||||
ptws := ptwsX.a
|
||||
for i, ptw := range ptws {
|
||||
singlePt := true
|
||||
@@ -368,13 +390,22 @@ func (tb *table) MustAddRows(rows []rawRow) {
|
||||
continue
|
||||
}
|
||||
|
||||
pt := mustCreatePartition(r.Timestamp, tb.smallPartitionsPath, tb.bigPartitionsPath, tb.s)
|
||||
pt := mustCreatePartition(r.Timestamp, tb.smallPartitionsPath, tb.bigPartitionsPath, tb.indexDBPath, tb.s)
|
||||
pt.AddRows(missingRows[i : i+1])
|
||||
tb.addPartitionLocked(pt)
|
||||
}
|
||||
tb.ptwsLock.Unlock()
|
||||
}
|
||||
|
||||
// MustGetIndexDBIDByHour returns the id of the indexDB which contains the
|
||||
// provided hour. If the indexDB does not exist it will be created.
|
||||
func (tb *table) MustGetIndexDBIDByHour(hour uint64) uint64 {
|
||||
ts := int64(hour * msecPerHour)
|
||||
ptw := tb.MustGetPartition(ts)
|
||||
defer tb.PutPartition(ptw)
|
||||
return ptw.pt.idb.id
|
||||
}
|
||||
|
||||
func (tb *table) getMinMaxTimestamps() (int64, int64) {
|
||||
now := int64(fasttime.UnixTimestamp() * 1000)
|
||||
minTimestamp := now - tb.s.retentionMsecs
|
||||
@@ -452,7 +483,7 @@ func (tb *table) historicalMergeWatcher() {
|
||||
}
|
||||
|
||||
f := func() {
|
||||
ptws := tb.GetPartitions(nil)
|
||||
ptws := tb.GetAllPartitions(nil)
|
||||
defer tb.PutPartitions(ptws)
|
||||
timestamp := timestampFromTime(time.Now())
|
||||
currentPartitionName := timestampToPartitionName(timestamp)
|
||||
@@ -519,11 +550,54 @@ func (tb *table) historicalMergeWatcher() {
|
||||
}
|
||||
}
|
||||
|
||||
// GetPartitions appends tb's partitions snapshot to dst and returns the result.
|
||||
// MustGetPartition returns a partition that corresponds to the given timestamp.
|
||||
//
|
||||
// If the partition does not exist yet, it will be created.
|
||||
//
|
||||
// The function increments the ref counter for the found partition.
|
||||
// The returned partition must be passed to PutPartition when no longer needed.
|
||||
func (tb *table) MustGetPartition(timestamp int64) *partitionWrapper {
|
||||
tb.ptwsLock.Lock()
|
||||
defer tb.ptwsLock.Unlock()
|
||||
|
||||
ptw := tb.getPartitionLocked(timestamp)
|
||||
if ptw != nil {
|
||||
return ptw
|
||||
}
|
||||
|
||||
pt := mustCreatePartition(timestamp, tb.smallPartitionsPath, tb.bigPartitionsPath, tb.indexDBPath, tb.s)
|
||||
ptw = tb.addPartitionWrapperLocked(pt)
|
||||
ptw.incRef()
|
||||
return ptw
|
||||
}
|
||||
|
||||
// GetPartition returns a partition that corresponds to the given timestamp or
|
||||
// nil if such partition does not exist.
|
||||
//
|
||||
// If the partition is found, the function increments its ref counter. When no
|
||||
// longer needed, the returned partition must be passed to PutPartition to
|
||||
// decrement its ref counter.
|
||||
func (tb *table) GetPartition(timestamp int64) *partitionWrapper {
|
||||
tb.ptwsLock.Lock()
|
||||
defer tb.ptwsLock.Unlock()
|
||||
return tb.getPartitionLocked(timestamp)
|
||||
}
|
||||
|
||||
func (tb *table) getPartitionLocked(timestamp int64) *partitionWrapper {
|
||||
for _, ptw := range tb.ptws {
|
||||
if ptw.pt.HasTimestamp(timestamp) {
|
||||
ptw.incRef()
|
||||
return ptw
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetAllPartitions appends tb's partitions snapshot to dst and returns the result.
|
||||
//
|
||||
// The returned partitions must be passed to PutPartitions
|
||||
// when they no longer needed.
|
||||
func (tb *table) GetPartitions(dst []*partitionWrapper) []*partitionWrapper {
|
||||
func (tb *table) GetAllPartitions(dst []*partitionWrapper) []*partitionWrapper {
|
||||
tb.ptwsLock.Lock()
|
||||
for _, ptw := range tb.ptws {
|
||||
ptw.incRef()
|
||||
@@ -534,19 +608,46 @@ func (tb *table) GetPartitions(dst []*partitionWrapper) []*partitionWrapper {
|
||||
return dst
|
||||
}
|
||||
|
||||
// PutPartitions deregisters ptws obtained via GetPartitions.
|
||||
// GetPartitions returns snapshot of partitions whose time ranges overlap with the
|
||||
// given time range.
|
||||
//
|
||||
// The returned partitions must be passed to PutPartitions
|
||||
// when they no longer needed.
|
||||
func (tb *table) GetPartitions(tr TimeRange) []*partitionWrapper {
|
||||
tb.ptwsLock.Lock()
|
||||
defer tb.ptwsLock.Unlock()
|
||||
|
||||
var ptws []*partitionWrapper
|
||||
|
||||
for _, ptw := range tb.ptws {
|
||||
if ptw.pt.tr.overlapsWith(tr) {
|
||||
ptw.incRef()
|
||||
ptws = append(ptws, ptw)
|
||||
}
|
||||
}
|
||||
|
||||
return ptws
|
||||
}
|
||||
|
||||
// PutPartition decrements the ref counter for the given partition.
|
||||
func (tb *table) PutPartition(ptw *partitionWrapper) {
|
||||
ptw.decRef()
|
||||
}
|
||||
|
||||
// PutPartitions deregisters ptws obtained via GetAllPartitions or GetPartitions.
|
||||
func (tb *table) PutPartitions(ptws []*partitionWrapper) {
|
||||
for _, ptw := range ptws {
|
||||
ptw.decRef()
|
||||
tb.PutPartition(ptw)
|
||||
}
|
||||
}
|
||||
|
||||
func mustOpenPartitions(smallPartitionsPath, bigPartitionsPath string, s *Storage) []*partition {
|
||||
func mustOpenPartitions(smallPartitionsPath, bigPartitionsPath, indexDBPath string, s *Storage) []*partition {
|
||||
// Certain partition directories in either `big` or `small` dir may be missing
|
||||
// after restoring from backup. So populate partition names from both dirs.
|
||||
ptNames := make(map[string]bool)
|
||||
mustPopulatePartitionNames(smallPartitionsPath, ptNames)
|
||||
mustPopulatePartitionNames(bigPartitionsPath, ptNames)
|
||||
mustPopulatePartitionNames(indexDBPath, ptNames)
|
||||
var pts []*partition
|
||||
var ptsLock sync.Mutex
|
||||
|
||||
@@ -564,7 +665,8 @@ func mustOpenPartitions(smallPartitionsPath, bigPartitionsPath string, s *Storag
|
||||
|
||||
smallPartsPath := filepath.Join(smallPartitionsPath, ptName)
|
||||
bigPartsPath := filepath.Join(bigPartitionsPath, ptName)
|
||||
pt := mustOpenPartition(smallPartsPath, bigPartsPath, s)
|
||||
indexDBPartsPath := filepath.Join(indexDBPath, ptName)
|
||||
pt := mustOpenPartition(smallPartsPath, bigPartsPath, indexDBPartsPath, s)
|
||||
|
||||
ptsLock.Lock()
|
||||
pts = append(pts, pt)
|
||||
|
||||
@@ -82,7 +82,7 @@ func (ts *tableSearch) Init(tb *table, tsids []TSID, tr TimeRange) {
|
||||
return
|
||||
}
|
||||
|
||||
ts.ptws = tb.GetPartitions(ts.ptws[:0])
|
||||
ts.ptws = tb.GetAllPartitions(ts.ptws[:0])
|
||||
|
||||
// Initialize the ptsPool.
|
||||
ts.ptsPool = slicesutil.SetLength(ts.ptsPool, len(ts.ptws))
|
||||
|
||||
@@ -1,7 +1,9 @@
|
||||
package storage
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
|
||||
)
|
||||
@@ -29,3 +31,57 @@ func TestTableOpenClose(t *testing.T) {
|
||||
|
||||
stopTestStorage(strg)
|
||||
}
|
||||
|
||||
func TestGetPartition(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
s := MustOpenStorage(t.Name(), OpenOptions{})
|
||||
defer s.MustClose()
|
||||
|
||||
var ptw *partitionWrapper
|
||||
timestamp := time.Date(2000, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
|
||||
ptw = s.tb.GetPartition(timestamp)
|
||||
if ptw != nil {
|
||||
name := ptw.pt.name
|
||||
s.tb.PutPartition(ptw)
|
||||
t.Fatalf("GetPartition() unexpectedly returned a partition that should not exist: %s", name)
|
||||
}
|
||||
|
||||
ptw = s.tb.MustGetPartition(timestamp)
|
||||
if ptw == nil {
|
||||
t.Fatalf("MustGetPartition() unexpectedly did not create a new partition")
|
||||
}
|
||||
s.tb.PutPartition(ptw)
|
||||
|
||||
ptw = s.tb.GetPartition(timestamp)
|
||||
if ptw == nil {
|
||||
t.Fatalf("GetPartition() unexpectedly did not find partition")
|
||||
}
|
||||
s.tb.PutPartition(ptw)
|
||||
}
|
||||
|
||||
func TestGetPartition_concurrent(t *testing.T) {
|
||||
defer testRemoveAll(t)
|
||||
|
||||
s := MustOpenStorage(t.Name(), OpenOptions{})
|
||||
defer s.MustClose()
|
||||
|
||||
begin := time.Date(2024, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
limit := time.Date(2025, 1, 1, 0, 0, 0, 0, time.UTC).UnixMilli()
|
||||
for ts := begin; ts < limit; ts += msecPerDay {
|
||||
var wg sync.WaitGroup
|
||||
for range 100 {
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
ptw := s.tb.MustGetPartition(ts)
|
||||
s.tb.PutPartition(ptw)
|
||||
|
||||
ptw = s.tb.GetPartition(ts)
|
||||
s.tb.PutPartition(ptw)
|
||||
wg.Done()
|
||||
}()
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user