Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
104782: concurrency: do not clear lock holders in tryActiveWait r=nvanbenschoten a=arulajmani

Previous to this patch, tryActiveWait would clear a lock's holder if it belonged to a finalized transaction and was only held with unreplicated durability. It would also nudge some request's in the lock's wait queue to proceed. This state transition inside of tryActiveWait for other requests was subtle. Arguably, the function which decides whether to wait on a lock or not should not be responsible for performing such state transitions for other requests.

This patch slightly improves this situation. We no longer return a transitionedToFree boolean and expect the caller to take some action based on it. Instead, we accumulate unreplicated locks that belong to finalized transactions in the request guard, as it sequences. The request then assumes the responsibility for clearing such locks and nudging waiters (if possible).

This last part is still not great. However, we need it for practical purposes. In the ideal world, we would address the TODO in TransactionIsFinalized, and perform this state transition there. But until then, this patch moves the needle slightly. More importantly, it allows us to remove some special casing when tryActiveWait is removed and replaced, over in #104620. In particular, locking requests that come across unreplicated locks that belong to finalized transactions and have empty wait queues will be able to acquire claims on such locks before proceeding. This can be seen in the test diff for
`clear_finalized_txn_locks`.

Release note: None

105146: multitenant: make the system tenant appear to have all capabilities r=yuzefovich a=knz

Epic: CRDB-26691
Fixes #98749.

The system tenant is currently defined to have access to all services.
Yet, the output of `SHOW TENANT system WITH CAPABILITIES` suggested
that was not true.

This patch fixes that.

105157: ui: improve timeperiod display r=maryliag a=maryliag

Previously, the period being shown on SQL Activity page could be confusing, since we no longer refresh the page automatically. This could result in a scenario where a query is executed, the user click on Apply on the Search Criteria on X:05, but the page shows that the results goes up to X:59, but then if you executed new statements they won't show until Apply is clicked again, but because we still show the message that the results are up to X:59 this is misleading.
This commit updates the value being displayed to use the time the request was made, so we know the end window value, and even if the user changes page and go back, the value is still the X:05, making more obvious they need to click on Apply again if they want to see newer results.

Here an example of the previous behaviour on Transactions page and the new Behaviour on Statement page:
(to clarify, this PR make this update on Statement, Statement Details, Transaction and Transaction Details pages)
https://www.loom.com/share/ec19aa79a5144aea9e44bec59a5101a4

Epic: none
Release note: None

Co-authored-by: Arul Ajmani <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: maryliag <[email protected]>
  • Loading branch information
4 people committed Jun 21, 2023
4 parents 4240e46 + 85f99fd + 7ab3e49 + d74648f commit 7cb2a89
Show file tree
Hide file tree
Showing 17 changed files with 378 additions and 74 deletions.
20 changes: 20 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/tenant_capability
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,26 @@ can_view_tsdb_metrics false
exempt_from_rate_limiting false
span_config_bounds {}

subtest end

subtest regression_98749

query TT colnames,rowsort
SELECT capability_name, capability_value FROM [SHOW TENANT system WITH CAPABILITIES]
----
capability_name capability_value
can_admin_relocate_range true
can_admin_scatter true
can_admin_split true
can_admin_unsplit true
can_check_consistency true
can_use_nodelocal_storage true
can_view_node_info true
can_view_tsdb_metrics true
exempt_from_rate_limiting true
span_config_bounds {}


subtest end


