diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 5fcd8d5291c1..025de97a34cc 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -86,13 +86,16 @@ func (n *dropIndexNode) startExec(params runParams) error { // drop need to be visible to the second drop. tableDesc, err := params.p.ResolveMutableTableDescriptor( ctx, index.tn, true /*required*/, ResolveRequireTableDesc) - if err != nil { + if sqlbase.IsUndefinedRelationError(err) { // Somehow the descriptor we had during planning is not there // any more. return errors.NewAssertionErrorWithWrappedErrf(err, "table descriptor for %q became unavailable within same txn", tree.ErrString(index.tn)) } + if err != nil { + return err + } // If we couldn't find the index by name, this is either a legitimate error or // this statement contains an 'IF EXISTS' qualifier. Both of these cases are diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index 9495c2e54a5a..45e9e5d5b590 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -15,6 +15,7 @@ import ( gosql "database/sql" "fmt" "math/rand" + "regexp" "testing" "github.com/cockroachdb/cockroach/pkg/base" @@ -23,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" @@ -36,9 +38,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" + "github.com/opentracing/opentracing-go" "github.com/stretchr/testify/require" ) @@ -1290,3 +1297,134 @@ CREATE VIEW test.acol(a) AS SELECT a FROM test.t; return err }) } + +// TestDropIndexHandlesRetriableErrors is a regression test against #48474. +// The bug was that retriable errors, which are generally possible, were being +// treated as assertion failures. +func TestDropIndexHandlesRetriableErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + rf := newDynamicRequestFilter() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + TestingRequestFilter: rf.filter, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + // What we want to do is have a transaction which does the planning work to + // drop an index. Then we want to expose the execution of the DROP INDEX to + // an error when retrieving the mutable table descriptor. We'll do this by + // injecting a ReadWithinUncertainty error underneath the DROP INDEX + // after planning has concluded. + + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY, j INT, INDEX j_idx (j))") + + var tableID uint32 + tdb.QueryRow(t, ` +SELECT + table_id +FROM + crdb_internal.tables +WHERE + name = $1 AND database_name = current_database();`, + "foo").Scan(&tableID) + + // Start the user transaction and enable tracing as we'll use the trace + // to determine when planning has concluded. + txn, err := tc.ServerConn(0).Begin() + require.NoError(t, err) + _, err = txn.Exec("SET TRACING = on") + require.NoError(t, err) + // Let's find out our transaction ID for our transaction by running a query. + // We'll also use this query to install a refresh span over the table data. + // Inject a request filter to snag the transaction ID. + tablePrefix := keys.SystemSQLCodec.TablePrefix(tableID) + tableSpan := roachpb.Span{ + Key: tablePrefix, + EndKey: tablePrefix.PrefixEnd(), + } + var filterState struct { + syncutil.Mutex + txnID uuid.UUID + } + getTxnID := func() uuid.UUID { + filterState.Lock() + defer filterState.Unlock() + return filterState.txnID + } + rf.setFilter(func(ctx context.Context, request roachpb.BatchRequest) *roachpb.Error { + if request.Txn == nil || request.Txn.Name != sql.SQLTxnName { + return nil + } + filterState.Lock() + defer filterState.Unlock() + if filterState.txnID != (uuid.UUID{}) { + return nil + } + if scanRequest, ok := request.GetArg(roachpb.Scan); ok { + scan := scanRequest.(*roachpb.ScanRequest) + if scan.Span().Overlaps(tableSpan) { + filterState.txnID = request.Txn.ID + } + } + return nil + }) + + // Run the scan of the table to activate the filter as well as add the + // refresh span over the table data. + var trash int + require.Equal(t, gosql.ErrNoRows, + txn.QueryRow("SELECT * FROM foo").Scan(&trash)) + rf.setFilter(nil) + require.NotEqual(t, uuid.UUID{}, getTxnID()) + + // Perform a write after the above read so that a refresh will fail and + // observe its timestamp. + tdb.Exec(t, "INSERT INTO foo VALUES (1)") + var afterInsertStr string + tdb.QueryRow(t, "SELECT cluster_logical_timestamp()").Scan(&afterInsertStr) + afterInsert, err := sql.ParseHLC(afterInsertStr) + require.NoError(t, err) + + // Now set up a filter to detect when the DROP INDEX execution will begin + // and inject an error forcing a refresh above the conflicting write which + // will fail. We'll want to ensure that we get a retriable error. + // Use the below pattern to detect when the user transaction has finished + // planning and is now executing. + dropIndexPlanningEndsRE := regexp.MustCompile("(?s)planning starts: DROP INDEX.*planning ends") + rf.setFilter(func(ctx context.Context, request roachpb.BatchRequest) *roachpb.Error { + if request.Txn == nil { + return nil + } + filterState.Lock() + defer filterState.Unlock() + if filterState.txnID != request.Txn.ID { + return nil + } + sp := opentracing.SpanFromContext(ctx) + rec := tracing.GetRecording(sp) + if !dropIndexPlanningEndsRE.MatchString(rec.String()) { + return nil + } + if getRequest, ok := request.GetArg(roachpb.Get); ok { + put := getRequest.(*roachpb.GetRequest) + if put.Key.Equal(sqlbase.MakeDescMetadataKey(keys.SystemSQLCodec, sqlbase.ID(tableID))) { + filterState.txnID = uuid.UUID{} + return roachpb.NewError(roachpb.NewReadWithinUncertaintyIntervalError( + request.Txn.ReadTimestamp, afterInsert, request.Txn)) + } + } + return nil + }) + + _, err = txn.Exec("DROP INDEX foo@j_idx") + require.Truef(t, isRetryableErr(err), "drop index error: %v", err) + require.NoError(t, txn.Rollback()) +} diff --git a/pkg/sql/helpers_test.go b/pkg/sql/helpers_test.go index b7f07ef83030..1f1c6b7c21b6 100644 --- a/pkg/sql/helpers_test.go +++ b/pkg/sql/helpers_test.go @@ -25,6 +25,9 @@ type tableVersionID struct { version sqlbase.DescriptorVersion } +// SQLTxnName is the transaction name used by sql transactions. +const SQLTxnName = sqlTxnName + // LeaseRemovalTracker can be used to wait for leases to be removed from the // store (leases are removed from the store async w.r.t. LeaseManager // operations). diff --git a/pkg/sql/pgwire/testdata/pgtest/notice b/pkg/sql/pgwire/testdata/pgtest/notice index bc5f6754f4a8..1e3ea5aa5386 100644 --- a/pkg/sql/pgwire/testdata/pgtest/notice +++ b/pkg/sql/pgwire/testdata/pgtest/notice @@ -29,7 +29,7 @@ ReadyForQuery ---- {"Type":"ParseComplete"} {"Type":"BindComplete"} -{"Severity":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":496,"Routine":"dropIndexByName","UnknownFields":null} +{"Severity":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":499,"Routine":"dropIndexByName","UnknownFields":null} {"Type":"CommandComplete","CommandTag":"DROP INDEX"} {"Type":"ReadyForQuery","TxStatus":"I"}