Skip to content

Commit

Permalink
sql: release save point did not retry for two version invariant errors
Browse files Browse the repository at this point in the history
Previously, when release save point was issued for cockroach_restart
this would drive a commit, which could run into retryable errors.
Like the two version invariant error, which is not tagged as a
user facing re triable error, so the client doesn't know to
retry. To address this, this patch converts two version invariant
errors in this case into user facing retryable errors;

Fixes: cockroachdb#107171

Release note (bug fix): Release save point could incorrectly emit
"cannot publish new versions for descriptors" instead of a
retryable error to applications.
  • Loading branch information
fqazi committed Aug 14, 2023
1 parent 0e8ff82 commit 801e44c
Show file tree
Hide file tree
Showing 5 changed files with 151 additions and 8 deletions.
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -374,6 +374,7 @@ go_library(
"//pkg/sql/catalog/multiregion",
"//pkg/sql/catalog/nstree",
"//pkg/sql/catalog/randgen",
"//pkg/sql/catalog/redact",
"//pkg/sql/catalog/resolver",
"//pkg/sql/catalog/schemadesc",
"//pkg/sql/catalog/schemaexpr",
Expand Down
115 changes: 115 additions & 0 deletions pkg/sql/catalog/lease/lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1471,6 +1471,121 @@ CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR);
}
}

// Tests that when a transaction has already returned results
// to the user and the transaction continues on to make a schema change,
// whenever the table lease two version invariant is violated and the
// transaction needs to be restarted, a retryable error is returned to the
// user. This specific version validations that release savepoint does the
// same with cockroach_restart, which commits on release.
func TestTwoVersionInvariantRetryErrorWitSavePoint(t *testing.T) {
defer leaktest.AfterTest(t)()
var violations int64
var params base.TestServerArgs
params.Knobs = base.TestingKnobs{
// Disable execution of schema changers after the schema change
// transaction commits. This is to prevent executing the default
// WaitForOneVersion() code that holds up a schema change
// transaction until the new version has been published to the
// entire cluster.
SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{
SchemaChangeJobNoOp: func() bool {
return true
},
TwoVersionLeaseViolation: func() {
atomic.AddInt64(&violations, 1)
},
},
}
s, sqlDB, kvDB := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.Background())

if _, err := sqlDB.Exec(`
CREATE DATABASE t;
CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR);
INSERT INTO t.kv VALUES ('a', 'b');
`); err != nil {
t.Fatal(err)
}

tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv")
if tableDesc.GetVersion() != 1 {
t.Fatalf("invalid version %d", tableDesc.GetVersion())
}

tx, err := sqlDB.Begin()
if err != nil {
t.Fatal(err)
}

// Grab a lease on the table.
rows, err := tx.Query("SELECT * FROM t.kv")
if err != nil {
t.Fatal(err)
}
if err := rows.Close(); err != nil {
t.Fatal(err)
}

// Modify the table descriptor increments the version.
if _, err := sqlDB.Exec(`ALTER TABLE t.kv RENAME to t.kv1`); err != nil {
t.Fatal(err)
}

txRetry, err := sqlDB.Begin()
if err != nil {
t.Fatal(err)
}
_, err = txRetry.Exec("SAVEPOINT cockroach_restart;")
if err != nil {
t.Fatal(err)
}

// Read some data using the transaction so that it cannot be
// retried internally
rows, err = txRetry.Query(`SELECT 1`)
if err != nil {
t.Fatal(err)
}
if err := rows.Close(); err != nil {
t.Fatal(err)
}

if _, err := txRetry.Exec(`ALTER TABLE t.kv1 RENAME TO t.kv2`); err != nil {
t.Fatal(err)
}

var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
// This can hang waiting for one version before tx.Commit() is
// called below, so it is executed in another goroutine.
_, err := txRetry.Exec("RELEASE SAVEPOINT cockroach_restart;")
if !testutils.IsError(err,
fmt.Sprintf(`TransactionRetryWithProtoRefreshError: cannot publish new versions for descriptors: \[\{kv1 %d 1\}\], old versions still in use`, tableDesc.GetID()),
) {
t.Errorf("err = %v", err)
}
err = txRetry.Rollback()
if err != nil {
t.Errorf("err = %v", err)
}
}()

