mirror of https://github.com/prometheus/prometheus
Browse Source
* Write chunks via queue, predicting the refs Our load tests have shown that there is a latency spike in the remote write handler whenever the head chunks need to be written, because chunkDiskMapper.WriteChunk() blocks until the chunks are written to disk. This adds a queue to the chunk disk mapper which makes the WriteChunk() method non-blocking unless the queue is full. Reads can still be served from the queue. Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * address PR feeddback Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * initialize metrics without .Add(0) Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * change isRunningMtx to normal lock Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * do not re-initialize chunkrefmap Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * update metric outside of lock scope Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * add benchmark for adding job to chunk write queue Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * remove unnecessary "success" var Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * gofumpt -extra Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * avoid WithLabelValues call in addJob Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * format comments Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * addressing PR feedback Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * rename cutExpectRef to cutAndExpectRef Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * use head.Init() instead of .initTime() Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * address PR feedback Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * PR feedback Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * update test according to PR feedback Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * replace callbackWg -> awaitCb Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * better test of truncation with empty files Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> * replace callbackWg -> awaitCb Signed-off-by: Mauro Stettler <mauro.stettler@gmail.com> Co-authored-by: Ganesh Vernekar <15064823+codesome@users.noreply.github.com>pull/10144/head
Mauro Stettler
3 years ago
committed by
GitHub
9 changed files with 960 additions and 124 deletions
@ -0,0 +1,165 @@
|
||||
// Copyright 2021 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package chunks |
||||
|
||||
import ( |
||||
"errors" |
||||
"sync" |
||||
|
||||
"github.com/prometheus/client_golang/prometheus" |
||||
|
||||
"github.com/prometheus/prometheus/tsdb/chunkenc" |
||||
) |
||||
|
||||
type chunkWriteJob struct { |
||||
cutFile bool |
||||
seriesRef HeadSeriesRef |
||||
mint int64 |
||||
maxt int64 |
||||
chk chunkenc.Chunk |
||||
ref ChunkDiskMapperRef |
||||
callback func(error) |
||||
} |
||||
|
||||
// chunkWriteQueue is a queue for writing chunks to disk in a non-blocking fashion.
|
||||
// Chunks that shall be written get added to the queue, which is consumed asynchronously.
|
||||
// Adding jobs to the job is non-blocking as long as the queue isn't full.
|
||||
type chunkWriteQueue struct { |
||||
jobs chan chunkWriteJob |
||||
|
||||
chunkRefMapMtx sync.RWMutex |
||||
chunkRefMap map[ChunkDiskMapperRef]chunkenc.Chunk |
||||
|
||||
isRunningMtx sync.Mutex // Protects the isRunning property.
|
||||
isRunning bool // Used to prevent that new jobs get added to the queue when the chan is already closed.
|
||||
|
||||
workerWg sync.WaitGroup |
||||
|
||||
writeChunk writeChunkF |
||||
|
||||
// Keeping three separate counters instead of only a single CounterVec to improve the performance of the critical
|
||||
// addJob() method which otherwise would need to perform a WithLabelValues call on the CounterVec.
|
||||
adds prometheus.Counter |
||||
gets prometheus.Counter |
||||
completed prometheus.Counter |
||||
} |
||||
|
||||
// writeChunkF is a function which writes chunks, it is dynamic to allow mocking in tests.
|
||||
type writeChunkF func(HeadSeriesRef, int64, int64, chunkenc.Chunk, ChunkDiskMapperRef, bool) error |
||||
|
||||
func newChunkWriteQueue(reg prometheus.Registerer, size int, writeChunk writeChunkF) *chunkWriteQueue { |
||||
counters := prometheus.NewCounterVec( |
||||
prometheus.CounterOpts{ |
||||
Name: "prometheus_tsdb_chunk_write_queue_operations_total", |
||||
Help: "Number of operations on the chunk_write_queue.", |
||||
}, |
||||
[]string{"operation"}, |
||||
) |
||||
|
||||
q := &chunkWriteQueue{ |
||||
jobs: make(chan chunkWriteJob, size), |
||||
chunkRefMap: make(map[ChunkDiskMapperRef]chunkenc.Chunk, size), |
||||
writeChunk: writeChunk, |
||||
|
||||
adds: counters.WithLabelValues("add"), |
||||
gets: counters.WithLabelValues("get"), |
||||
completed: counters.WithLabelValues("complete"), |
||||
} |
||||
|
||||
if reg != nil { |
||||
reg.MustRegister(counters) |
||||
} |
||||
|
||||
q.start() |
||||
return q |
||||
} |
||||
|
||||
func (c *chunkWriteQueue) start() { |
||||
c.workerWg.Add(1) |
||||
go func() { |
||||
defer c.workerWg.Done() |
||||
|
||||
for job := range c.jobs { |
||||
c.processJob(job) |
||||
} |
||||
}() |
||||
|
||||
c.isRunningMtx.Lock() |
||||
c.isRunning = true |
||||
c.isRunningMtx.Unlock() |
||||
} |
||||
|
||||
func (c *chunkWriteQueue) processJob(job chunkWriteJob) { |
||||
err := c.writeChunk(job.seriesRef, job.mint, job.maxt, job.chk, job.ref, job.cutFile) |
||||
if job.callback != nil { |
||||
job.callback(err) |
||||
} |
||||
|
||||
c.chunkRefMapMtx.Lock() |
||||
defer c.chunkRefMapMtx.Unlock() |
||||
|
||||
delete(c.chunkRefMap, job.ref) |
||||
|
||||
c.completed.Inc() |
||||
} |
||||
|
||||
func (c *chunkWriteQueue) addJob(job chunkWriteJob) (err error) { |
||||
defer func() { |
||||
if err == nil { |
||||
c.adds.Inc() |
||||
} |
||||
}() |
||||
|
||||
c.isRunningMtx.Lock() |
||||
defer c.isRunningMtx.Unlock() |
||||
|
||||
if !c.isRunning { |
||||
return errors.New("queue is not started") |
||||
} |
||||
|
||||
c.chunkRefMapMtx.Lock() |
||||
c.chunkRefMap[job.ref] = job.chk |
||||
c.chunkRefMapMtx.Unlock() |
||||
|
||||
c.jobs <- job |
||||
|
||||
return nil |
||||
} |
||||
|
||||
func (c *chunkWriteQueue) get(ref ChunkDiskMapperRef) chunkenc.Chunk { |
||||
c.chunkRefMapMtx.RLock() |
||||
defer c.chunkRefMapMtx.RUnlock() |
||||
|
||||
chk, ok := c.chunkRefMap[ref] |
||||
if ok { |
||||
c.gets.Inc() |
||||
} |
||||
|
||||
return chk |
||||
} |
||||
|
||||
func (c *chunkWriteQueue) stop() { |
||||
c.isRunningMtx.Lock() |
||||
defer c.isRunningMtx.Unlock() |
||||
|
||||
if !c.isRunning { |
||||
return |
||||
} |
||||
|
||||
c.isRunning = false |
||||
|
||||
close(c.jobs) |
||||
|
||||
c.workerWg.Wait() |
||||
} |
@ -0,0 +1,287 @@
|
||||
// Copyright 2021 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package chunks |
||||
|
||||
import ( |
||||
"errors" |
||||
"fmt" |
||||
"sync" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/stretchr/testify/require" |
||||
"go.uber.org/atomic" |
||||
|
||||
"github.com/prometheus/prometheus/tsdb/chunkenc" |
||||
) |
||||
|
||||
func TestChunkWriteQueue_GettingChunkFromQueue(t *testing.T) { |
||||
var blockWriterWg sync.WaitGroup |
||||
blockWriterWg.Add(1) |
||||
|
||||
// blockingChunkWriter blocks until blockWriterWg is done.
|
||||
blockingChunkWriter := func(_ HeadSeriesRef, _, _ int64, _ chunkenc.Chunk, _ ChunkDiskMapperRef, _ bool) error { |
||||
blockWriterWg.Wait() |
||||
return nil |
||||
} |
||||
|
||||
q := newChunkWriteQueue(nil, 1000, blockingChunkWriter) |
||||
|
||||
defer q.stop() |
||||
defer blockWriterWg.Done() |
||||
|
||||
testChunk := chunkenc.NewXORChunk() |
||||
var ref ChunkDiskMapperRef |
||||
job := chunkWriteJob{ |
||||
chk: testChunk, |
||||
ref: ref, |
||||
} |
||||
require.NoError(t, q.addJob(job)) |
||||
|
||||
// Retrieve chunk from the queue.
|
||||
gotChunk := q.get(ref) |
||||
require.Equal(t, testChunk, gotChunk) |
||||
} |
||||
|
||||
func TestChunkWriteQueue_WritingThroughQueue(t *testing.T) { |
||||
var ( |
||||
gotSeriesRef HeadSeriesRef |
||||
gotMint, gotMaxt int64 |
||||
gotChunk chunkenc.Chunk |
||||
gotRef ChunkDiskMapperRef |
||||
gotCutFile bool |
||||
) |
||||
|
||||
blockingChunkWriter := func(seriesRef HeadSeriesRef, mint, maxt int64, chunk chunkenc.Chunk, ref ChunkDiskMapperRef, cutFile bool) error { |
||||
gotSeriesRef = seriesRef |
||||
gotMint = mint |
||||
gotMaxt = maxt |
||||
gotChunk = chunk |
||||
gotRef = ref |
||||
gotCutFile = cutFile |
||||
return nil |
||||
} |
||||
|
||||
q := newChunkWriteQueue(nil, 1000, blockingChunkWriter) |
||||
defer q.stop() |
||||
|
||||
seriesRef := HeadSeriesRef(1) |
||||
var mint, maxt int64 = 2, 3 |
||||
chunk := chunkenc.NewXORChunk() |
||||
ref := newChunkDiskMapperRef(321, 123) |
||||
cutFile := true |
||||
awaitCb := make(chan struct{}) |
||||
require.NoError(t, q.addJob(chunkWriteJob{seriesRef: seriesRef, mint: mint, maxt: maxt, chk: chunk, ref: ref, cutFile: cutFile, callback: func(err error) { |
||||
close(awaitCb) |
||||
}})) |
||||
<-awaitCb |
||||
|
||||
// Compare whether the write function has received all job attributes correctly.
|
||||
require.Equal(t, seriesRef, gotSeriesRef) |
||||
require.Equal(t, mint, gotMint) |
||||
require.Equal(t, maxt, gotMaxt) |
||||
require.Equal(t, chunk, gotChunk) |
||||
require.Equal(t, ref, gotRef) |
||||
require.Equal(t, cutFile, gotCutFile) |
||||
} |
||||
|
||||
func TestChunkWriteQueue_WrappingAroundSizeLimit(t *testing.T) { |
||||
sizeLimit := 100 |
||||
unblockChunkWriterCh := make(chan struct{}, sizeLimit) |
||||
|
||||
// blockingChunkWriter blocks until the unblockChunkWriterCh channel returns a value.
|
||||
blockingChunkWriter := func(seriesRef HeadSeriesRef, mint, maxt int64, chunk chunkenc.Chunk, ref ChunkDiskMapperRef, cutFile bool) error { |
||||
<-unblockChunkWriterCh |
||||
return nil |
||||
} |
||||
|
||||
q := newChunkWriteQueue(nil, sizeLimit, blockingChunkWriter) |
||||
defer q.stop() |
||||
// Unblock writers when shutting down.
|
||||
defer close(unblockChunkWriterCh) |
||||
|
||||
var chunkRef ChunkDiskMapperRef |
||||
var callbackWg sync.WaitGroup |
||||
addChunk := func() { |
||||
callbackWg.Add(1) |
||||
require.NoError(t, q.addJob(chunkWriteJob{ |
||||
ref: chunkRef, |
||||
callback: func(err error) { |
||||
callbackWg.Done() |
||||
}, |
||||
})) |
||||
chunkRef++ |
||||
} |
||||
|
||||
unblockChunkWriter := func() { |
||||
unblockChunkWriterCh <- struct{}{} |
||||
} |
||||
|
||||
// Fill the queue to the middle of the size limit.
|
||||
for job := 0; job < sizeLimit/2; job++ { |
||||
addChunk() |
||||
} |
||||
|
||||
// Consume the jobs.
|
||||
for job := 0; job < sizeLimit/2; job++ { |
||||
unblockChunkWriter() |
||||
} |
||||
|
||||
// Add jobs until the queue is full.
|
||||
// Note that one more job than <sizeLimit> can be added because one will be processed by the worker already
|
||||
// and it will block on the chunk write function.
|
||||
for job := 0; job < sizeLimit+1; job++ { |
||||
addChunk() |
||||
} |
||||
|
||||
// The queue should be full.
|
||||
require.True(t, queueIsFull(q)) |
||||
|
||||
// Adding another job should block as long as no job from the queue gets consumed.
|
||||
addedJob := atomic.NewBool(false) |
||||
go func() { |
||||
addChunk() |
||||
addedJob.Store(true) |
||||
}() |
||||
|
||||
// Wait for 10ms while the adding of a new job is blocked.
|
||||
time.Sleep(time.Millisecond * 10) |
||||
require.False(t, addedJob.Load()) |
||||
|
||||
// Consume one job from the queue.
|
||||
unblockChunkWriter() |
||||
|
||||
// Wait until the job has been added to the queue.
|
||||
require.Eventually(t, func() bool { return addedJob.Load() }, time.Second, time.Millisecond*10) |
||||
|
||||
// The queue should be full again.
|
||||
require.True(t, queueIsFull(q)) |
||||
|
||||
// Consume <sizeLimit>+1 jobs from the queue.
|
||||
// To drain the queue we need to consume <sizeLimit>+1 jobs because 1 job
|
||||
// is already in the state of being processed.
|
||||
for job := 0; job < sizeLimit+1; job++ { |
||||
require.False(t, queueIsEmpty(q)) |
||||
unblockChunkWriter() |
||||
} |
||||
|
||||
// Wait until all jobs have been processed.
|
||||
callbackWg.Wait() |
||||
require.True(t, queueIsEmpty(q)) |
||||
} |
||||
|
||||
func TestChunkWriteQueue_HandlerErrorViaCallback(t *testing.T) { |
||||
testError := errors.New("test error") |
||||
chunkWriter := func(_ HeadSeriesRef, _, _ int64, _ chunkenc.Chunk, _ ChunkDiskMapperRef, _ bool) error { |
||||
return testError |
||||
} |
||||
|
||||
awaitCb := make(chan struct{}) |
||||
var gotError error |
||||
callback := func(err error) { |
||||
gotError = err |
||||
close(awaitCb) |
||||
} |
||||
|
||||
q := newChunkWriteQueue(nil, 1, chunkWriter) |
||||
defer q.stop() |
||||
|
||||
job := chunkWriteJob{callback: callback} |
||||
require.NoError(t, q.addJob(job)) |
||||
|
||||
<-awaitCb |
||||
|
||||
require.Equal(t, testError, gotError) |
||||
} |
||||
|
||||
func BenchmarkChunkWriteQueue_addJob(b *testing.B) { |
||||
for _, withReads := range []bool{false, true} { |
||||
b.Run(fmt.Sprintf("with reads %t", withReads), func(b *testing.B) { |
||||
for _, concurrentWrites := range []int{1, 10, 100, 1000} { |
||||
b.Run(fmt.Sprintf("%d concurrent writes", concurrentWrites), func(b *testing.B) { |
||||
issueReadSignal := make(chan struct{}) |
||||
q := newChunkWriteQueue(nil, 1000, func(ref HeadSeriesRef, i, i2 int64, chunk chunkenc.Chunk, ref2 ChunkDiskMapperRef, b bool) error { |
||||
if withReads { |
||||
select { |
||||
case issueReadSignal <- struct{}{}: |
||||
default: |
||||
// Can't write to issueReadSignal, don't block but omit read instead.
|
||||
} |
||||
} |
||||
return nil |
||||
}) |
||||
b.Cleanup(func() { |
||||
// Stopped already, so no more writes will happen.
|
||||
close(issueReadSignal) |
||||
}) |
||||
b.Cleanup(q.stop) |
||||
|
||||
start := sync.WaitGroup{} |
||||
start.Add(1) |
||||
|
||||
jobs := make(chan chunkWriteJob, b.N) |
||||
for i := 0; i < b.N; i++ { |
||||
jobs <- chunkWriteJob{ |
||||
seriesRef: HeadSeriesRef(i), |
||||
ref: ChunkDiskMapperRef(i), |
||||
} |
||||
} |
||||
close(jobs) |
||||
|
||||
go func() { |
||||
for range issueReadSignal { |
||||
// We don't care about the ID we're getting, we just want to grab the lock.
|
||||
_ = q.get(ChunkDiskMapperRef(0)) |
||||
} |
||||
}() |
||||
|
||||
done := sync.WaitGroup{} |
||||
done.Add(concurrentWrites) |
||||
for w := 0; w < concurrentWrites; w++ { |
||||
go func() { |
||||
start.Wait() |
||||
for j := range jobs { |
||||
_ = q.addJob(j) |
||||
} |
||||
done.Done() |
||||
}() |
||||
} |
||||
|
||||
b.ResetTimer() |
||||
start.Done() |
||||
done.Wait() |
||||
}) |
||||
} |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func queueIsEmpty(q *chunkWriteQueue) bool { |
||||
return queueSize(q) == 0 |
||||
} |
||||
|
||||
func queueIsFull(q *chunkWriteQueue) bool { |
||||
// When the queue is full and blocked on the writer the chunkRefMap has one more job than the cap of the jobCh
|
||||
// because one job is currently being processed and blocked in the writer.
|
||||
return queueSize(q) == cap(q.jobs)+1 |
||||
} |
||||
|
||||
func queueSize(q *chunkWriteQueue) int { |
||||
q.chunkRefMapMtx.Lock() |
||||
defer q.chunkRefMapMtx.Unlock() |
||||
|
||||
// Looking at chunkRefMap instead of jobCh because the job is popped from the chan before it has
|
||||
// been fully processed, it remains in the chunkRefMap until the processing is complete.
|
||||
return len(q.chunkRefMap) |
||||
} |
Loading…
Reference in new issue