Skip to content

Commit

Permalink
Merge #68217
Browse files Browse the repository at this point in the history
68217: sql: fix show_create_all_tables in explicit transactions r=nvanbenschoten a=nvanbenschoten

Related to #68216.

Previously, running `crdb_internal.show_create_all_tables` in an existing explicit transaction would crash for the reasons discussed in #68216. This should not have crashed but it should have thrown an error. This revealed that the internal executor in the `showCreateAllTablesGenerator` was a little confused. It was issuing AS OF SYSTEM TIME queries, but also supplying its parent transaction. So it wanted to execute in the context of its parent transaction, but also to run at a very specific timestamp - contradictory desires.

This commit fixes this by removing the AS OF SYSTEM TIME clauses from the internal executor queries. This has the effect of making this builtin transactional.

Example crash:
```
➜ ./cockroach demo --insecure

> BEGIN;

OPEN> SELECT count(*) FROM rides;
  count
---------
    500

OPEN> SELECT crdb_internal.show_create_all_tables('movr');
ERROR: internal error: crdb_internal.show_create_all_tables: unexpected batch read timestamp: 1627529007.202325000,0. Expected refreshed timestamp: 1627528898.793460000,0. ba: Scan [/Table/3/1,/Table/3/2), [txn: 3565aed1]. txn: "sql txn" meta={id=3565aed1 pri=0.05136598 epo=0 ts=1627529007.202325000,0 min=1627528898.793460000,0 seq=0} lock=false stat=PENDING rts=1627529007.202325000,0 wto=false gul=1627529007.202325000,0
SQLSTATE: XX000
DETAIL: stack trace:
github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go:160: SendLocked()
github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go:285: SendLocked()
github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator.go:105: SendLocked()
github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go:240: SendLocked()
github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord/txn_coord_sender.go:513: Send()
github.com/cockroachdb/cockroach/pkg/kv/db.go:831: sendUsingSender()
github.com/cockroachdb/cockroach/pkg/kv/txn.go:982: Send()
github.com/cockroachdb/cockroach/pkg/kv/db.go:742: sendAndFill()
github.com/cockroachdb/cockroach/pkg/kv/txn.go:635: Run()
github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv/catalogkv.go:281: getAllDescriptorsAndMaybeNamespaceEntriesUnvalidated()
github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv/catalogkv.go:343: GetAllDescriptors()
github.com/cockroachdb/cockroach/pkg/sql/catalog/descs/kv_descriptors.go:265: getAllDescriptors()
github.com/cockroachdb/cockroach/pkg/sql/catalog/descs/collection.go:270: GetAllDescriptors()
github.com/cockroachdb/cockroach/pkg/sql/information_schema.go:2335: forEachTableDescWithTableLookupInternal()
github.com/cockroachdb/cockroach/pkg/sql/information_schema.go:2289: forEachTableDescWithTableLookup()
github.com/cockroachdb/cockroach/pkg/sql/pg_catalog.go:964: func1()
github.com/cockroachdb/cockroach/pkg/sql/virtual_schema.go:533: 1()
github.com/cockroachdb/cockroach/pkg/sql/virtual_table.go:120: func3()
github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:442: func2()
runtime/asm_amd64.s:1371: goexit()

HINT: You have encountered an unexpected error.

Please check the public issue tracker to check whether this problem is
already tracked. If you cannot find it there, please report the error
with details by creating a new issue.

If you would rather not post publicly, please contact us directly
using the support form.

We appreciate your feedback.
```

Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
craig[bot] and nvanbenschoten committed Jul 30, 2021
2 parents aa2d749 + cdb1332 commit ef59955
Show file tree
Hide file tree
Showing 3 changed files with 59 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -369,3 +369,45 @@ query T
SELECT crdb_internal.show_create_all_tables('test_sequence')
----
CREATE SEQUENCE public.s1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 123 START 1;

# Ensure that the builtin can be run in unused and used transactions.

statement ok
USE test_schema;

query T
BEGIN;
SELECT crdb_internal.show_create_all_tables('test_schema');
COMMIT;
----
CREATE TABLE sc1.t (
x INT8 NULL,
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (x, rowid)
);
CREATE TABLE sc2.t (
x INT8 NULL,
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (x, rowid)
);

