-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
sql: introduce Transaction ID Cache #74115
sql: introduce Transaction ID Cache #74115
Conversation
9be1d1a
to
1849110
Compare
// | +-------------+ | writes to | | ||
// | | Writer |<-* | | ||
// | +-----+-------+ | | ||
// +------------|-----------------------+ | ||
// | | ||
// when full, Writer flushes into a channel. | ||
// | | ||
// v | ||
// channel | ||
// ^ | ||
// | | ||
// Cache runs a goroutine that polls the channel --* | ||
// | and sends the message to its | | ||
// | corresponding shard | | ||
// | *-----------------------------* | ||
// +----------------------------|---+ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It feels to me like the connExecutor should just write straight to a shard. The shard is sharded and the channel is not. What would happen if you simplified it such that when a shard is full, you send the shard itself on the channel to flush and swap it out with a new shard? You can then sync.Pool the shards
fa1222e
to
7d09bcb
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @Azhng, and @nvanbenschoten)
pkg/sql/contention/txnidcache/txn_id_cache.go, line 52 at r2 (raw file):
Previously, ajwerner wrote…
It feels to me like the connExecutor should just write straight to a shard. The shard is sharded and the channel is not. What would happen if you simplified it such that when a shard is full, you send the shard itself on the channel to flush and swap it out with a new shard? You can then sync.Pool the shards
I agree that we want the connExecutor calling directly into a shard — until reading this comment, I thought that was the point of the ConcurrentWriteBuffer
. But I don't understand your "shard is full" comment. We seem to be using a fixed number of shards and enforcing a separate memory limit + FIFO eviction policy within each shard. So unless I'm completely misreading this (very possible), the shards are static.
pkg/sql/contention/txnidcache/txn_id_cache.go, line 80 at r3 (raw file):
8 // the size of a hash
Where does this come from?
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 51 at r3 (raw file):
txnIDCache: t, } shard.mu.store = cache.NewUnorderedCache(cache.Config{
I've been following this work from afar since reading the original RFC. It seems like we're making great progress in terms of reducing the cost of this cache. The ConcurrentWriteBuffer
should significantly reduce the lock contention we see when recording transaction IDs.
Another suggestion to make this cheaper is to avoid per-transaction accounting in each cacheShard
. We are currently tracking the aggregate size of entries and enforcing a FIFO policy that keeps each cache shard as full as possible while staying below a memory limit. In exchange, we use cache.UnorderedCache
, which is quite heavyweight (i.e. all entries are wrapped, a per-entry linked-list is maintained, etc.).
We really don't need precise memory accounting or FIFO enforcement, we just need something that approximates it. So you can imagine much cheaper methods for enforcing a hard memory limit with a loose temporal eviction preference. A strawman proposal is a ring buffer with each slot containing a size-limited bounded map[uuid.UUID]roachpb.TransactionFingerprintID
. When inserting, you push into the head of the ring buffer, rotating if necessary when a map fills up. When performing a lookup, you would iterate over the maps (in reverse time order, to advantage recent entries) and perform lookups, short-circuiting if you found what you were looking for. You could play with the size of the ring buffer (i.e. the number of maps), which becomes a trade-off between read amplification and eviction granularity. My guess is that a value of 4 or 8 would be reasonable. Then the maximum capacity of each map would become ((MaxSize / shardCount) / ringBufferSize) / entrySize
, which is O(1000).
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 79 at r3 (raw file):
// Record implements the writer interface. func (s *cacheShard) Record(msg ResolvedTxnID) {
I don't mean to pick a fight with the interfaces here, as I imagine they have been useful during the design iteration, but let's make sure they don't force us into a suboptimal final state. For instance, does it make sense for the mutex locking around a cacheShard
and its associated ConcurrentWriteBuffer
to be separate? And does it make sense to record ResolvedTxnID
s to a cacheShard
one-by-one when the caller wants to do so in a large batch?
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 80 at r3 (raw file):
// Record implements the writer interface. func (s *cacheShard) Record(msg ResolvedTxnID) { s.writeBuffer = append(s.writeBuffer, msg)
What is the benefit of this writeBuffer
? Will is be replaced by ConcurrentWriteBuffer
at some point?
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 94 at r3 (raw file):
// Lookup implements the reader interface. func (s *cacheShard) Lookup(txnID uuid.UUID) (result roachpb.TransactionFingerprintID, found bool) {
Do we need to flush on lookup?
0eccf56
to
afba614
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @nvanbenschoten)
pkg/sql/contention/txnidcache/txn_id_cache.go, line 52 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I agree that we want the connExecutor calling directly into a shard — until reading this comment, I thought that was the point of the
ConcurrentWriteBuffer
. But I don't understand your "shard is full" comment. We seem to be using a fixed number of shards and enforcing a separate memory limit + FIFO eviction policy within each shard. So unless I'm completely misreading this (very possible), the shards are static.
Rewrote part of the diagram to reflect the new changes.
pkg/sql/contention/txnidcache/txn_id_cache.go, line 80 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
8 // the size of a hash
Where does this come from?
It was used to account the "hash" inside of Go's hashmap. Removed.
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 51 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I've been following this work from afar since reading the original RFC. It seems like we're making great progress in terms of reducing the cost of this cache. The
ConcurrentWriteBuffer
should significantly reduce the lock contention we see when recording transaction IDs.Another suggestion to make this cheaper is to avoid per-transaction accounting in each
cacheShard
. We are currently tracking the aggregate size of entries and enforcing a FIFO policy that keeps each cache shard as full as possible while staying below a memory limit. In exchange, we usecache.UnorderedCache
, which is quite heavyweight (i.e. all entries are wrapped, a per-entry linked-list is maintained, etc.).We really don't need precise memory accounting or FIFO enforcement, we just need something that approximates it. So you can imagine much cheaper methods for enforcing a hard memory limit with a loose temporal eviction preference. A strawman proposal is a ring buffer with each slot containing a size-limited bounded
map[uuid.UUID]roachpb.TransactionFingerprintID
. When inserting, you push into the head of the ring buffer, rotating if necessary when a map fills up. When performing a lookup, you would iterate over the maps (in reverse time order, to advantage recent entries) and perform lookups, short-circuiting if you found what you were looking for. You could play with the size of the ring buffer (i.e. the number of maps), which becomes a trade-off between read amplification and eviction granularity. My guess is that a value of 4 or 8 would be reasonable. Then the maximum capacity of each map would become((MaxSize / shardCount) / ringBufferSize) / entrySize
, which is O(1000).
This is a very neat idea. I rewrote the implementation behind each shard
to use his ring buffer size approach.
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 79 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I don't mean to pick a fight with the interfaces here, as I imagine they have been useful during the design iteration, but let's make sure they don't force us into a suboptimal final state. For instance, does it make sense for the mutex locking around a
cacheShard
and its associatedConcurrentWriteBuffer
to be separate? And does it make sense to recordResolvedTxnID
s to acacheShard
one-by-one when the caller wants to do so in a large batch?
Reworked the interface so that the write path happens through a batch-insertion push()
call.
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 80 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
What is the benefit of this
writeBuffer
? Will is be replaced byConcurrentWriteBuffer
at some point?
It was kind left behind when I was playing around with different idea of reduce mutex contentions. Now removed since the entire shard implementation was rewritten.
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 94 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Do we need to flush on lookup?
Hmm I'm leaning towards no. Since flush is a pretty expensive operation and it will block every single active writer (and consequently, every singe connExecutor). I think the caller should be responsible for retrying if the lookup fail.
afba614
to
f280662
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 2 of 18 files at r4.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @Azhng, and @nvanbenschoten)
pkg/sql/contention/txnidcache/strip.go, line 30 at r4 (raw file):
func newStrip(capacity capacityLimiter) *strip { c := &strip{ data: make(map[uuid.UUID]roachpb.TransactionFingerprintID),
You could give the map an initial capacity of capacity()
.
pkg/sql/contention/txnidcache/strip.go, line 61 at r4 (raw file):
blockIdx := blockStartingOffset capn := c.capacity()
nit: Grab the capacity reading before entering the lock, since it might be an "expensive" atomic load.
pkg/sql/contention/txnidcache/strip.go, line 64 at r4 (raw file):
for ; blockIdx < messageBlockSize && block[blockIdx].valid() && c.size < capn; blockIdx++ { c.data[block[blockIdx].TxnID] = block[blockIdx].TxnFingerprintID c.size++
Isn't size
the same thing as len(c.data)
?
pkg/sql/contention/txnidcache/txn_id_cache.go, line 52 at r2 (raw file):
Previously, Azhng (Archer Zhang) wrote…
Rewrote part of the diagram to reflect the new changes.
Doesn't this diagram still show the connExecutor writing to the cache through a channel and goroutine?
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 94 at r3 (raw file):
Previously, Azhng (Archer Zhang) wrote…
Hmm I'm leaning towards no. Since flush is a pretty expensive operation and it will block every single active writer (and consequently, every singe connExecutor). I think the caller should be responsible for retrying if the lookup fail.
Retrying is an interesting idea. You can imagine a hybrid scheme where each caller performs an "optimistic" read and only flushes and retries if that misses.
f280662
to
fc8b1cc
Compare
fc8b1cc
to
3cf7120
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @nvanbenschoten)
pkg/sql/contention/txnidcache/strip.go, line 30 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
You could give the map an initial capacity of
capacity()
.
Done.
pkg/sql/contention/txnidcache/strip.go, line 61 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: Grab the capacity reading before entering the lock, since it might be an "expensive" atomic load.
Done.
pkg/sql/contention/txnidcache/strip.go, line 64 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Isn't
size
the same thing aslen(c.data)
?
Ah good point. Done.
pkg/sql/contention/txnidcache/txn_id_cache.go, line 52 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Doesn't this diagram still show the connExecutor writing to the cache through a channel and goroutine?
Hmm previously the set up was ConcurrentWriteBuffer
-> channel
--- polled via goroutine ---> sharded store
.
It felt a bit redundant for sure, however after I updated the code to remove the channel
in the middle to directly to the sharded store
,
However, after running the benchmark it looks like there's a jump in the overhead compared to the version where channel
is used.
$ benchstat ~/tmp/lightweightmap/bench ~/tmp/lightweightmap-no-chan/bench
name old overhead(ns/op) new overhead(ns/op) delta
ConcurrentSelect1/concurrentConn=24-24 4.05k ±21% 5.44k ±18% +34.41% (p=0.000 n=10+9)
ConcurrentSelect1/concurrentConn=48-24 5.08k ±20% 5.83k ±98% ~ (p=0.211 n=9+10)
ConcurrentSelect1/concurrentConn=64-24 5.58k ±22% 4.30k ±19% -22.99% (p=0.003 n=8+8)
I'm not sure how this will look on TPCC, I'll go ahead and run the benchmark and report back later.
pkg/sql/contention/txnidcache/txn_id_cache_shard.go, line 94 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Retrying is an interesting idea. You can imagine a hybrid scheme where each caller performs an "optimistic" read and only flushes and retries if that misses.
Interesting 🤔. That could definitely be a possibility. Though I will hold off on implementing that here and explore that when I work on the read-path of the Transaction ID Resolution protocol.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @nvanbenschoten)
pkg/sql/contention/txnidcache/txn_id_cache.go, line 52 at r2 (raw file):
Previously, Azhng (Archer Zhang) wrote…
Hmm previously the set up was
ConcurrentWriteBuffer
->channel
--- polled via goroutine --->sharded store
.It felt a bit redundant for sure, however after I updated the code to remove the
channel
in the middle to directly to thesharded store
,However, after running the benchmark it looks like there's a jump in the overhead compared to the version where
channel
is used.$ benchstat ~/tmp/lightweightmap/bench ~/tmp/lightweightmap-no-chan/bench name old overhead(ns/op) new overhead(ns/op) delta ConcurrentSelect1/concurrentConn=24-24 4.05k ±21% 5.44k ±18% +34.41% (p=0.000 n=10+9) ConcurrentSelect1/concurrentConn=48-24 5.08k ±20% 5.83k ±98% ~ (p=0.211 n=9+10) ConcurrentSelect1/concurrentConn=64-24 5.58k ±22% 4.30k ±19% -22.99% (p=0.003 n=8+8)I'm not sure how this will look on TPCC, I'll go ahead and run the benchmark and report back later.
Reporting back. I did a few TPCC runs. Compared to the baseline latency (master
entries), the p75 (and above) latency is higher for the implementation without using the channel (if we disregard the last bad run). The implementation using the channel is a lot more comparable to the baseline latency. I also attached the benchmark result for the implementation where it directly writes to the channel without using ConcurrentWriteBuffer
as a reference.
Branch | tpmC | %efficiency | Avg (ms) | p50 (ms) | p75 (ms) | p90 (ms) | p99 (ms) | pMax (ms) |
---|---|---|---|---|---|---|---|---|
master | 31386.0 | 97.6% | 76.2 | 30.4 | 58.7 | 71.3 | 109.1 | 23622.3 |
master | 31430.2 | 97.8% | 50.4 | 29.4 | 58.7 | 71.3 | 113.2 | 11811.2 |
no-chan | 31419.2 | 97.7% | 64.1 | 29.4 | 75.5 | 92.3 | 151.0 | 12348.0 |
no-chan | 31397.2 | 97.7% | 73.9 | 31.5 | 83.9 | 104.9 | 176.2 | 16106.1 |
no-chan* | 31324.9 | 97.4% | 135.7 | 35.7 | 113.2 | 142.6 | 1476.4 | 38654.7 |
chan | 31419.3 | 97.7% | 54.7 | 28.3 | 56.6 | 67.1 | 109.1 | 16106.1 |
chan | 31421.9 | 97.7% | 56.6 | 28.3 | 58.7 | 75.5 | 121.6 | 17179.9 |
channel without concurrent buffer | 31298.1 | 97.4% | 153.3 | 58.7 | 218.1 | 469.8 | 2013.3 | 15032.4 |
47f0fbc
to
1e68c05
Compare
74114: bench: add concurrect select1 benchmark r=maryliag a=Azhng Follow up PR: #74115 This commit introduces new BenchmarkConcurrentSelect1 benchmark. This benchmark is similar to the existing BenchmarkSelect1, but with higher concurrency. This benchmark is introduced to test how different components behave with increased concurrency to simulate the their behaviors in the macrobenchmark setting. Release note: None Co-authored-by: Azhng <[email protected]>
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 1 of 7 files at r1, 3 of 29 files at r5, 23 of 26 files at r6, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @Azhng, @maryliag, and @nvanbenschoten)
pkg/sql/conn_executor.go, line 286 at r6 (raw file):
// txnIDCache stores the mapping from transaction ID to transaction //fingerprint IDs for all recently executed.
nit: add space after //
... all recently executed transactions.
pkg/sql/contention/txnidcache/shard.go, line 57 at r6 (raw file):
for i := s.head; ; i = s.prevIdx(i) { fingerprintID, found := s.ring[i].Lookup(txnID)
do we need both the ID and the found
? Can't we just check the ID value, if is not 0 we know it was found, so we won't need the extra variable
pkg/sql/contention/txnidcache/cluster_settings.go, line 20 at r6 (raw file):
`sql.contention.txn_id_cache.max_size`, "the maximum byte size TxnID cache will use", 64*1024*1024, // 64 MB
where there any tests made to decide on this value?
1e68c05
to
07fb431
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @andreimatei, @Azhng, @maryliag, and @nvanbenschoten)
pkg/ts/catalog/chart_catalog.go, line 1887 at r14 (raw file):
Previously, ajwerner wrote…
is this actually a useful metric for anybody? I feel like the useful metric would be the miss rate and maybe the total number of reads. I can't imagine anybody ever using this.
By miss rate, I guess I mean, misses and reads as two counters. From that you can compute rate.
c8d139c
to
59293eb
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@andreimatei updated the conn_fsm
to return txnID
in the advanceInfo
. I have a few questions below.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @andreimatei, @maryliag, and @nvanbenschoten)
pkg/sql/conn_executor.go, line 1559 at r13 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
at the very least call the new arg
prevTxnID
. But have you considered putting it insideev
?
Done.
pkg/sql/conn_executor.go, line 1033 at r15 (raw file):
// that we can write it to transaction ID cache later. // TODO(azhng): wip: question: should we consume the advanceInfo here? finishedTxnID = ex.state.adv.txnID
in txnStateTransitionsApplyWrapper
we always consume the advanceInfo
(consumeAdvanceInfo()
) before reading from it. Should we still do the same here?
pkg/sql/conn_executor.go, line 2268 at r15 (raw file):
txnOpt = copyTxnOpt{ resetExtraTxnState: func(ctx context.Context, txnID uuid.UUID) error { return ex.resetExtraTxnState(ctx, noEvent, txnID)
@andreimatei it seems like the copyMachine
doesn't quite go through the same conn_fsm
transitions. Do you mind taking a look at the integration point here too as well to see if this is correct?
pkg/sql/contention/txnidcache/txn_id_cache.go, line 193 at r14 (raw file):
Previously, ajwerner wrote…
I think you need to detect the error here and close
t.closeCh
. Otherwise it seems possible for there to be a race where a flush routine is blocked and nothing will ever read it.
Done.
pkg/ts/catalog/chart_catalog.go, line 1887 at r14 (raw file):
Previously, ajwerner wrote…
By miss rate, I guess I mean, misses and reads as two counters. From that you can compute rate.
Done.
59293eb
to
0ca6273
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @andreimatei, @maryliag, and @nvanbenschoten)
pkg/sql/txn_state.go, line 280 at r16 (raw file):
if ts.mu.txn != nil { finishedTxnID = ts.mu.txn.ID() }
Hmm I'm a bit surprised by this, without the if nil
check, the call to ts.mu.txn.ID()
will result in nil pointer exception on the CI. Anyone knows why this would be the case 🤔 ?
E.g.
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @andreimatei, @Azhng, @maryliag, and @nvanbenschoten)
pkg/sql/conn_executor.go, line 1033 at r15 (raw file):
Previously, Azhng (Archer Zhang) wrote…
in
txnStateTransitionsApplyWrapper
we always consume theadvanceInfo
(consumeAdvanceInfo()
) before reading from it. Should we still do the same here?
Yes. The adv
field is not meant to be read directly.
pkg/sql/conn_executor.go, line 2268 at r15 (raw file):
Previously, Azhng (Archer Zhang) wrote…
@andreimatei it seems like the
copyMachine
doesn't quite go through the sameconn_fsm
transitions. Do you mind taking a look at the integration point here too as well to see if this is correct?
I don't really know... But are these changes necessary? Wouldn't it be suitable to pass emptyTxnID
to resetExtraTxnState
here, particularly since we're passing in noEvent
?
pkg/sql/conn_executor_exec.go, line 1932 at r16 (raw file):
} func (ex *connExecutor) onTxnFinish(ctx context.Context, ev txnEvent, prevTxnID uuid.UUID) {
seeing how much plumbing it required to take this prevTxnID
here, I think a better option would have been to turn txnEvent
into a
type txnEvent struct {
eventType <enum>
txnID UUID
}
I would also consider making the txnID
optional and not passing it in in the closing cases. Just don't collect the data for those transactions; if the connection drops in the middle of a transaction, I doubt anybody will be interested in those fingerprints. Like, undo the changes to finishExternalTxn()
- remove the return value.
pkg/sql/conn_fsm.go, line 512 at r16 (raw file):
func (ts *txnState) finishTxn(ev txnEvent) error { returnedTxnID := uuid.UUID{} finishedTxnID := ts.finishSQLTxn()
move this line inside the if
block below
pkg/sql/conn_fsm.go, line 514 at r16 (raw file):
finishedTxnID := ts.finishSQLTxn() // Only set the txnID field in the advanceInfo if the txnEvent is either
would anything go wrong if you blindly filled in the txnID
, even for noEvent
? If you can get away with it, I'd consider doing it to get rid of the need for this comment.
pkg/sql/txn_state.go, line 151 at r16 (raw file):
// (unless otherwise specified). // tranCtx: A bag of extra execution context. // This function returns the ID of the new transaction.
s/This function//
And move this above the parameters.
pkg/sql/txn_state.go, line 209 at r16 (raw file):
ts.mu.txn = txn } txnID = ts.mu.txn.ID()
why do we need the txnID
variable?
pkg/sql/txn_state.go, line 280 at r16 (raw file):
Previously, Azhng (Archer Zhang) wrote…
Hmm I'm a bit surprised by this, without the
if nil
check, the call tots.mu.txn.ID()
will result in nil pointer exception on the CI. Anyone knows why this would be the case 🤔 ?
I think because of this transition, no?
Looks like this finishExternalTxn()
is called regardless of whether the txn had been previously finished.
d4a02ad
to
ac5e305
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @andreimatei, @Azhng, @maryliag, and @nvanbenschoten)
pkg/sql/conn_executor.go, line 2268 at r15 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I don't really know... But are these changes necessary? Wouldn't it be suitable to pass
emptyTxnID
toresetExtraTxnState
here, particularly since we're passing innoEvent
?
Good point. Done.
pkg/sql/conn_executor_exec.go, line 1932 at r16 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
seeing how much plumbing it required to take this
prevTxnID
here, I think a better option would have been to turntxnEvent
into atype txnEvent struct { eventType <enum> txnID UUID }
I would also consider making the
txnID
optional and not passing it in in the closing cases. Just don't collect the data for those transactions; if the connection drops in the middle of a transaction, I doubt anybody will be interested in those fingerprints. Like, undo the changes tofinishExternalTxn()
- remove the return value.
Done. Also not passing txnID
in the closing case.
pkg/sql/conn_fsm.go, line 514 at r16 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
would anything go wrong if you blindly filled in the
txnID
, even fornoEvent
? If you can get away with it, I'd consider doing it to get rid of the need for this comment.
Done. Filling in txnID
unconditionally.
pkg/sql/txn_state.go, line 151 at r16 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
s/This function//
And move this above the parameters.
Done.
pkg/sql/txn_state.go, line 209 at r16 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
why do we need the
txnID
variable?
This is because here we are reading ts.mu.txn.ID()
while we are holding the mutex. We need to have access to this variable to return after we release the mutex a few lines later.
pkg/sql/txn_state.go, line 280 at r16 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
I think because of this transition, no?
Looks like thisfinishExternalTxn()
is called regardless of whether the txn had been previously finished.
Ah makes sense.
5d79072
to
f44c694
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
for the changes around the conn_executor
Reviewable status: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @andreimatei, @Azhng, @maryliag, and @nvanbenschoten)
pkg/sql/conn_executor_exec.go, line 1982 at r19 (raw file):
} func (ex *connExecutor) recordTransaction(
not your code, but pls take the opportunity to put a comment on this method. Also explain the ev
parameter.
We have a recordTransactionStart()
; should this one be called recordTransactionFinish()
? If so, and these methods are related, consider also putting their code next to each other in a new commit.
pkg/sql/txn_state.go, line 209 at r16 (raw file):
Previously, Azhng (Archer Zhang) wrote…
This is because here we are reading
ts.mu.txn.ID()
while we are holding the mutex. We need to have access to this variable to return after we release the mutex a few lines later.
I see.
pkg/sql/txn_state.go, line 360 at r19 (raw file):
// with a SQL transaction (other than the state encapsulated in TxnState; e.g. // schema changes and portals). eventTyp txnEventType
a particular reason why this is not called eventType
?
pkg/sql/txn_state.go, line 362 at r19 (raw file):
eventTyp txnEventType // txnID is filled when transaction starts, commits or aborts.
when a transaction
Previously, it was impossible to correlate an individual execution of a transaction (identified via transaction ID) to this historical execution statistics (identified via transaction fingerprint ID). This commit introduces Transaction ID Cache (TxnIDCache), a FIFO cache that stores the mapping from transaction ID to transaction fingerprint ID. This buffer records the mapping at the end of the transaction execution. The oldest entry in the buffer will be evicted through FIFO policy. The default size of this Transaction ID Cache is capped at 64 MB and it is configurable via the sql.contention.txn_id_cache.max_size cluster setting. Release note (sql change): Transaction ID to Transaction Fingerprint ID mapping is now stored in the new Transaction ID Cache, a FIFO unordered in-memory buffer. The size of the buffer is 64 MB by default and configurable via sql.contention.txn_id_cache.max_size cluster setting. Consequentially, two additioanl metrics are introduced: * sql.contention.txn_id_cache.size: tracks the current memory usage of transaction ID Cache * sql.contention.txn_id_cache.discarded_count: number of resolved transaction IDs that are dropped due to memory constraints.
Previously, we had a pair of functions called recordTransactionStart() and recordTransaction(). These two functions were called at the beginning and the end of the a transaction. This commit renamed recordTransaction() to recordTransactionFinish() to reflect the time in which this function is called, and also moved this function right next to recordTransactionStart(). Release note: None
f44c694
to
a545cdf
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ajwerner do you mind to take another look? 🙏
Reviewable status: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @ajwerner, @andreimatei, @maryliag, and @nvanbenschoten)
pkg/sql/conn_executor_exec.go, line 1982 at r19 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
not your code, but pls take the opportunity to put a comment on this method. Also explain the
ev
parameter.
We have arecordTransactionStart()
; should this one be calledrecordTransactionFinish()
? If so, and these methods are related, consider also putting their code next to each other in a new commit.
Done. Moved the code in the new commit.
pkg/sql/txn_state.go, line 360 at r19 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
a particular reason why this is not called
eventType
?
Done. Changed to eventType
.
pkg/sql/txn_state.go, line 362 at r19 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
when a transaction
Done.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @ajwerner, @andreimatei, @maryliag, and @nvanbenschoten)
Thank you everyone for the detailed reviews! bors r+ |
Build succeeded: |
Previous PR: #74114
Previously, it was impossible to correlate an individual execution
of a transaction (identified via transaction ID) to this historical
execution statistics (identified via transaction fingerprint ID).
This commit introduces Transaction ID Cache (TxnIDCache), a FIFO
cache that stores the mapping from transaction ID to transaction
fingerprint ID. This buffer records the mapping at the
end of the transaction execution. The oldest entry in the buffer
will be evicted through FIFO policy. The default size of this
Transaction ID Cache is capped at 64 MB and it is configurable via
the sql.contention.txn_id_cache.max_size cluster setting.
Release note (sql change): Transaction ID to Transaction Fingerprint
ID mapping is now stored in the new Transaction ID Cache, a FIFO
unordered in-memory buffer. The size of the buffer is 64 MB by default
and configurable via sql.contention.txn_id_cache.max_size cluster
setting. Consequentially, two additioanl metrics are introduced:
of transaction ID Cache
transaction IDs that are dropped due to memory constraints.