Skip to content

Commit

Permalink
Merge #46858
Browse files Browse the repository at this point in the history
46858: release-20.1: kvserver: disallow timestamp regression with locking scans r=irfansharif a=irfansharif

Backport 1/1 commits from #46750.

/cc @cockroachdb/release

---

After #46004, locking scans can now hit WriteTooOld errors if they
encounter values at timestamps higher than their read timestamps. The
`ActualTimestamp` recorded is the `encountered ts + 1`. When determining
what the new timestamp for the txn should be, previously we blindly used
the generated `encountered ts + 1`. This was buggy, and could lead to a
timestamp regression in the case where a txn with (read_ts, write_ts) =
(1, 4) finds a value with `ts = 2`. If we try to "bump" the txn to
`ts = 3`, we're regressing the write ts.

Now, when determining what the new timestamp should be, we ensure we use
`max(encountered ts + 1, txn's current write ts)`.

Fixes #43273.

Release note: None


Co-authored-by: irfan sharif <[email protected]>
  • Loading branch information
craig[bot] and irfansharif committed Apr 1, 2020
2 parents bcca546 + 2dee6d7 commit 01409a8
Show file tree
Hide file tree
Showing 6 changed files with 39 additions and 7 deletions.
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/scaledata.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ func runSqlapp(ctx context.Context, t *test, c *cluster, app, flags string, dur
}
m.Go(func(ctx context.Context) error {
t.Status("installing schema")
err = c.RunE(ctx, appNode, fmt.Sprintf("./%s --install_schema "+
err := c.RunE(ctx, appNode, fmt.Sprintf("./%s --install_schema "+
"--cockroach_ip_addresses_csv='%s' %s", app, addrStr, flags))
if err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/concurrency/lock_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ add-discovered r=<name> k=<key> txn=<name>
----
<error string>
Adds a discovered lock that is disovered by the named request.
Adds a discovered lock that is discovered by the named request.
dequeue r=<name>
----
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_batch_updates.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ func tryBumpBatchTimestamp(
ts, ba.Txn.ReadTimestamp, ba.Txn.WriteTimestamp)
ba.Txn = ba.Txn.Clone()
ba.Txn.ReadTimestamp = ts
ba.Txn.WriteTimestamp = ba.Timestamp
ba.Txn.WriteTimestamp = ts
ba.Txn.WriteTooOld = false
return true
}
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/replica_evaluate.go
Original file line number Diff line number Diff line change
Expand Up @@ -548,7 +548,16 @@ func canDoServersideRetry(
if pErr != nil {
switch tErr := pErr.GetDetail().(type) {
case *roachpb.WriteTooOldError:
// Locking scans hit WriteTooOld errors if they encounter values at
// timestamps higher than their read timestamps. The encountered
// timestamps are guaranteed to be greater than the txn's read
// timestamp, but not its write timestamp. So, when determining what
// the new timestamp should be, we make sure to not regress the
// txn's write timestamp.
newTimestamp = tErr.ActualTimestamp
if ba.Txn != nil {
newTimestamp.Forward(pErr.GetTxn().WriteTimestamp)
}
case *roachpb.TransactionRetryError:
if ba.Txn == nil {
// TODO(andrei): I don't know if TransactionRetryError is possible for
Expand Down
23 changes: 23 additions & 0 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10035,6 +10035,29 @@ func TestReplicaServersideRefreshes(t *testing.T) {
return
},
},
// Regression test for #43273. When locking scans run into write too old
// errors, the refreshed timestamp should not be below the txn's
// existing write timestamp.
{
name: "serverside-refresh with write too old errors during locking scan",
setupFn: func() (hlc.Timestamp, error) {
return put("lscan", "put")
},
batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) {
// Txn with (read_ts, write_ts) = (1, 4) finds a value with
// `ts = 2`. Final timestamp should be `ts = 4`.
ba.Txn = newTxn("lscan", ts.Prev())
ba.Txn.WriteTimestamp = ts.Next().Next()
ba.CanForwardReadTimestamp = true

expTS = ba.Txn.WriteTimestamp

scan := scanArgs(roachpb.Key("lscan"), roachpb.Key("lscan\x00"))
scan.KeyLocking = lock.Upgrade
ba.Add(scan)
return
},
},
// Serializable transaction will commit with WriteTooOld flag if no refresh spans.
{
name: "serializable commit with write-too-old flag",
Expand Down
8 changes: 4 additions & 4 deletions pkg/storage/enginepb/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,10 @@ import (
// retries, it increments its epoch, invalidating all of its previous writes.
type TxnEpoch int32

// TxnSeq is a zero-indexed sequence number asssigned to a a request performed
// by a transaction. Writes within a transaction have unique sequences and start
// at sequence number 1. Reads within a transaction have non-unique sequences
// and start at sequence number 0.
// TxnSeq is a zero-indexed sequence number assigned to a request performed by a
// transaction. Writes within a transaction have unique sequences and start at
// sequence number 1. Reads within a transaction have non-unique sequences and
// start at sequence number 0.
//
// Writes within a transaction logically take place in sequence number order.
// Reads within a transaction observe only writes performed by the transaction
Expand Down

0 comments on commit 01409a8

Please sign in to comment.