query T
BEGIN;
SELECT * FROM sc1.t;
SELECT crdb_internal.show_create_all_tables('test_schema');
COMMIT;
----
CREATE TABLE sc1.t (
x INT8 NULL,
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (x, rowid)
);
CREATE TABLE sc2.t (
x INT8 NULL,
rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT "primary" PRIMARY KEY (rowid ASC),
FAMILY "primary" (x, rowid)
);
30 changes: 11 additions & 19 deletions pkg/sql/sem/builtins/generator_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/json"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
pbtypes "github.com/gogo/protobuf/types"
Expand Down Expand Up @@ -1690,12 +1689,11 @@ const (
// showCreateAllTablesGenerator supports the execution of
// crdb_internal.show_create_all_tables(dbName).
type showCreateAllTablesGenerator struct {
ie sqlutil.InternalExecutor
txn *kv.Txn
timestamp string
ids []int64
dbName string
acc mon.BoundAccount
ie sqlutil.InternalExecutor
txn *kv.Txn
ids []int64
dbName string
acc mon.BoundAccount

// The following variables are updated during
// calls to Next() and change throughout the lifecycle of
Expand Down Expand Up @@ -1727,7 +1725,7 @@ func (s *showCreateAllTablesGenerator) Start(ctx context.Context, txn *kv.Txn) e
// We also account for the memory in the BoundAccount memory monitor in
// showCreateAllTablesGenerator.
ids, err := getTopologicallySortedTableIDs(
ctx, s.ie, txn, s.dbName, s.timestamp, &s.acc,
ctx, s.ie, txn, s.dbName, &s.acc,
)
if err != nil {
return err
Expand All @@ -1753,7 +1751,7 @@ func (s *showCreateAllTablesGenerator) Next(ctx context.Context) (bool, error) {
}

createStmt, err := getCreateStatement(
ctx, s.ie, s.txn, s.ids[s.idx], s.timestamp, s.dbName,
ctx, s.ie, s.txn, s.ids[s.idx], s.dbName,
)
if err != nil {
return false, err
Expand Down Expand Up @@ -1799,7 +1797,7 @@ func (s *showCreateAllTablesGenerator) Next(ctx context.Context) (bool, error) {
statementReturnType = alterValidateFKStatements
}
alterStmt, err := getAlterStatements(
ctx, s.ie, s.txn, s.ids[s.idx], s.timestamp, s.dbName, statementReturnType,
ctx, s.ie, s.txn, s.ids[s.idx], s.dbName, statementReturnType,
)
if err != nil {
return false, err
Expand Down Expand Up @@ -1836,15 +1834,9 @@ func makeShowCreateAllTablesGenerator(
ctx *tree.EvalContext, args tree.Datums,
) (tree.ValueGenerator, error) {
dbName := string(tree.MustBeDString(args[0]))
tsI, err := tree.MakeDTimestamp(timeutil.Now(), time.Microsecond)
if err != nil {
return nil, err
}
ts := tsI.String()
return &showCreateAllTablesGenerator{
timestamp: ts,
dbName: dbName,
ie: ctx.InternalExecutor.(sqlutil.InternalExecutor),
acc: ctx.Mon.MakeBoundAccount(),
dbName: dbName,
ie: ctx.InternalExecutor.(sqlutil.InternalExecutor),
acc: ctx.Mon.MakeBoundAccount(),
}, nil
}
19 changes: 6 additions & 13 deletions pkg/sql/sem/builtins/show_create_all_tables_builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,10 +53,9 @@ func getTopologicallySortedTableIDs(
ie sqlutil.InternalExecutor,
txn *kv.Txn,
dbName string,
ts string,
acc *mon.BoundAccount,
) ([]int64, error) {
ids, err := getTableIDs(ctx, ie, txn, ts, dbName, acc)
ids, err := getTableIDs(ctx, ie, txn, dbName, acc)
if err != nil {
return nil, err
}
Expand All @@ -74,9 +73,8 @@ func getTopologicallySortedTableIDs(
query := fmt.Sprintf(`
SELECT dependson_id
FROM %s.crdb_internal.backward_dependencies
AS OF SYSTEM TIME %s
WHERE descriptor_id = $1
`, dbName, ts)
`, dbName)
it, err := ie.QueryIteratorEx(
ctx,
"crdb_internal.show_create_all_tables",
Expand Down Expand Up @@ -158,18 +156,16 @@ func getTableIDs(
ctx context.Context,
ie sqlutil.InternalExecutor,
txn *kv.Txn,
ts string,
dbName string,
acc *mon.BoundAccount,
) ([]int64, error) {
query := fmt.Sprintf(`
SELECT descriptor_id
FROM %s.crdb_internal.create_statements
AS OF SYSTEM TIME %s
WHERE database_name = $1
AND is_virtual = FALSE
AND is_temporary = FALSE
`, dbName, ts)
`, dbName)
it, err := ie.QueryIteratorEx(
ctx,
"crdb_internal.show_create_all_tables",
Expand Down Expand Up @@ -246,15 +242,14 @@ func topologicalSort(
// getCreateStatement gets the create statement to recreate a table (ignoring fks)
// for a given table id in a database.
func getCreateStatement(
ctx context.Context, ie sqlutil.InternalExecutor, txn *kv.Txn, id int64, ts string, dbName string,
ctx context.Context, ie sqlutil.InternalExecutor, txn *kv.Txn, id int64, dbName string,
) (tree.Datum, error) {
query := fmt.Sprintf(`
SELECT
create_nofks
FROM %s.crdb_internal.create_statements
AS OF SYSTEM TIME %s
WHERE descriptor_id = $1
`, dbName, ts)
`, dbName)
row, err := ie.QueryRowEx(
ctx,
"crdb_internal.show_create_all_tables",
Expand All @@ -277,17 +272,15 @@ func getAlterStatements(
ie sqlutil.InternalExecutor,
txn *kv.Txn,
id int64,
ts string,
dbName string,
statementType string,
) (tree.Datum, error) {
query := fmt.Sprintf(`
SELECT
%s
FROM %s.crdb_internal.create_statements
AS OF SYSTEM TIME %s
WHERE descriptor_id = $1
`, statementType, dbName, ts)
`, statementType, dbName)
row, err := ie.QueryRowEx(
ctx,
"crdb_internal.show_create_all_tables",
Expand Down

0 comments on commit ef59955

Please sign in to comment.