Skip to content

Commit

Permalink
[DNM] sql/contention/txnidcache: reuse blocks in list, account for space
Browse files Browse the repository at this point in the history
This change does two things to the txnidcache:
1) It accounts for the space used by the fifo eviction list. Previously
   we'd use more than double the intended space. We should probably also
   subtrace out the size of the buffers we're currently filling and the
   channel we use to communicate them, but I'll leave that for later.
2) It stops trying to compact the blocks. Compacting the blocks ends up
   being a good deal of overhead because we have to copy across every
   single message. Instead we can just append the block directly to the
   list. This does have the hazard of wasting a lot of space when the
   blocks are sparse. However, if the blocks are sparse, we know that the
   throughput is low, so it's fine.

This is DNM because the tests need to change.

Touches #76738

Release justification: bug fixes and low-risk updates to new functionality

Release note: None
  • Loading branch information
ajwerner committed Mar 1, 2022
1 parent f71e753 commit 8ac79c0
Show file tree
Hide file tree
Showing 3 changed files with 31 additions and 56 deletions.
76 changes: 27 additions & 49 deletions pkg/sql/contention/txnidcache/fifo_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,10 @@ package txnidcache

import (
"sync"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/contention/contentionutils"
"github.com/cockroachdb/cockroach/pkg/sql/contentionpb"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)
Expand All @@ -39,21 +39,19 @@ type fifoCache struct {
}

type blockListNode struct {
block
*block
next *blockListNode
}

// blockList is a singly-linked list of blocks. The list is used to
// implement FIFO eviction.
type blockList struct {
head *blockListNode
tail *blockListNode

// tailIdx is an index pointing into the next empty slot in block
// stored in the tail pointer.
tailIdx int
numNodes int
head *blockListNode
tail *blockListNode
}

// newFifoCache takes a function which returns a capacity in bytes.
func newFIFOCache(capacity contentionutils.CapacityLimiter) *fifoCache {
c := &fifoCache{
capacity: capacity,
Expand All @@ -70,22 +68,14 @@ func (c *fifoCache) add(b *block) {
c.mu.Lock()
defer c.mu.Unlock()

blockSize := 0
for i := range b {
if !b[i].Valid() {
break
}

c.mu.data[b[i].TxnID] = b[i].TxnFingerprintID
blockSize++
}

c.mu.eviction.append(b[:blockSize])

// Zeros out the block and put it back into the blockPool.
*b = block{}
blockPool.Put(b)

c.mu.eviction.addNode(b)
c.maybeEvictLocked()
}

Expand All @@ -97,24 +87,25 @@ func (c *fifoCache) get(txnID uuid.UUID) (roachpb.TransactionFingerprintID, bool
return fingerprintID, found
}

func (c *fifoCache) size() int {
func (c *fifoCache) size() int64 {
c.mu.RLock()
defer c.mu.RUnlock()
return len(c.mu.data)
return c.sizeLocked()
}

func (c *fifoCache) sizeLocked() int64 {
return int64(c.mu.eviction.numNodes)*
((entrySize*blockSize)+int64(unsafe.Sizeof(blockListNode{}))) +
int64(len(c.mu.data))*entrySize
}

func (c *fifoCache) maybeEvictLocked() {
for int64(len(c.mu.data)) > c.capacity() {
for c.sizeLocked() > c.capacity() {
node := c.mu.eviction.removeFront()
if node == nil {
return
}

c.evictNodeLocked(node)

// Zero out the node and put it back into the pool.
*node = blockListNode{}
nodePool.Put(node)
}
}

Expand All @@ -127,48 +118,35 @@ func (c *fifoCache) evictNodeLocked(node *blockListNode) {

delete(c.mu.data, node.block[i].TxnID)
}
}

func (e *blockList) append(block []contentionpb.ResolvedTxnID) {
block = e.appendToTail(block)
for len(block) > 0 {
e.addNode()
block = e.appendToTail(block)
}
*node.block = block{}
blockPool.Put(node.block)
*node = blockListNode{}
nodePool.Put(node)
}

func (e *blockList) addNode() {
func (e *blockList) addNode(b *block) {
newNode := nodePool.Get().(*blockListNode)
newNode.block = b
if e.head == nil {
e.head = newNode
} else {
e.tail.next = newNode
}
e.tail = newNode
e.tailIdx = 0
}

func (e *blockList) appendToTail(
block []contentionpb.ResolvedTxnID,
) (remaining []contentionpb.ResolvedTxnID) {
if e.head == nil {
return block
}
toCopy := blockSize - e.tailIdx
if toCopy > len(block) {
toCopy = len(block)
}
copy(e.tail.block[e.tailIdx:], block[:toCopy])
e.tailIdx += toCopy
return block[toCopy:]
e.numNodes++
}

func (e *blockList) removeFront() *blockListNode {
if e.head == nil {
return nil
}

e.numNodes--
removedBlock := e.head
e.head = e.head.next
if e.head == nil {
e.tail = nil
}
return removedBlock
}
2 changes: 1 addition & 1 deletion pkg/sql/contention/txnidcache/fifo_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import (
)

func TestFIFOCache(t *testing.T) {
cache := newFIFOCache(func() int64 { return 2 * blockSize } /* capacity */)
cache := newFIFOCache(func() int64 { return 4 * blockSize * entrySize } /* capacity */)

// Fill the first eviction block in cache to 1/4 capacity.
input1, expected1 := generateInputBlock(blockSize * 1 / 4 /* size */)
Expand Down
9 changes: 3 additions & 6 deletions pkg/sql/contention/txnidcache/txn_id_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,10 +132,7 @@ func NewTxnIDCache(st *cluster.Settings, metrics *Metrics) *Cache {
closeCh: make(chan struct{}),
}

t.store = newFIFOCache(func() int64 {
return MaxSize.Get(&st.SV) / entrySize
} /* capacity */)

t.store = newFIFOCache(func() int64 { return MaxSize.Get(&st.SV) })
t.writer = newWriter(st, t)
return t
}
Expand Down Expand Up @@ -189,7 +186,7 @@ func (t *Cache) DrainWriteBuffer() {
t.writer.DrainWriteBuffer()
}

// Size return the current size of the Cache.
// Size return the current size of the Cache in bytes.
func (t *Cache) Size() int64 {
return int64(t.store.size()) * entrySize
return t.store.size()
}

0 comments on commit 8ac79c0

Please sign in to comment.