Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Replay WAL concurrently without blocking #10973

Merged
merged 2 commits into from
Aug 17, 2022

Conversation

damnever
Copy link
Contributor

@damnever damnever commented Jul 4, 2022

The WAL replay is extremely slow maybe because we have tons of series, and I have found the time.Sleep comes up multiple times. After some learning, I think we better put the WAL process logic in one place for the sake of speed, even if we need a little more memory to get the task done.

@damnever damnever requested a review from codesome as a code owner July 4, 2022 15:49
@damnever damnever force-pushed the fix/replay-walseries-concurrently branch from 2c530ae to e6e47e8 Compare July 5, 2022 00:58
Copy link
Member

@codesome codesome left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is actually a great idea. I am not sure why we did not do this before. I will give this a review in coming days to verify if we are missing something. Could you also try and benchmark this?

tsdb/head_wal.go Outdated Show resolved Hide resolved
@jesusvazquez
Copy link
Member

@damnever thanks for this, we actually saw the same issue with time.Sleep and replays taking too long. We tried to optimize the existing code removing the lock, then we realized we only needed to wait if the series had already been created so we were going to optimize that bit.

			// There's already a different ref for this series.
			// A duplicate series record is only possible when the old samples were already compacted into a block.
			// Hence we can discard all the samples and m-mapped chunks replayed till now for this series.

However your idea is even better.

I'm sure the benchmark will show great results, you can use this benchmark here

func BenchmarkLoadWAL(b *testing.B) {
cases := []struct {
// Total series is (batches*seriesPerBatch).
batches int
seriesPerBatch int
samplesPerSeries int
mmappedChunkT int64
}{
{ // Less series and more samples. 2 hour WAL with 1 second scrape interval.
batches: 10,
seriesPerBatch: 100,
samplesPerSeries: 7200,
},
{ // More series and less samples.
batches: 10,
seriesPerBatch: 10000,
samplesPerSeries: 50,
},
{ // In between.
batches: 10,
seriesPerBatch: 1000,
samplesPerSeries: 480,
},
{ // 2 hour WAL with 15 second scrape interval, and mmapped chunks up to last 100 samples.
batches: 100,
seriesPerBatch: 1000,
samplesPerSeries: 480,
mmappedChunkT: 3800,
},
}
labelsPerSeries := 5
// Rough estimates of most common % of samples that have an exemplar for each scrape.
exemplarsPercentages := []float64{0, 0.5, 1, 5}
lastExemplarsPerSeries := -1
for _, c := range cases {
for _, p := range exemplarsPercentages {
exemplarsPerSeries := int(math.RoundToEven(float64(c.samplesPerSeries) * p / 100))
// For tests with low samplesPerSeries we could end up testing with 0 exemplarsPerSeries
// multiple times without this check.
if exemplarsPerSeries == lastExemplarsPerSeries {
continue
}
lastExemplarsPerSeries = exemplarsPerSeries
// fmt.Println("exemplars per series: ", exemplarsPerSeries)
b.Run(fmt.Sprintf("batches=%d,seriesPerBatch=%d,samplesPerSeries=%d,exemplarsPerSeries=%d,mmappedChunkT=%d", c.batches, c.seriesPerBatch, c.samplesPerSeries, exemplarsPerSeries, c.mmappedChunkT),
func(b *testing.B) {
dir := b.TempDir()
w, err := wal.New(nil, nil, dir, false)
require.NoError(b, err)
// Write series.
refSeries := make([]record.RefSeries, 0, c.seriesPerBatch)
for k := 0; k < c.batches; k++ {
refSeries = refSeries[:0]
for i := k * c.seriesPerBatch; i < (k+1)*c.seriesPerBatch; i++ {
lbls := make(map[string]string, labelsPerSeries)
lbls[defaultLabelName] = strconv.Itoa(i)
for j := 1; len(lbls) < labelsPerSeries; j++ {
lbls[defaultLabelName+strconv.Itoa(j)] = defaultLabelValue + strconv.Itoa(j)
}
refSeries = append(refSeries, record.RefSeries{Ref: chunks.HeadSeriesRef(i) * 101, Labels: labels.FromMap(lbls)})
}
populateTestWAL(b, w, []interface{}{refSeries})
}
// Write samples.
refSamples := make([]record.RefSample, 0, c.seriesPerBatch)
for i := 0; i < c.samplesPerSeries; i++ {
for j := 0; j < c.batches; j++ {
refSamples = refSamples[:0]
for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
refSamples = append(refSamples, record.RefSample{
Ref: chunks.HeadSeriesRef(k) * 101,
T: int64(i) * 10,
V: float64(i) * 100,
})
}
populateTestWAL(b, w, []interface{}{refSamples})
}
}
// Write mmapped chunks.
if c.mmappedChunkT != 0 {
chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, mmappedChunksDir(dir), chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize)
require.NoError(b, err)
for k := 0; k < c.batches*c.seriesPerBatch; k++ {
// Create one mmapped chunk per series, with one sample at the given time.
s := newMemSeries(labels.Labels{}, chunks.HeadSeriesRef(k)*101, c.mmappedChunkT, nil, defaultIsolationDisabled)
s.append(c.mmappedChunkT, 42, 0, chunkDiskMapper)
s.mmapCurrentHeadChunk(chunkDiskMapper)
}
require.NoError(b, chunkDiskMapper.Close())
}
// Write exemplars.
refExemplars := make([]record.RefExemplar, 0, c.seriesPerBatch)
for i := 0; i < exemplarsPerSeries; i++ {
for j := 0; j < c.batches; j++ {
refExemplars = refExemplars[:0]
for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
refExemplars = append(refExemplars, record.RefExemplar{
Ref: chunks.HeadSeriesRef(k) * 101,
T: int64(i) * 10,
V: float64(i) * 100,
Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", i)),
})
}
populateTestWAL(b, w, []interface{}{refExemplars})
}
}
b.ResetTimer()
// Load the WAL.
for i := 0; i < b.N; i++ {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = w.Dir()
h, err := NewHead(nil, nil, w, opts, nil)
require.NoError(b, err)
h.Init(0)
}
b.StopTimer()
w.Close()
})
}
}
}

