Skip to content

Commit

Permalink
kv: alter error for SET TRANSACTION AS OF SYSTEM TIME
Browse files Browse the repository at this point in the history
if reads or writes  are already performed

When a txn performed a read or write before setting up a historical timestamp a crash
report was generated. This commit handles the error case.

Release note: None
  • Loading branch information
e-mbrown committed May 18, 2022
1 parent 2c1ac2d commit a1e7e05
Show file tree
Hide file tree
Showing 6 changed files with 82 additions and 0 deletions.
14 changes: 14 additions & 0 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -1406,3 +1406,17 @@ func (tc *TxnCoordSender) ClearTxnRetryableErr(ctx context.Context) {
tc.mu.txnState = txnPending
}
}

// HasPerformedReads is part of the TxnSender interface.
func (tc *TxnCoordSender) HasPerformedReads() bool {
tc.mu.Lock()
defer tc.mu.Unlock()
return !tc.interceptorAlloc.txnSpanRefresher.refreshFootprint.empty()
}

// HasPerformedWrites is part of the TxnSender interface.
func (tc *TxnCoordSender) HasPerformedWrites() bool {
tc.mu.Lock()
defer tc.mu.Unlock()
return tc.mu.txn.Sequence != 0
}
10 changes: 10 additions & 0 deletions pkg/kv/mock_transactional_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,6 +231,16 @@ func (m *MockTransactionalSender) GetTxnRetryableErr(
func (m *MockTransactionalSender) ClearTxnRetryableErr(ctx context.Context) {
}

// HasPerformedReads is part of TxnSenderFactory.
func (m *MockTransactionalSender) HasPerformedReads() bool {
panic("unimplemented")
}

// HasPerformedWrites is part of TxnSenderFactory.
func (m *MockTransactionalSender) HasPerformedWrites() bool {
panic("unimplemented")
}

// MockTxnSenderFactory is a TxnSenderFactory producing MockTxnSenders.
type MockTxnSenderFactory struct {
senderFunc func(context.Context, *roachpb.Transaction, roachpb.BatchRequest) (
Expand Down
6 changes: 6 additions & 0 deletions pkg/kv/sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -336,6 +336,12 @@ type TxnSender interface {

// ClearTxnRetryableErr clears the retryable error, if any.
ClearTxnRetryableErr(ctx context.Context)

// HasPerformedReads returns true if a read has been performed.
HasPerformedReads() bool

// HasPerformedWrites returns true if a write has been performed.
HasPerformedWrites() bool
}

// SteppingMode is the argument type to ConfigureStepping.
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2585,6 +2585,9 @@ func (ex *connExecutor) setTransactionModes(
return errors.AssertionFailedf("expected an evaluated AS OF timestamp")
}
if !asOfTs.IsEmpty() {
if err := ex.state.checkReadsAndWrites(); err != nil {
return err
}
if err := ex.state.setHistoricalTimestamp(ctx, asOfTs); err != nil {
return err
}
Expand Down
24 changes: 24 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/as_of
Original file line number Diff line number Diff line change
Expand Up @@ -108,3 +108,27 @@ SELECT * FROM t AS OF SYSTEM TIME with_min_timestamp(statement_timestamp())
skipif config 3node-tenant
statement error pgcode XXC01 with_max_staleness can only be used with a CCL distribution
SELECT * FROM t AS OF SYSTEM TIME with_max_staleness('1s'::interval)

statement ok
BEGIN

statement ok
SELECT * from t

statement error cannot set fixed timestamp, .* already performed reads
SET TRANSACTION AS OF system time '-1s'

statement ok
ROLLBACK

statement ok
BEGIN

statement ok
INSERT INTO t VALUES(1)

statement error cannot set fixed timestamp, .* already performed writes
SET TRANSACTION AS OF system time '-1s'

statement ok
ROLLBACK
25 changes: 25 additions & 0 deletions pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
Expand Down Expand Up @@ -287,6 +289,7 @@ func (ts *txnState) setHistoricalTimestamp(
) error {
ts.mu.Lock()
defer ts.mu.Unlock()

if err := ts.mu.txn.SetFixedTimestamp(ctx, historicalTimestamp); err != nil {
return err
}
Expand Down Expand Up @@ -465,3 +468,25 @@ func (ts *txnState) consumeAdvanceInfo() advanceInfo {
ts.adv = advanceInfo{}
return adv
}

// checkReadsAndWrites returns an error if the transaction has performed reads
// or writes.
func (ts *txnState) checkReadsAndWrites() error {
ts.mu.Lock()
defer ts.mu.Unlock()

if ts.mu.txn.Sender().HasPerformedReads() {
return pgerror.Newf(
pgcode.InvalidTransactionState,
"cannot set fixed timestamp, txn %s already performed reads",
ts.mu.txn)
}

if ts.mu.txn.Sender().HasPerformedWrites() {
return pgerror.Newf(
pgcode.InvalidTransactionState,
"cannot set fixed timestamp, txn %s already performed writes",
ts.mu.txn)
}
return nil
}

0 comments on commit a1e7e05

Please sign in to comment.