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