tsdb/head_wal.go Outdated Show resolved Hide resolved
@colega
Copy link
Contributor

colega commented Jul 5, 2022

This is definitely much cleaner now! My comments are optional and opinionated nitpicks.

Copy link
Member

@codesome codesome left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

One immediate thing: now we have to process tombstones and exemplars in the same processing function because they rely on the series being created first and should be in order.

@damnever
Copy link
Contributor Author

damnever commented Jul 5, 2022

@jesusvazquez I have run the benchmark test against the main branch, and interestingly there is no significant difference on my old mac. I may look into this later.

@damnever damnever force-pushed the fix/replay-walseries-concurrently branch from e6e47e8 to 99a1f9e Compare July 5, 2022 14:09
@jesusvazquez
Copy link
Member

jesusvazquez commented Jul 6, 2022

@damnever I actually ran the benchmark locally myself and these are the results I got

Platform:

goos: linux
goarch: amd64
pkg: github.com/prometheus/prometheus/tsdb
cpu: 11th Gen Intel(R) Core(TM) i7-1165G7 @ 2.80GHz
❯ benchstat current_main.txt damnnever-concurrent-wal-replay.txt
name                                                                                                     old time/op  new time/op  delta
LoadWAL/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=0,mmappedChunkT=0-8        396ms ± 4%   315ms ±22%  -20.42%  (p=0.008 n=5+5)
LoadWAL/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=36,mmappedChunkT=0-8       391ms ± 7%   376ms ± 2%   -3.93%  (p=0.016 n=5+5)
LoadWAL/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=72,mmappedChunkT=0-8       395ms ± 4%   381ms ± 6%     ~     (p=0.095 n=5+5)
LoadWAL/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=360,mmappedChunkT=0-8      463ms ±28%   421ms ± 3%     ~     (p=0.095 n=5+5)
LoadWAL/batches=10,seriesPerBatch=10000,samplesPerSeries=50,exemplarsPerSeries=0,mmappedChunkT=0-8        573ms ± 2%   511ms ± 2%  -10.76%  (p=0.008 n=5+5)
LoadWAL/batches=10,seriesPerBatch=10000,samplesPerSeries=50,exemplarsPerSeries=2,mmappedChunkT=0-8        615ms ± 3%   552ms ± 2%  -10.25%  (p=0.008 n=5+5)
LoadWAL/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=0,mmappedChunkT=0-8        210ms ± 2%   198ms ± 2%   -5.87%  (p=0.008 n=5+5)
LoadWAL/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=2,mmappedChunkT=0-8        214ms ± 4%   206ms ± 3%     ~     (p=0.056 n=5+5)
LoadWAL/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=5,mmappedChunkT=0-8        215ms ± 1%   205ms ± 1%   -4.25%  (p=0.008 n=5+5)
LoadWAL/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=24,mmappedChunkT=0-8       290ms ±27%   257ms ±10%     ~     (p=0.421 n=5+5)
LoadWAL/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=0,mmappedChunkT=3800-8    2.38s ± 6%   2.19s ± 2%   -8.15%  (p=0.008 n=5+5)
LoadWAL/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=2,mmappedChunkT=3800-8    2.95s ±62%   2.22s ± 4%  -24.92%  (p=0.016 n=5+5)
LoadWAL/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=5,mmappedChunkT=3800-8    2.42s ± 6%   2.28s ± 4%   -5.60%  (p=0.032 n=5+5)
LoadWAL/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=24,mmappedChunkT=3800-8   3.01s ±14%   2.62s ± 3%  -12.81%  (p=0.016 n=5+5)

