Skip to content

Commit

Permalink
roachpb: redefine Transaction.IsWriting to IsLocking
Browse files Browse the repository at this point in the history
This is the first of a few terminology updates. Before this change, "writing"
and "locking" were synonymous so we used the concepts interchangeably. Now
that we plan to introduce "read-only" requests that acquire unreplicated
locks, we need to split these concepts. This commit starts this process by
renaming `Transaction.IsWriting` to `Transaction.IsLocking`, which is the
concept that the method was intending to talk about.
  • Loading branch information
nvanbenschoten committed Mar 5, 2020
1 parent e630928 commit f048094
Show file tree
Hide file tree
Showing 29 changed files with 100 additions and 98 deletions.
6 changes: 5 additions & 1 deletion pkg/ccl/followerreadsccl/followerreads.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,11 @@ func batchCanBeEvaluatedOnFollower(ba roachpb.BatchRequest) bool {
// txnCanPerformFollowerRead determines if the provided transaction can perform
// follower reads.
func txnCanPerformFollowerRead(txn *roachpb.Transaction) bool {
return txn != nil && !txn.IsWriting()
// If the request is transactional and that transaction has acquired any
// locks then that request should not perform follower reads. Doing so could
// allow the request to miss its own writes or observe state that conflicts
// with its locks.
return txn != nil && !txn.IsLocking()
}

// canUseFollowerRead determines if a query can be sent to a follower.
Expand Down
2 changes: 1 addition & 1 deletion pkg/internal/client/txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func TestTxnSnowballTrace(t *testing.T) {
// dump:
// 0.105ms 0.000ms event:inside txn
// 0.275ms 0.171ms event:client.Txn did AutoCommit. err: <nil>
//txn: "internal/client/txn_test.go:67 TestTxnSnowballTrace" id=<nil> key=/Min rw=false pri=0.00000000 iso=SERIALIZABLE stat=COMMITTED epo=0 ts=0.000000000,0 orig=0.000000000,0 max=0.000000000,0 wto=false rop=false
//txn: "internal/client/txn_test.go:67 TestTxnSnowballTrace" id=<nil> key=/Min lock=false pri=0.00000000 iso=SERIALIZABLE stat=COMMITTED epo=0 ts=0.000000000,0 orig=0.000000000,0 max=0.000000000,0 wto=false rop=false
// 0.278ms 0.173ms event:txn complete
found, err := regexp.MatchString(
// The (?s) makes "." match \n. This makes the test resilient to other log
Expand Down
16 changes: 7 additions & 9 deletions pkg/kv/kvserver/replica_evaluate.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,15 +168,13 @@ func evaluateBatch(
if baHeader.Txn != nil {
baHeader.Txn = baHeader.Txn.Clone()

// Check whether this transaction has been aborted, if applicable.
// This applies to writes that leave intents (the use of the
// IsTransactionWrite flag excludes operations like HeartbeatTxn),
// and reads that occur in a transaction that has already written
// (see #2231 for more about why we check for aborted transactions
// on reads). Note that 1PC transactions have had their
// transaction field cleared by this point so we do not execute
// this check in that case.
if ba.IsTransactionWrite() || baHeader.Txn.IsWriting() {
// Check whether this transaction has been aborted, if applicable. This
// applies to reads and writes once a transaction that has begun to
// acquire locks (see #2231 for more about why we check for aborted
// transactions on reads). Note that 1PC transactions have had their
// transaction field cleared by this point so we do not execute this
// check in that case.
if baHeader.Txn.IsLocking() {
// We don't check the abort span for a couple of special requests:
// - if the request is asking to abort the transaction, then don't check the
// AbortSpan; we don't want the request to be rejected if the transaction
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_follower_read.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ func (r *Replica) canServeFollowerRead(
if lErr, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); ok &&
lErr.LeaseHolder != nil && lErr.Lease.Type() == roachpb.LeaseEpoch &&
ba.IsAllTransactional() && // followerreadsccl.batchCanBeEvaluatedOnFollower
(ba.Txn == nil || !ba.Txn.IsWriting()) && // followerreadsccl.txnCanPerformFollowerRead
(ba.Txn == nil || !ba.Txn.IsLocking()) && // followerreadsccl.txnCanPerformFollowerRead
FollowerReadsEnabled.Get(&r.store.cfg.Settings.SV) {

ts := ba.Timestamp
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/txnwait/queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -473,7 +473,7 @@ func (q *Queue) MaybeWaitForPush(
var readyCh chan struct{} // signaled when pusher txn should be queried

// Query the pusher if it's a valid read-write transaction.
if req.PusherTxn.ID != uuid.Nil && req.PusherTxn.IsWriting() {
if req.PusherTxn.ID != uuid.Nil && req.PusherTxn.IsLocking() {
// Create a context which will be canceled once this call completes.
// This ensures that the goroutine created to query the pusher txn
// is properly cleaned up.
Expand Down
14 changes: 7 additions & 7 deletions pkg/roachpb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -1077,9 +1077,9 @@ func (t *Transaction) UpgradePriority(minPriority enginepb.TxnPriority) {
}
}

// IsWriting returns whether the transaction has begun writing intents.
// This method will never return true for a read-only transaction.
func (t *Transaction) IsWriting() bool {
// IsLocking returns whether the transaction has begun acquiring locks.
// This method will never return false for a writing transaction.
func (t *Transaction) IsLocking() bool {
return t.Key != nil
}

Expand All @@ -1091,8 +1091,8 @@ func (t Transaction) String() string {
if len(t.Name) > 0 {
fmt.Fprintf(&buf, "%q ", t.Name)
}
fmt.Fprintf(&buf, "meta={%s} rw=%t stat=%s rts=%s wto=%t max=%s",
t.TxnMeta, t.IsWriting(), t.Status, t.ReadTimestamp, t.WriteTooOld, t.MaxTimestamp)
fmt.Fprintf(&buf, "meta={%s} lock=%t stat=%s rts=%s wto=%t max=%s",
t.TxnMeta, t.IsLocking(), t.Status, t.ReadTimestamp, t.WriteTooOld, t.MaxTimestamp)
if ni := len(t.IntentSpans); t.Status != PENDING && ni > 0 {
fmt.Fprintf(&buf, " int=%d", ni)
}
Expand All @@ -1114,8 +1114,8 @@ func (t Transaction) SafeMessage() string {
if len(t.Name) > 0 {
fmt.Fprintf(&buf, "%q ", t.Name)
}
fmt.Fprintf(&buf, "meta={%s} rw=%t stat=%s rts=%s wto=%t max=%s",
t.TxnMeta.SafeMessage(), t.IsWriting(), t.Status, t.ReadTimestamp, t.WriteTooOld, t.MaxTimestamp)
fmt.Fprintf(&buf, "meta={%s} lock=%t stat=%s rts=%s wto=%t max=%s",
t.TxnMeta.SafeMessage(), t.IsLocking(), t.Status, t.ReadTimestamp, t.WriteTooOld, t.MaxTimestamp)
if ni := len(t.IntentSpans); t.Status != PENDING && ni > 0 {
fmt.Fprintf(&buf, " int=%d", ni)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/roachpb/string_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ func TestTransactionString(t *testing.T) {
MaxTimestamp: hlc.Timestamp{WallTime: 40, Logical: 41},
}
expStr := `"name" meta={id=d7aa0f5e key="foo" pri=44.58039917 epo=2 ts=0.000000020,21 min=0.000000010,11 seq=15}` +
` rw=true stat=COMMITTED rts=0.000000030,31 wto=false max=0.000000040,41`
` lock=true stat=COMMITTED rts=0.000000030,31 wto=false max=0.000000040,41`

if str := txn.String(); str != expStr {
t.Errorf(
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/testdata/mvcc_histories/clear_range
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ with t=A v=abc resolve
put k=c
----
>> at end:
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000044,0 min=0,0 seq=0} rw=true stat=PENDING rts=0.000000044,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000044,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000044,0 wto=false max=0,0
data: "a"/0.000000044,0 -> /BYTES/abc
data: "a/123"/0.000000044,0 -> /BYTES/abc
data: "b"/0.000000044,0 -> /BYTES/abc
Expand Down
14 changes: 7 additions & 7 deletions pkg/storage/testdata/mvcc_histories/conditional_put_with_txn
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ run ok
txn_begin t=A ts=123
----
>> at end:
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=0} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000123,0 wto=false max=0,0

# Write value1.

Expand All @@ -12,7 +12,7 @@ with t=A
cput k=k v=v
----
>> at end:
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=1} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000123,0 wto=false max=0,0
meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=1} ts=0.000000123,0 del=false klen=12 vlen=6
data: "k"/0.000000123,0 -> /BYTES/v

Expand All @@ -25,7 +25,7 @@ with t=A
cput k=k v=v2 cond=v
----
>> at end:
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=2} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000123,0 wto=false max=0,0
meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=2} ts=0.000000123,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}}
data: "k"/0.000000123,0 -> /BYTES/v2

Expand All @@ -38,7 +38,7 @@ with t=A
cput k=k v=v3
----
>> at end:
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000123,0 min=0,0 seq=1} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000123,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000123,0 wto=false max=0,0
meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000123,0 min=0,0 seq=1} ts=0.000000123,0 del=false klen=12 vlen=7
data: "k"/0.000000123,0 -> /BYTES/v3

Expand Down Expand Up @@ -86,7 +86,7 @@ with t=A
cput k=c v=cput cond=value
----
>> at end:
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=1} rw=true stat=PENDING rts=0.000000002,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0
meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=1} ts=0.000000002,0 del=false klen=12 vlen=9
data: "c"/0.000000002,0 -> /BYTES/cput
data: "c"/0.000000001,0 -> /BYTES/value
Expand All @@ -100,9 +100,9 @@ with t=A
cput k=c v=cput cond=value
----
>> txn_restart ts=3 t=A
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=0} rw=true stat=PENDING rts=0.000000003,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000003,0 wto=false max=0,0
>> txn_step t=A
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=1} rw=true stat=PENDING rts=0.000000003,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000003,0 wto=false max=0,0
>> cput k=c v=cput cond=value t=A
meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=9
data: "c"/0.000000003,0 -> /BYTES/cput
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ with t=a
cput k=k v=v2 cond=v1
----
>> at end:
txn: "a" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=0} rw=true stat=PENDING rts=0.000000001,0 wto=false max=0,0
txn: "a" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0
data: "k"/0.000000010,1 -> /BYTES/v2
data: "k"/0.000000010,0 -> /BYTES/v1
error: (*roachpb.ConditionFailedError:) unexpected value: <nil>
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/testdata/mvcc_histories/empty_key
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,6 @@ txn_begin t=A
resolve_intent t=A k=
----
>> at end:
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0,0 min=0,0 seq=0} rw=true stat=PENDING rts=0,0 wto=false max=0,0
txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0,0 min=0,0 seq=0} lock=true stat=PENDING rts=0,0 wto=false max=0,0
<no data>
error: (*errors.fundamental:) attempted access to empty key
10 changes: 5 additions & 5 deletions pkg/storage/testdata/mvcc_histories/idempotent_transactions
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@ with t=a k=a
put v=first
----
>> at end:
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} ts=0.000000011,0 del=false klen=12 vlen=10
data: "a"/0.000000011,0 -> /BYTES/first

Expand Down Expand Up @@ -36,7 +36,7 @@ with t=a k=a
----
meta: "a" -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}}
>> at end:
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}}
data: "a"/0.000000011,0 -> /BYTES/second

Expand All @@ -47,7 +47,7 @@ with t=a k=a
put v=second
----
>> at end:
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=-1} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=-1} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}}
data: "a"/0.000000011,0 -> /BYTES/second
error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 1 missing an intent with lower sequence -1
Expand All @@ -67,7 +67,7 @@ inc: current value = 1
inc: current value = 1
inc: current value = 1
>> at end:
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=2} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}}
data: "a"/0.000000011,0 -> /BYTES/second
meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=2} ts=0.000000011,0 del=false klen=12 vlen=6
Expand All @@ -89,7 +89,7 @@ inc: current value = 1
inc: current value = 1
inc: current value = 1
>> at end:
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=2} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0
meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}}
data: "a"/0.000000011,0 -> /BYTES/second
meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=3} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{2 /INT/1}}
Expand Down
Loading

0 comments on commit f048094

Please sign in to comment.