// Make sure that txRetry does violate the two version lease invariant.
testutils.SucceedsSoon(t, func() error {
if atomic.LoadInt64(&violations) == 0 {
return errors.Errorf("didnt retry schema change")
}
return nil
})
// Commit the first transaction, unblocking txRetry.
if err := tx.Commit(); err != nil {
t.Fatal(err)
}
wg.Wait()
}

// Tests that when a transaction has already returned results
// to the user and the transaction continues on to make a schema change,
// whenever the table lease two version invariant is violated and the
Expand Down
23 changes: 23 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/redact"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schematelemetry/schematelemetrycontroller"
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
"github.com/cockroachdb/cockroach/pkg/sql/contention/txnidcache"
Expand Down Expand Up @@ -3245,6 +3246,28 @@ func errIsRetriable(err error) bool {
descs.IsTwoVersionInvariantViolationError(err)
}

// convertRetriableErrorIntoUserVisibleError converts internal retriable
// errors into external, so that the client goes and retries this
// transaction. One example of this is two version invariant errors, which
// happens when a schema change is waiting for a schema change transition to
// propagate. When this happens, we either need to retry externally or internally,
// depending on if we are in an explicit transaction.
func (ex *connExecutor) convertRetriableErrorIntoUserVisibleError(
ctx context.Context, origErr error,
) (modifiedErr error, err error) {
if descs.IsTwoVersionInvariantViolationError(origErr) {
if resetErr := ex.resetTransactionOnSchemaChangeRetry(ctx); resetErr != nil {
return nil, resetErr
}
// Generating a forced retry error here, right after resetting the
// transaction is not exactly necessary, but it's a sound way to
// generate the only type of ClientVisibleRetryError we have.
return ex.state.mu.txn.GenerateForcedRetryableError(ctx, redact.Sprint(origErr)), nil
}
// Return the original error, this error will not be surfaced to the user.
return origErr, nil
}

// makeErrEvent takes an error and returns either an eventRetriableErr or an
// eventNonRetriableErr, depending on the error type.
func (ex *connExecutor) makeErrEvent(err error, stmt tree.Statement) (fsm.Event, fsm.EventPayload) {
Expand Down
14 changes: 6 additions & 8 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -1232,14 +1232,12 @@ func (ex *connExecutor) commitSQLTransaction(
}
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionStartTransactionCommit, timeutil.Now())
if err := commitFn(ctx); err != nil {
if descs.IsTwoVersionInvariantViolationError(err) {
if resetErr := ex.resetTransactionOnSchemaChangeRetry(ctx); resetErr != nil {
return ex.makeErrEvent(err, ast)
}
// Generating a forced retry error here, right after resetting the
// transaction is not exactly necessary, but it's a sound way to
// generate the only type of ClientVisibleRetryError we have.
err = ex.state.mu.txn.GenerateForcedRetryableError(ctx, redact.Sprint(err))
// For certain retryable errors, we should turn them into client visible
// errors, since the client needs to retry now.
var conversionError error
err, conversionError = ex.convertRetriableErrorIntoUserVisibleError(ctx, err)
if conversionError != nil {
return ex.makeErrEvent(conversionError, ast)
}
return ex.makeErrEvent(err, ast)
}
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/conn_executor_savepoints.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,12 @@ func (ex *connExecutor) execRelease(
// Committing the transaction failed. We'll go to state RestartWait if
// it's a retriable error, or to state RollbackWait otherwise.
if errIsRetriable(err) {
// For certain retryable errors, we should turn them into client visible
// errors, since the client needs to retry now.
var conversionError error
if err, conversionError = ex.convertRetriableErrorIntoUserVisibleError(ctx, err); conversionError != nil {
return ex.makeErrEvent(conversionError, s)
}
// Add the savepoint back. We want to allow a ROLLBACK TO SAVEPOINT
// cockroach_restart (that's the whole point of commitOnRelease).
env.push(*entry)
Expand Down

0 comments on commit 801e44c

Please sign in to comment.