We'd actually need to test this on a real world WAL since the WALs on these benchmarks are pretty small. The ratio of samples vs series on a real world WAL is much higher

Note: Originally I compared the benchmarks in the wrong order and it was saying that the changes were taking longer but it was my bad.

@damnever damnever force-pushed the fix/replay-walseries-concurrently branch from 99a1f9e to 3a6deda Compare July 6, 2022 08:34
@damnever
Copy link
Contributor Author

damnever commented Jul 6, 2022

The benchmark result looks better on my mac if I modify the test like this:

@@ -185,9 +185,13 @@ func BenchmarkLoadWAL(b *testing.B) {
 
 					// Write series.
 					refSeries := make([]record.RefSeries, 0, c.seriesPerBatch)
+					refSamples := make([]record.RefSample, 0, c.seriesPerBatch)
 					for k := 0; k < c.batches; k++ {
 						refSeries = refSeries[:0]
 						for i := k * c.seriesPerBatch; i < (k+1)*c.seriesPerBatch; i++ {
+							if i%2 == 0 {
+								continue
+							}
 							lbls := make(map[string]string, labelsPerSeries)
 							lbls[defaultLabelName] = strconv.Itoa(i)
 							for j := 1; len(lbls) < labelsPerSeries; j++ {
@@ -196,22 +200,48 @@ func BenchmarkLoadWAL(b *testing.B) {
 							refSeries = append(refSeries, record.RefSeries{Ref: chunks.HeadSeriesRef(i) * 101, Labels: labels.FromMap(lbls)})
 						}
 						populateTestWAL(b, w, []interface{}{refSeries})
-					}
+						// Write samples.
+						for i := k * c.seriesPerBatch; i < (k+1)*c.seriesPerBatch; i++ {
+							if i%2 == 0 {
+								continue
+							}
+							refSamples = refSamples[:0]
+							for j := 0; j < c.samplesPerSeries; j++ {
+								refSamples = append(refSamples, record.RefSample{
+									Ref: chunks.HeadSeriesRef(i) * 101,
+									T:   int64(j) * 10,
+									V:   float64(j) * 100,
+								})
 
-					// Write samples.
-					refSamples := make([]record.RefSample, 0, c.seriesPerBatch)
-					for i := 0; i < c.samplesPerSeries; i++ {
-						for j := 0; j < c.batches; j++ {
+							}
+							populateTestWAL(b, w, []interface{}{refSamples})
+						}
+
+						refSeries = refSeries[:0]
+						for i := k * c.seriesPerBatch; i < (k+1)*c.seriesPerBatch; i++ {
+							if i%2 != 0 {
+								continue
+							}
+							lbls := make(map[string]string, labelsPerSeries)
+							lbls[defaultLabelName] = strconv.Itoa(i)
+							for j := 1; len(lbls) < labelsPerSeries; j++ {
+								lbls[defaultLabelName+strconv.Itoa(j)] = defaultLabelValue + strconv.Itoa(j)
+							}
+							refSeries = append(refSeries, record.RefSeries{Ref: chunks.HeadSeriesRef(i) * 101, Labels: labels.FromMap(lbls)})
+
+							// Write samples.
 							refSamples = refSamples[:0]
-							for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
+							for j := 0; j < c.samplesPerSeries; j++ {
 								refSamples = append(refSamples, record.RefSample{
-									Ref: chunks.HeadSeriesRef(k) * 101,
-									T:   int64(i) * 10,
-									V:   float64(i) * 100,
+									Ref: chunks.HeadSeriesRef(i) * 101,
+									T:   int64(j) * 10,
+									V:   float64(j) * 100,
 								})
+
 							}
 							populateTestWAL(b, w, []interface{}{refSamples})
 						}
+						populateTestWAL(b, w, []interface{}{refSeries})
 					}
 
 					// Write mmapped chunks.
goos: darwin
goarch: amd64
name                                                                                                      old time/op    new time/op    delta
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=0,mmappedChunkT=0-8          162ms ± 2%     147ms ± 4%   -9.46%  (p=0.000 n=10+9)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=36,mmappedChunkT=0-8         166ms ± 4%     154ms ± 8%   -7.47%  (p=0.001 n=9+10)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=72,mmappedChunkT=0-8         170ms ± 3%     156ms ± 4%   -8.07%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=360,mmappedChunkT=0-8        209ms ± 2%     191ms ± 3%   -8.49%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=10000,samplesPerSeries=50,exemplarsPerSeries=0,mmappedChunkT=0-8          580ms ± 5%     528ms ± 4%   -8.88%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=10000,samplesPerSeries=50,exemplarsPerSeries=2,mmappedChunkT=0-8          629ms ± 6%     562ms ± 5%  -10.75%  (p=0.000 n=9+9)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=0,mmappedChunkT=0-8          159ms ± 4%     151ms ± 6%   -4.78%  (p=0.002 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=2,mmappedChunkT=0-8          162ms ± 3%     151ms ± 4%   -6.50%  (p=0.000 n=9+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=5,mmappedChunkT=0-8          165ms ± 4%     155ms ± 5%   -6.08%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=24,mmappedChunkT=0-8         191ms ± 3%     180ms ± 4%   -6.04%  (p=0.000 n=10+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=0,mmappedChunkT=3800-8      1.23s ± 5%     1.07s ± 3%  -12.34%  (p=0.000 n=10+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=2,mmappedChunkT=3800-8      1.28s ± 5%     1.14s ± 3%  -11.13%  (p=0.000 n=10+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=5,mmappedChunkT=3800-8      1.34s ±12%     1.16s ± 3%  -13.48%  (p=0.000 n=10+9)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=24,mmappedChunkT=3800-8     1.67s ± 4%     1.49s ± 3%  -10.98%  (p=0.000 n=9+9)

name                                                                                                      old alloc/op   new alloc/op   delta
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=0,mmappedChunkT=0-8          328MB ± 3%     284MB ± 5%  -13.64%  (p=0.000 n=10+9)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=36,mmappedChunkT=0-8         335MB ± 2%     287MB ± 8%  -14.40%  (p=0.000 n=9+9)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=72,mmappedChunkT=0-8         336MB ± 6%     294MB ± 5%  -12.38%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=360,mmappedChunkT=0-8        361MB ± 6%     301MB ± 3%  -16.79%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=10000,samplesPerSeries=50,exemplarsPerSeries=0,mmappedChunkT=0-8          183MB ± 2%     164MB ± 1%  -10.32%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=10000,samplesPerSeries=50,exemplarsPerSeries=2,mmappedChunkT=0-8          224MB ± 2%     203MB ± 2%   -9.39%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=0,mmappedChunkT=0-8         79.8MB ± 8%    44.3MB ± 2%  -44.45%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=2,mmappedChunkT=0-8         82.2MB ± 5%    47.8MB ± 4%  -41.83%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=5,mmappedChunkT=0-8         84.6MB ± 3%    50.4MB ± 4%  -40.46%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=24,mmappedChunkT=0-8        99.1MB ± 2%    64.7MB ± 3%  -34.70%  (p=0.000 n=9+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=0,mmappedChunkT=3800-8      798MB ±20%     290MB ± 7%  -63.63%  (p=0.000 n=10+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=2,mmappedChunkT=3800-8      790MB ±12%     302MB ±13%  -61.82%  (p=0.000 n=10+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=5,mmappedChunkT=3800-8      812MB ±20%     343MB ± 7%  -57.79%  (p=0.000 n=9+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=24,mmappedChunkT=3800-8     955MB ± 5%     476MB ± 4%  -50.19%  (p=0.000 n=8+9)

name                                                                                                      old allocs/op  new allocs/op  delta
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=0,mmappedChunkT=0-8           256k ± 0%      313k ± 2%  +22.30%  (p=0.000 n=10+9)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=36,mmappedChunkT=0-8          401k ± 0%      464k ± 0%  +15.83%  (p=0.000 n=9+7)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=72,mmappedChunkT=0-8          547k ± 0%      609k ± 0%  +11.49%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=100,samplesPerSeries=7200,exemplarsPerSeries=360,mmappedChunkT=0-8        1.71M ± 0%     1.75M ± 0%   +2.65%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=10000,samplesPerSeries=50,exemplarsPerSeries=0,mmappedChunkT=0-8          2.37M ± 0%     2.29M ± 0%   -3.60%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=10000,samplesPerSeries=50,exemplarsPerSeries=2,mmappedChunkT=0-8          3.17M ± 0%     3.09M ± 0%   -2.72%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=0,mmappedChunkT=0-8           380k ± 1%      357k ± 0%   -6.06%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=2,mmappedChunkT=0-8           460k ± 0%      437k ± 0%   -4.92%  (p=0.000 n=9+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=5,mmappedChunkT=0-8           580k ± 1%      557k ± 0%   -3.99%  (p=0.000 n=10+10)
LoadWALX/batches=10,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=24,mmappedChunkT=0-8         1.34M ± 0%     1.32M ± 0%   -1.69%  (p=0.000 n=9+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=0,mmappedChunkT=3800-8      3.27M ± 2%     2.96M ± 0%   -9.39%  (p=0.000 n=10+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=2,mmappedChunkT=3800-8      4.06M ± 1%     3.76M ± 0%   -7.33%  (p=0.000 n=10+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=5,mmappedChunkT=3800-8      5.27M ± 2%     4.97M ± 0%   -5.71%  (p=0.000 n=10+10)
LoadWALX/batches=100,seriesPerBatch=1000,samplesPerSeries=480,exemplarsPerSeries=24,mmappedChunkT=3800-8     12.9M ± 0%     12.6M ± 0%   -2.23%  (p=0.000 n=8+9)

@damnever damnever force-pushed the fix/replay-walseries-concurrently branch from 3a6deda to 2572856 Compare July 6, 2022 11:50
@roidelapluie
Copy link
Member

"we need a little more memory to get the task done" How much are we talking?

@jesusvazquez
Copy link
Member

You can run the benchmark with -benchmem flag to measure the differences in memory allocations and utilisation.

Just pointing that out because its something i learned recently 😂

@damnever
Copy link
Contributor Author

damnever commented Jul 7, 2022

"we need a little more memory to get the task done" How much are we talking?

@richardkiene I think there is no significant difference in memory consumption. I'd expect better memory utilization because I think the slice reuse ratio should be better, and the slice reuse leads to better CPU utilization.

I have posted some benchmark results in the comment. The reason why I modified the test is that I think it is more proper for this case (maybe real world case).

@damnever
Copy link
Contributor Author

damnever commented Jul 11, 2022

I have done some tests manually with more than 1 million series (7 checkpoint segments, 27 WAL segments). The steps are as follows:

  1. The workload definition: https://gist.github.com/damnever/a4648665737b2d0366cce6a9b255d1bc
  2. Deploy the workload.
  3. Prepare the data: kubectl cp -c prometheus-server load-wal-test/ namespace/prometheus-wal-test-0:/data/
  4. Repeat: delete the workload, edit prometheus image, deploy the workload.

Here are the results:

branch=[fix/replay-walseries-concurrently] branch=main revision=44fcf876caad9a5d28b92a728d2c98c34015d377
checkpoint_replay_duration=7.788286858s wal_replay_duration=14.421856543s total_replay_duration=22.210192687s pod_ip=9.218.6.233 checkpoint_replay_duration=9.538009348s wal_replay_duration=27.967002521s total_replay_duration=37.505067869s
checkpoint_replay_duration=7.887006797s wal_replay_duration=13.680547743s total_replay_duration=21.567604935s pod_ip=9.218.13.156 checkpoint_replay_duration=8.416435278s wal_replay_duration=18.420739919s total_replay_duration=26.837219169s
checkpoint_replay_duration=7.599726712s wal_replay_duration=14.332984201s total_replay_duration=21.932770356s checkpoint_replay_duration=8.520331941s wal_replay_duration=19.561665807s total_replay_duration=28.082042252s
checkpoint_replay_duration=8.058968769s wal_replay_duration=16.533978494s total_replay_duration=24.592997806s checkpoint_replay_duration=10.214446315s wal_replay_duration=20.207923702s total_replay_duration=30.422498235s
checkpoint_replay_duration=8.010912025s wal_replay_duration=14.842476964s total_replay_duration=22.853434415s checkpoint_replay_duration=8.66008611s wal_replay_duration=21.65027909s total_replay_duration=30.31038208s
checkpoint_replay_duration=7.427420235s wal_replay_duration=14.628869199s total_replay_duration=22.056335304s checkpoint_replay_duration=9.909061049s wal_replay_duration=24.159262162s total_replay_duration=34.068380678s

As the result shows, this change is faster, but it may use more CPU since now we get a chance to take advantage of the available CPU.

@damnever damnever requested a review from codesome July 14, 2022 15:26
Copy link
Member

@codesome codesome left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We also need to process []tombstones.Stone and []record.RefExemplar in the same way as how series processing is sharded. This is because processing tombstones and exemplars follow similar ordering requirement with its corresponding series as we do for samples.

I haven't checked other code changes yet , I will do it after this change. But the idea in general and the benchmarks look good, so we can go ahead with this approach.

@codesome
Copy link
Member

Oh, ignore my above comment. I see what you did with series creation. Not all happens in the sharded goroutine.

@damnever
Copy link
Contributor Author

Failed tests seem to be unrelated.

Signed-off-by: Xiaochao Dong (@damnever) <the.xcdong@gmail.com>
@damnever damnever force-pushed the fix/replay-walseries-concurrently branch from 2572856 to 699e9ea Compare July 29, 2022 02:12
@damnever damnever requested a review from codesome July 29, 2022 05:51
@damnever
Copy link
Contributor Author

damnever commented Aug 5, 2022

Friendly ping @codesome, is there anything we should do here?

@damnever damnever requested a review from colega August 16, 2022 06:01
tsdb/head_wal.go Outdated Show resolved Hide resolved
tsdb/head_wal.go Outdated Show resolved Hide resolved
Copy link
Contributor

@colega colega left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FWIW (I'm not a maintainer here, nor an expert in this code, but you've asked for my review 🙂) this LGTM, good job!

Please consider my nitpick comments, although they're definitely are not blockers.

@damnever
Copy link
Contributor Author

Thanks, @colega. I would like to hear some advice from @codesome before I commit more changes.. 😅

@codesome
Copy link
Member

Apologies in the delay. I plan to review it tomorrow. Thanks for your patience!

tsdb/head_wal.go Outdated Show resolved Hide resolved
tsdb/head_wal.go Outdated Show resolved Hide resolved
tsdb/head_wal.go Outdated Show resolved Hide resolved
@codesome
Copy link
Member

LGTM with nits above from me and @colega

@codesome
Copy link
Member

Nice work!

Signed-off-by: Xiaochao Dong (@damnever) <the.xcdong@gmail.com>
@damnever damnever requested a review from codesome August 17, 2022 12:41
@codesome codesome merged commit 09187fb into prometheus:main Aug 17, 2022
Mama59 pushed a commit to Arnoways/prometheus that referenced this pull request Aug 25, 2022
* Replay WAL concurrently without blocking

Signed-off-by: Xiaochao Dong (@damnever) <the.xcdong@gmail.com>

* Resolve review comments

Signed-off-by: Xiaochao Dong (@damnever) <the.xcdong@gmail.com>

Signed-off-by: Xiaochao Dong (@damnever) <the.xcdong@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants