Skip to content

Commit

Permalink
sql: add cluster setting to enable/disable txn id cache
Browse files Browse the repository at this point in the history
Resolves #76329

Release note (sql change): new boolean cluster setting
`sql.contention.txn_id_cache.enabled` can be used to turn on/off
transaction ID cache. The default value of this new cluster setting
is `true`.
  • Loading branch information
Azhng committed Feb 14, 2022
1 parent a4a1aa9 commit 8db642f
Show file tree
Hide file tree
Showing 6 changed files with 69 additions and 3 deletions.
1 change: 1 addition & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ server.web_session.purge.max_deletions_per_cycle integer 10 the maximum number o
server.web_session.purge.period duration 1h0m0s the time until old sessions are deleted
server.web_session.purge.ttl duration 1h0m0s if nonzero, entries in system.web_sessions older than this duration are periodically purged
server.web_session_timeout duration 168h0m0s the duration that a newly created web session will be valid
sql.contention.txn_id_cache.enabled boolean true whether the txn id cache is enabled
sql.contention.txn_id_cache.max_size byte size 64 MiB the maximum byte size TxnID cache will use
sql.cross_db_fks.enabled boolean false if true, creating foreign key references across databases is allowed
sql.cross_db_sequence_owners.enabled boolean false if true, creating sequences owned by tables from other databases is allowed
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@
<tr><td><code>server.web_session.purge.period</code></td><td>duration</td><td><code>1h0m0s</code></td><td>the time until old sessions are deleted</td></tr>
<tr><td><code>server.web_session.purge.ttl</code></td><td>duration</td><td><code>1h0m0s</code></td><td>if nonzero, entries in system.web_sessions older than this duration are periodically purged</td></tr>
<tr><td><code>server.web_session_timeout</code></td><td>duration</td><td><code>168h0m0s</code></td><td>the duration that a newly created web session will be valid</td></tr>
<tr><td><code>sql.contention.txn_id_cache.enabled</code></td><td>boolean</td><td><code>true</code></td><td>whether the txn id cache is enabled</td></tr>
<tr><td><code>sql.contention.txn_id_cache.max_size</code></td><td>byte size</td><td><code>64 MiB</code></td><td>the maximum byte size TxnID cache will use</td></tr>
<tr><td><code>sql.cross_db_fks.enabled</code></td><td>boolean</td><td><code>false</code></td><td>if true, creating foreign key references across databases is allowed</td></tr>
<tr><td><code>sql.cross_db_sequence_owners.enabled</code></td><td>boolean</td><td><code>false</code></td><td>if true, creating sequences owned by tables from other databases is allowed</td></tr>
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/contention/txnidcache/cluster_settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,3 +19,11 @@ var MaxSize = settings.RegisterByteSizeSetting(
"the maximum byte size TxnID cache will use",
64*1024*1024, // 64 MB
).WithPublic()

// IsEnabled is used to turn on/off the txn id cache.
var IsEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"sql.contention.txn_id_cache.enabled",
"whether the txn id cache is enabled",
true, // defaultValue
).WithPublic()
2 changes: 1 addition & 1 deletion pkg/sql/contention/txnidcache/txn_id_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ func NewTxnIDCache(st *cluster.Settings, metrics *Metrics) *Cache {
return MaxSize.Get(&st.SV) / entrySize
} /* capacity */)

t.writer = newWriter(t)
t.writer = newWriter(st, t)
return t
}

Expand Down
9 changes: 8 additions & 1 deletion pkg/sql/contention/txnidcache/writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package txnidcache

import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)
Expand All @@ -20,15 +21,18 @@ import (
const shardCount = 16

type writer struct {
st *cluster.Settings

shards [shardCount]*concurrentWriteBuffer

sink messageSink
}

var _ Writer = &writer{}

func newWriter(sink messageSink) *writer {
func newWriter(st *cluster.Settings, sink messageSink) *writer {
w := &writer{
st: st,
sink: sink,
}

Expand All @@ -41,6 +45,9 @@ func newWriter(sink messageSink) *writer {

// Record implements the Writer interface.
func (w *writer) Record(resolvedTxnID ResolvedTxnID) {
if !IsEnabled.Get(&w.st.SV) {
return
}
shardIdx := hashTxnID(resolvedTxnID.TxnID)
buffer := w.shards[shardIdx]
buffer.Record(resolvedTxnID)
Expand Down
51 changes: 50 additions & 1 deletion pkg/sql/contention/txnidcache/writer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/stretchr/testify/require"
)

type blackHoleSink struct {
Expand Down Expand Up @@ -71,11 +72,12 @@ func BenchmarkWriter(b *testing.B) {
defer log.Scope(b).Close(b)

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()

run := func(b *testing.B, sink messageSink, numOfConcurrentWriter int) {
starter := make(chan struct{})

w := newWriter(sink)
w := newWriter(st, sink)

b.ResetTimer()
b.SetBytes(messageBlockSize * entrySize)
Expand Down Expand Up @@ -149,3 +151,50 @@ func BenchmarkWriter(b *testing.B) {
})
}
}

type counterSink struct {
numOfRecord int
}

var _ messageSink = &counterSink{}

func (c *counterSink) push(block *messageBlock) {
for i := 0; i < messageBlockSize; i++ {
if !block[i].valid() {
break
}
c.numOfRecord++
}
}

func TestTxnIDCacheCanBeDisabledViaClusterSetting(t *testing.T) {
st := cluster.MakeTestingClusterSettings()
ctx := context.Background()

sink := &counterSink{}
w := newWriter(st, sink)
w.Record(ResolvedTxnID{
TxnID: uuid.FastMakeV4(),
})

w.Flush()
require.Equal(t, 1, sink.numOfRecord)

// This should disable txn id cache.
IsEnabled.Override(ctx, &st.SV, false)

w.Record(ResolvedTxnID{
TxnID: uuid.FastMakeV4(),
})
w.Flush()
require.Equal(t, 1, sink.numOfRecord)

// This should re-enable txn id cache.
IsEnabled.Override(ctx, &st.SV, true)

w.Record(ResolvedTxnID{
TxnID: uuid.FastMakeV4(),
})
w.Flush()
require.Equal(t, 2, sink.numOfRecord)
}

0 comments on commit 8db642f

Please sign in to comment.