Skip to content

Commit

Permalink
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.

Resolves #76738

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

Release note: None
  • Loading branch information
ajwerner authored and Azhng committed Mar 2, 2022
1 parent 1cc1725 commit e2b5e1e
Show file tree
Hide file tree
Showing 5 changed files with 1,894 additions and 234 deletions.
3 changes: 3 additions & 0 deletions pkg/sql/contention/txnidcache/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ go_test(
"txn_id_cache_test.go",
"writer_test.go",
],
data = glob(["testdata/**"]),
embed = [":txnidcache"],
deps = [
"//pkg/kv",
Expand All @@ -55,7 +56,9 @@ go_test(
"//pkg/util/log",
"//pkg/util/stop",
"//pkg/util/syncutil",
"//pkg/util/uint128",
"//pkg/util/uuid",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
],
Expand Down
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
}
Loading

0 comments on commit e2b5e1e

Please sign in to comment.