diff --git a/pkg/sql/contention/txnidcache/fifo_cache.go b/pkg/sql/contention/txnidcache/fifo_cache.go index 5217d608e5d1..aef77400207e 100644 --- a/pkg/sql/contention/txnidcache/fifo_cache.go +++ b/pkg/sql/contention/txnidcache/fifo_cache.go @@ -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" ) @@ -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, @@ -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() } @@ -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) } } @@ -127,40 +118,23 @@ 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 { @@ -168,7 +142,11 @@ func (e *blockList) removeFront() *blockListNode { return nil } + e.numNodes-- removedBlock := e.head e.head = e.head.next + if e.head == nil { + e.tail = nil + } return removedBlock } diff --git a/pkg/sql/contention/txnidcache/fifo_cache_test.go b/pkg/sql/contention/txnidcache/fifo_cache_test.go index e43fff26ce03..68f6bb46d340 100644 --- a/pkg/sql/contention/txnidcache/fifo_cache_test.go +++ b/pkg/sql/contention/txnidcache/fifo_cache_test.go @@ -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 */) diff --git a/pkg/sql/contention/txnidcache/txn_id_cache.go b/pkg/sql/contention/txnidcache/txn_id_cache.go index 5baad3dab7e0..63c1a56727b4 100644 --- a/pkg/sql/contention/txnidcache/txn_id_cache.go +++ b/pkg/sql/contention/txnidcache/txn_id_cache.go @@ -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 } @@ -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() }