Expand Down
14 changes: 7 additions & 7 deletions pkg/configprofiles/testdata/multitenant-app
Original file line number Diff line number Diff line change
Expand Up @@ -49,15 +49,15 @@ ORDER BY tenant_id, name
system-sql
SHOW TENANTS WITH CAPABILITIES
----
1 system ready shared can_admin_relocate_range false
1 system ready shared can_admin_relocate_range true
1 system ready shared can_admin_scatter true
1 system ready shared can_admin_split true
1 system ready shared can_admin_unsplit false
1 system ready shared can_check_consistency false
1 system ready shared can_use_nodelocal_storage false
1 system ready shared can_view_node_info false
1 system ready shared can_view_tsdb_metrics false
1 system ready shared exempt_from_rate_limiting false
1 system ready shared can_admin_unsplit true
1 system ready shared can_check_consistency true
1 system ready shared can_use_nodelocal_storage true
1 system ready shared can_view_node_info true
1 system ready shared can_view_tsdb_metrics true
1 system ready shared exempt_from_rate_limiting true
1 system ready shared span_config_bounds {}
2 template ready none can_admin_relocate_range true
2 template ready none can_admin_scatter true
Expand Down
14 changes: 7 additions & 7 deletions pkg/configprofiles/testdata/multitenant-noapp
Original file line number Diff line number Diff line change
Expand Up @@ -42,15 +42,15 @@ ORDER BY tenant_id, name
system-sql
SHOW TENANTS WITH CAPABILITIES
----
1 system ready shared can_admin_relocate_range false
1 system ready shared can_admin_relocate_range true
1 system ready shared can_admin_scatter true
1 system ready shared can_admin_split true
1 system ready shared can_admin_unsplit false
1 system ready shared can_check_consistency false
1 system ready shared can_use_nodelocal_storage false
1 system ready shared can_view_node_info false
1 system ready shared can_view_tsdb_metrics false
1 system ready shared exempt_from_rate_limiting false
1 system ready shared can_admin_unsplit true
1 system ready shared can_check_consistency true
1 system ready shared can_use_nodelocal_storage true
1 system ready shared can_view_node_info true
1 system ready shared can_view_tsdb_metrics true
1 system ready shared exempt_from_rate_limiting true
1 system ready shared span_config_bounds {}
2 template ready none can_admin_relocate_range true
2 template ready none can_admin_scatter true
Expand Down
106 changes: 68 additions & 38 deletions pkg/kv/kvserver/concurrency/lock_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -480,7 +480,24 @@ type lockTableGuardImpl struct {
// Locks to resolve before scanning again. Doesn't need to be protected by
// mu since should only be read after the caller has already synced with mu
// in realizing that it is doneWaiting.
//
// toResolve should only include replicated locks; for unreplicated locks,
// toResolveUnreplicated is used instead.
toResolve []roachpb.LockUpdate

// toResolveUnreplicated is a list of locks (only held with durability
// unreplicated) that are known to belong to finalized transactions. Such
// locks may be cleared from the lock table (and some requests queueing in the
// lock's wait queue may be able to proceed). If set, the request should
// perform these actions on behalf of the lock table, either before proceeding
// to evaluation, or before waiting on a conflicting lock.
//
// TODO(arul): We need to push the responsibility of doing so on to a request
// because TransactionIsFinalized does not take proactive action. If we
// addressed the TODO in TransactionIsFinalized, and taught it to take action
// on locks belonging to finalized transactions, we wouldn't need to bother
// scanning requests.
toResolveUnreplicated []roachpb.LockUpdate
}

var _ lockTableGuard = &lockTableGuardImpl{}
Expand Down Expand Up @@ -727,6 +744,15 @@ func (g *lockTableGuardImpl) isSameTxnAsReservation(ws waitingState) bool {
return !ws.held && g.isSameTxn(ws.txn)
}

// takeToResolveUnreplicated returns the list of unreplicated locks accumulated
// by the guard for resolution. Ownership, and responsibility to resolve these
// locks, is passed to the caller.
func (g *lockTableGuardImpl) takeToResolveUnreplicated() []roachpb.LockUpdate {
toResolveUnreplicated := g.toResolveUnreplicated
g.toResolveUnreplicated = nil
return toResolveUnreplicated
}

