Skip to content

Commit

Permalink
Job queue (#247)
Browse files Browse the repository at this point in the history
This PR reimplements chan chunkWriteJob with custom buffered queue that should use less memory, because it doesn't preallocate entire buffer for maximum queue size at once. Instead it allocates individual "segments" with smaller size.

As elements are added to the queue, they fill individual segments. When elements are removed from the queue (and segments), empty segments can be thrown away. This doesn't change memory usage of the queue when it's full, but should decrease its memory footprint when it's empty (queue will keep max 1 segment in such case).
  • Loading branch information
pstibrany authored Jun 7, 2022
1 parent 459f599 commit 1e2d2fb
Show file tree
Hide file tree
Showing 3 changed files with 464 additions and 6 deletions.
33 changes: 27 additions & 6 deletions tsdb/chunks/chunk_write_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,10 @@ const (

// Minimum interval between shrinking of chunkWriteQueue.chunkRefMap.
chunkRefMapMinShrinkInterval = 10 * time.Minute

// Maximum size of segment used by job queue (number of elements). With chunkWriteJob being 64 bytes,
// this will use ~512 KiB for empty queue.
maxChunkQueueSegmentSize = 8192
)

type chunkWriteJob struct {
Expand All @@ -45,7 +49,7 @@ type chunkWriteJob struct {
// Chunks that shall be written get added to the queue, which is consumed asynchronously.
// Adding jobs to the queue is non-blocking as long as the queue isn't full.
type chunkWriteQueue struct {
jobs chan chunkWriteJob
jobs *writeJobQueue

chunkRefMapMtx sync.RWMutex
chunkRefMap map[ChunkDiskMapperRef]chunkenc.Chunk
Expand Down Expand Up @@ -83,8 +87,13 @@ func newChunkWriteQueue(reg prometheus.Registerer, size int, writeChunk writeChu
[]string{"operation"},
)

segmentSize := size
if segmentSize > maxChunkQueueSegmentSize {
segmentSize = maxChunkQueueSegmentSize
}

q := &chunkWriteQueue{
jobs: make(chan chunkWriteJob, size),
jobs: newWriteJobQueue(size, segmentSize),
chunkRefMap: make(map[ChunkDiskMapperRef]chunkenc.Chunk),
chunkRefMapLastShrink: time.Now(),
writeChunk: writeChunk,
Expand All @@ -108,7 +117,12 @@ func (c *chunkWriteQueue) start() {
go func() {
defer c.workerWg.Done()

for job := range c.jobs {
for {
job, ok := c.jobs.pop()
if !ok {
return
}

c.processJob(job)
}
}()
Expand Down Expand Up @@ -191,7 +205,14 @@ func (c *chunkWriteQueue) addJob(job chunkWriteJob) (err error) {
}
c.chunkRefMapMtx.Unlock()

c.jobs <- job
ok := c.jobs.push(job)
if !ok {
c.chunkRefMapMtx.Lock()
delete(c.chunkRefMap, job.ref)
c.chunkRefMapMtx.Unlock()

return errors.New("queue is closed")
}

return nil
}
Expand All @@ -218,7 +239,7 @@ func (c *chunkWriteQueue) stop() {

c.isRunning = false

close(c.jobs)
c.jobs.close()

c.workerWg.Wait()
}
Expand All @@ -230,7 +251,7 @@ func (c *chunkWriteQueue) queueIsEmpty() bool {
func (c *chunkWriteQueue) queueIsFull() 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 c.queueSize() == cap(c.jobs)+1
return c.queueSize() == c.jobs.maxSize+1
}

func (c *chunkWriteQueue) queueSize() int {
Expand Down
127 changes: 127 additions & 0 deletions tsdb/chunks/queue.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
package chunks

import "sync"

// writeJobQueue is similar to buffered channel of chunkWriteJob, but manages its own buffers
// to avoid using a lot of memory when it's empty. It does that by storing elements into segments
// of equal size (segmentSize). When segment is not used anymore, reference to it are removed,
// so it can be treated as a garbage.
type writeJobQueue struct {
maxSize int
segmentSize int

mtx sync.Mutex // protects all following variables
pushed, popped *sync.Cond // signalled when something is pushed into the queue or popped from it
first, last *writeJobQueueSegment // pointer to first and last segment, if any
size int // total size of the queue
closed bool // after closing the queue, nothing can be pushed to it
}

type writeJobQueueSegment struct {
segment []chunkWriteJob
nextRead, nextWrite int // index of next read and next write in this segment.
nextSegment *writeJobQueueSegment // next segment, if any
}

func newWriteJobQueue(maxSize, segmentSize int) *writeJobQueue {
if maxSize <= 0 || segmentSize <= 0 {
panic("invalid queue")
}

q := &writeJobQueue{
maxSize: maxSize,
segmentSize: segmentSize,
}

q.pushed = sync.NewCond(&q.mtx)
q.popped = sync.NewCond(&q.mtx)
return q
}

func (q *writeJobQueue) close() {
q.mtx.Lock()
defer q.mtx.Unlock()

q.closed = true

// unblock all blocked goroutines
q.pushed.Broadcast()
q.popped.Broadcast()
}

// push blocks until there is space available in the queue, and then adds job to the queue.
// If queue is closed or gets closed while waiting for space, push returns false.
func (q *writeJobQueue) push(job chunkWriteJob) bool {
q.mtx.Lock()
defer q.mtx.Unlock()

// wait until queue has more space or is closed
for !q.closed && q.size >= q.maxSize {
q.popped.Wait()
}

if q.closed {
return false
}

// Check if this segment has more space for writing, and create new one if not.
if q.last == nil || q.last.nextWrite >= q.segmentSize {
prevLast := q.last
q.last = &writeJobQueueSegment{
segment: make([]chunkWriteJob, q.segmentSize),
}

if prevLast != nil {
prevLast.nextSegment = q.last
}
if q.first == nil {
q.first = q.last
}
}

q.last.segment[q.last.nextWrite] = job
q.last.nextWrite++
q.size++
q.pushed.Signal()
return true
}

// pop returns first job from the queue, and true.
// if queue is empty, pop blocks until there is a job (returns true), or until queue is closed (returns false).
// If queue was already closed, pop first returns all remaining elements from the queue (with true value), and only then returns false.
func (q *writeJobQueue) pop() (chunkWriteJob, bool) {
q.mtx.Lock()
defer q.mtx.Unlock()

// wait until something is pushed to the queue, or queue is closed.
for q.size == 0 {
if q.closed {
return chunkWriteJob{}, false
}

q.pushed.Wait()
}

res := q.first.segment[q.first.nextRead]
q.first.segment[q.first.nextRead] = chunkWriteJob{} // clear just-read element
q.first.nextRead++
q.size--

// If we have read all possible elements from first segment, we can drop it.
if q.first.nextRead >= q.segmentSize {
q.first = q.first.nextSegment
if q.first == nil {
q.last = nil
}
}

q.popped.Signal()
return res, true
}

func (q *writeJobQueue) length() int {
q.mtx.Lock()
defer q.mtx.Unlock()

return q.size
}
Loading

0 comments on commit 1e2d2fb

Please sign in to comment.