// resumeScan resumes the request's (receiver's) scan of the lock table. The
// scan continues until either all overlapping locks in the lock table have been
// considered and no conflict is found, or until the request encounters a lock
Expand All @@ -750,12 +776,19 @@ func (g *lockTableGuardImpl) resumeScan(notify bool) {
span = &spans[g.index]
resumingInSameSpan = true
}
// Locks that transition to free because of the txnStatusCache are GC'd
// before returning. Note that these are only unreplicated locks. Replicated
// locks are handled via the g.toResolve.
var locksToGC []*lockState
defer func() {
g.lt.tryGCLocks(&g.lt.locks, locksToGC)
// Eagerly update any unreplicated locks that are known to belong to
// finalized transactions. We do so regardless of whether this request can
// proceed to evaluation or needs to wait at some conflicting lock.
//
// Note that replicated locks are handled differently, using the g.toResolve
// slice. Additionally, they're only resolved when a request is done
// waiting and can proceed to evaluation.
if toResolveUnreplicated := g.takeToResolveUnreplicated(); len(toResolveUnreplicated) > 0 {
for i := range toResolveUnreplicated {
g.lt.updateLockInternal(&toResolveUnreplicated[i])
}
}
}()

for span != nil {
Expand All @@ -782,10 +815,7 @@ func (g *lockTableGuardImpl) resumeScan(notify bool) {
// Else, past the lock where it stopped waiting. We may not
// encounter that lock since it may have been garbage collected.
}
wait, transitionedToFree := l.tryActiveWait(g, g.str, notify, g.lt.clock)
if transitionedToFree {
locksToGC = append(locksToGC, l)
}
wait := l.tryActiveWait(g, g.str, notify, g.lt.clock)
if wait {
return
}
Expand Down Expand Up @@ -1666,7 +1696,7 @@ func (l *lockState) clearLockHolder() {
// Acquires l.mu, g.mu.
func (l *lockState) tryActiveWait(
g *lockTableGuardImpl, str lock.Strength, notify bool, clock *hlc.Clock,
) (wait bool, transitionedToFree bool) {
) (wait bool) {
l.mu.Lock()
defer l.mu.Unlock()

Expand All @@ -1678,31 +1708,25 @@ func (l *lockState) tryActiveWait(

// It is possible that this lock is empty and has not yet been deleted.
if l.isEmptyLock() {
return false, false
return false
}

// Lock is not empty.
lockHolderTxn, lockHolderTS := l.getLockHolder()
if lockHolderTxn != nil && g.isSameTxn(lockHolderTxn) {
// Already locked by this txn.
return false, false
return false
}

var replicatedLockFinalizedTxn *roachpb.Transaction
var unreplicatedLockFinalizedTxn *roachpb.Transaction
if lockHolderTxn != nil {
finalizedTxn, ok := g.lt.txnStatusCache.finalizedTxns.get(lockHolderTxn.ID)
if ok {
if l.holder.holder[lock.Replicated].txn == nil {
// Only held unreplicated. Release immediately.
up := roachpb.MakeLockUpdate(finalizedTxn, roachpb.Span{Key: l.key})
_, gc := l.tryUpdateLockLocked(up)
if gc {
// Empty lock.
return false, true
}
lockHolderTxn = nil
// There is a reservation holder, which may be the caller itself,
// so fall through to the processing below.
g.toResolveUnreplicated = append(
g.toResolveUnreplicated, roachpb.MakeLockUpdate(finalizedTxn, roachpb.Span{Key: l.key}))
unreplicatedLockFinalizedTxn = finalizedTxn
} else {
replicatedLockFinalizedTxn = finalizedTxn
}
Expand All @@ -1712,13 +1736,13 @@ func (l *lockState) tryActiveWait(
if str == lock.None {
if lockHolderTxn == nil {
// Non locking reads only care about locks, not reservations.
return false, false
return false
}
// Locked by some other txn.
// TODO(arul): this will need to change once we start supporting different
// lock strengths.
if g.ts.Less(lockHolderTS) {
return false, false
return false
}

// If the non-locking reader is reading at a higher timestamp than the lock
Expand All @@ -1739,18 +1763,20 @@ func (l *lockState) tryActiveWait(
if ok && g.ts.Less(pushedTxn.WriteTimestamp) {
up := roachpb.MakeLockUpdate(pushedTxn, roachpb.Span{Key: l.key})
if l.holder.holder[lock.Replicated].txn == nil {
// Only held unreplicated. Update lock directly in case other
// waiting readers can benefit from the pushed timestamp.
// Only held unreplicated. Accumulate a unreplicated lock update in
// case any other waiting readers can benefit from the pushed
// timestamp.
//
// TODO(arul): this case is only possible while non-locking reads
// block on Exclusive locks. Once non-locking reads start only
// blocking on intents, it can be removed and asserted against.
_, _ = l.tryUpdateLockLocked(up)
g.toResolveUnreplicated = append(
g.toResolveUnreplicated, up)
} else {
// Resolve to push the replicated intent.
g.toResolve = append(g.toResolve, up)
}
return false, false
return false
}
}

Expand Down Expand Up @@ -1789,23 +1815,23 @@ func (l *lockState) tryActiveWait(
// and reservation holders anyway, so I'm not entirely sure what we get by
// storing them in the same queue as locking requests.
if alsoLocksWithHigherStrength {
return false, false
return false
}
}

if !l.holder.locked && l.queuedWriters.Len() > 0 {
qg := l.queuedWriters.Front().Value.(*queuedGuard)
if qg.guard == g {
// Already claimed by this request.
return false, false
return false
}
// A non-transactional write request never makes or breaks claims, and only
// waits for a claim if the claim holder has a lower seqNum. Note that `str
// == lock.None && lockHolderTxn == nil` was already checked above.
if g.txn == nil && qg.guard.seqNum > g.seqNum {
// Claimed by a request with a higher seqNum and g is a non-transactional
// request. Ignore the claim.
return false, false
return false
}
}

Expand Down Expand Up @@ -1862,7 +1888,7 @@ func (l *lockState) tryActiveWait(
g.maybeUpdateWaitingStateLocked(state, notify)
// NOTE: we return wait=true not because the request is waiting, but
// because it should not continue scanning for conflicting locks.
return true, false
return true
} else {
var e *list.Element
for e = l.queuedWriters.Back(); e != nil; e = e.Prev() {
Expand All @@ -1880,16 +1906,20 @@ func (l *lockState) tryActiveWait(
g.mu.locks[l] = struct{}{}
waitForState.queuedWriters = l.queuedWriters.Len() // update field
}
if (replicatedLockFinalizedTxn != nil || !l.holder.locked) && l.queuedWriters.Front().Value.(*queuedGuard) == qg {
if (replicatedLockFinalizedTxn != nil ||
unreplicatedLockFinalizedTxn != nil ||
!l.holder.locked) &&
l.queuedWriters.Front().Value.(*queuedGuard) == qg {
_ = unreplicatedLockFinalizedTxn
// First waiter, so should not wait. NB: this inactive waiter can be
// non-transactional.
qg.active = false
wait = false
}
} else {
if replicatedLockFinalizedTxn != nil {
// Don't add to waitingReaders since all readers in waitingReaders are
// active waiters, and this request is not an active waiter here.
if replicatedLockFinalizedTxn != nil || unreplicatedLockFinalizedTxn != nil {
// Non-locking readers do not wait on finalized {replicated,unreplicated}
// locks.
wait = false
} else {
l.waitingReaders.PushFront(g)
Expand All @@ -1902,7 +1932,7 @@ func (l *lockState) tryActiveWait(
g.toResolve = append(
g.toResolve, roachpb.MakeLockUpdate(replicatedLockFinalizedTxn, roachpb.Span{Key: l.key}))
}
return false, false
return false
}
// Make it an active waiter.
g.key = l.key
Expand All @@ -1920,7 +1950,7 @@ func (l *lockState) tryActiveWait(
}
g.maybeUpdateWaitingStateLocked(state, notify)
}
return true, false
return true
}

func (l *lockState) isNonConflictingLock(g *lockTableGuardImpl, str lock.Strength) bool {
Expand Down
Loading

0 comments on commit 7cb2a89

Please sign in to comment.