From e77a25afa4e5112052aecc3ce606a4fd8725b834 Mon Sep 17 00:00:00 2001 From: richardjcai Date: Mon, 8 Feb 2021 18:21:43 -0500 Subject: [PATCH] sql: add crdb_internal.show_create_all_tables builtin Release note (sql change): crdb_internal.show_create_all_tables is a new builtin that takes in a database name (string) and returns a flat log of all the CREATE TABLE statements in the database followed by alter statements to add constraints. The output can be used to recreate a database. This builtin was added to replace old dump logic. --- docs/generated/sql/functions.md | 4 + pkg/sql/BUILD.bazel | 1 + .../logic_test/show_create_all_tables_builtin | 383 ++++++++++++++++++ pkg/sql/sem/builtins/BUILD.bazel | 2 + pkg/sql/sem/builtins/builtins.go | 15 + .../show_create_all_tables_builtin.go | 310 ++++++++++++++ .../show_create_all_tables_builtin_test.go | 70 ++++ 7 files changed, 785 insertions(+) create mode 100644 pkg/sql/logictest/testdata/logic_test/show_create_all_tables_builtin create mode 100644 pkg/sql/sem/builtins/show_create_all_tables_builtin.go create mode 100644 pkg/sql/show_create_all_tables_builtin_test.go diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index dae3c9dcae1a..f37c4a0e3bb1 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -2311,6 +2311,10 @@ The swap_ordinate_string parameter is a 2-character string naming the ordinates convert_to(str: string, enc: string) → bytes

Encode the string str as a byte array using encoding enc. Supports encodings ‘UTF8’ and ‘LATIN1’.

+crdb_internal.show_create_all_tables(dbName: string) → string

Returns a flat log of CREATE table statements followed by +ALTER table statements that add table constraints. The flat log can be used +to recreate a database.’

+
decode(text: string, format: string) → bytes

Decodes data using format (hex / escape / base64).

difference(source: string, target: string) → string

Convert two strings to their Soundex codes and then reports the number of matching code positions.

diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 675fd3358ac5..62e23c68c049 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -461,6 +461,7 @@ go_test( "scrub_test.go", "sequence_test.go", "set_zone_config_test.go", + "show_create_all_tables_builtin_test.go", "show_fingerprints_test.go", "show_ranges_test.go", "show_stats_test.go", diff --git a/pkg/sql/logictest/testdata/logic_test/show_create_all_tables_builtin b/pkg/sql/logictest/testdata/logic_test/show_create_all_tables_builtin new file mode 100644 index 000000000000..0c43dcf729c0 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/show_create_all_tables_builtin @@ -0,0 +1,383 @@ +query T +SELECT crdb_internal.show_create_all_tables('d') +---- +· + +statement ok +CREATE DATABASE d + +query T +SELECT crdb_internal.show_create_all_tables('d') +---- +· + +statement ok +CREATE TABLE d.parent ( + x INT, + y INT, + z INT, + UNIQUE (x, y, z), + FAMILY f1 (x, y, z), + UNIQUE (x) +); + +statement ok +CREATE TABLE d.full_test ( + x INT, + y INT, + z INT, + FOREIGN KEY (x, y, z) REFERENCES d.parent (x, y, z) MATCH FULL ON DELETE CASCADE ON UPDATE CASCADE, + FAMILY f1 (x, y, z), + UNIQUE (x) + ); + +statement ok +ALTER TABLE d.full_test ADD CONSTRAINT test_fk FOREIGN KEY (x) REFERENCES d.parent (x) ON DELETE CASCADE + +statement ok +CREATE VIEW d.vx AS SELECT 1 + +statement ok +CREATE SEQUENCE d.s + +# parent should come before full_test due to dependency ordering. +# if dependency's aren't considered, full_test will appear first due to +# lexicographical ordering. +query T +SELECT crdb_internal.show_create_all_tables('d') +---- +CREATE TABLE public.parent ( + x INT8 NULL, + y INT8 NULL, + z INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + UNIQUE INDEX parent_x_y_z_key (x ASC, y ASC, z ASC), + UNIQUE INDEX parent_x_key (x ASC), + FAMILY f1 (x, y, z, rowid) +); +CREATE TABLE public.full_test ( + x INT8 NULL, + y INT8 NULL, + z INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + UNIQUE INDEX full_test_x_key (x ASC), + FAMILY f1 (x, y, z, rowid) +); +CREATE SEQUENCE public.s MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1; +CREATE VIEW public.vx ("?column?") AS SELECT 1; +ALTER TABLE public.full_test ADD CONSTRAINT fk_x_ref_parent FOREIGN KEY (x, y, z) REFERENCES public.parent(x, y, z) MATCH FULL ON DELETE CASCADE ON UPDATE CASCADE; +ALTER TABLE public.full_test ADD CONSTRAINT test_fk FOREIGN KEY (x) REFERENCES public.parent(x) ON DELETE CASCADE; +-- Validate foreign key constraints. These can fail if there was unvalidated data during the SHOW CREATE ALL TABLES +ALTER TABLE public.full_test VALIDATE CONSTRAINT fk_x_ref_parent; +ALTER TABLE public.full_test VALIDATE CONSTRAINT test_fk; + + +# testuser does not have CONNECT on database d and cannot see any tables. +user testuser + +query T +SELECT crdb_internal.show_create_all_tables('d') +---- +· + +user root + +statement ok +GRANT CREATE on DATABASE d TO testuser + +# testuser should be able to see the descriptors with +# CREATE privilege on the database. +# TODO(richardjcai): Replace this with CONNECT and +# add CONNECT privilege required on the builtin description once #59676 is in. +query T +SELECT crdb_internal.show_create_all_tables('d') +---- +CREATE TABLE public.parent ( + x INT8 NULL, + y INT8 NULL, + z INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + UNIQUE INDEX parent_x_y_z_key (x ASC, y ASC, z ASC), + UNIQUE INDEX parent_x_key (x ASC), + FAMILY f1 (x, y, z, rowid) +); +CREATE TABLE public.full_test ( + x INT8 NULL, + y INT8 NULL, + z INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + UNIQUE INDEX full_test_x_key (x ASC), + FAMILY f1 (x, y, z, rowid) +); +CREATE SEQUENCE public.s MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1; +CREATE VIEW public.vx ("?column?") AS SELECT 1; +ALTER TABLE public.full_test ADD CONSTRAINT fk_x_ref_parent FOREIGN KEY (x, y, z) REFERENCES public.parent(x, y, z) MATCH FULL ON DELETE CASCADE ON UPDATE CASCADE; +ALTER TABLE public.full_test ADD CONSTRAINT test_fk FOREIGN KEY (x) REFERENCES public.parent(x) ON DELETE CASCADE; +-- Validate foreign key constraints. These can fail if there was unvalidated data during the SHOW CREATE ALL TABLES +ALTER TABLE public.full_test VALIDATE CONSTRAINT fk_x_ref_parent; +ALTER TABLE public.full_test VALIDATE CONSTRAINT test_fk; + + +user root + +# Make sure temp tables don't show up in crdb_internal.show_create_all_tables. +statement ok +CREATE DATABASE temp_test + +statement ok +USE temp_test + +statement ok +SET experimental_enable_temp_tables = 'on' + +statement ok +CREATE TEMPORARY TABLE t() + +query T +SELECT crdb_internal.show_create_all_tables('temp_test') +---- +· + +# Test that a database with foreign keys has the right order. +statement ok +CREATE DATABASE test_fk_order; +USE test_fk_order; +-- B -> A +CREATE TABLE b (i int PRIMARY KEY); +CREATE TABLE a (i int REFERENCES b); +INSERT INTO b VALUES (1); +INSERT INTO a VALUES (1); +-- Test multiple tables to make sure transitive deps are sorted correctly. +-- E -> D -> C +-- G -> F -> D -> C +CREATE TABLE g (i int PRIMARY KEY); +CREATE TABLE f (i int PRIMARY KEY, g int REFERENCES g, FAMILY f1 (i, g)); +CREATE TABLE e (i int PRIMARY KEY); +CREATE TABLE d (i int PRIMARY KEY, e int REFERENCES e, f int REFERENCES f, FAMILY f1 (i, e, f)); +CREATE TABLE c (i int REFERENCES d); +-- Test a table that uses a sequence to make sure the sequence is dumped first. +CREATE SEQUENCE s; +CREATE TABLE s_tbl (id INT PRIMARY KEY DEFAULT nextval('s'), v INT, FAMILY f1 (id, v)); + +# Table order should be B, A, E, G, F, D, C, sequence s, s_tbl. +query T +SELECT crdb_internal.show_create_all_tables('test_fk_order') +---- +CREATE TABLE public.b ( + i INT8 NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (i ASC), + FAMILY "primary" (i) +); +CREATE TABLE public.a ( + i INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + FAMILY "primary" (i, rowid) +); +CREATE TABLE public.e ( + i INT8 NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (i ASC), + FAMILY "primary" (i) +); +CREATE TABLE public.g ( + i INT8 NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (i ASC), + FAMILY "primary" (i) +); +CREATE TABLE public.f ( + i INT8 NOT NULL, + g INT8 NULL, + CONSTRAINT "primary" PRIMARY KEY (i ASC), + FAMILY f1 (i, g) +); +CREATE TABLE public.d ( + i INT8 NOT NULL, + e INT8 NULL, + f INT8 NULL, + CONSTRAINT "primary" PRIMARY KEY (i ASC), + FAMILY f1 (i, e, f) +); +CREATE TABLE public.c ( + i INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + FAMILY "primary" (i, rowid) +); +CREATE SEQUENCE public.s MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1; +CREATE TABLE public.s_tbl ( + id INT8 NOT NULL DEFAULT nextval('s':::STRING), + v INT8 NULL, + CONSTRAINT "primary" PRIMARY KEY (id ASC), + FAMILY f1 (id, v) +); +ALTER TABLE public.a ADD CONSTRAINT fk_i_ref_b FOREIGN KEY (i) REFERENCES public.b(i); +ALTER TABLE public.f ADD CONSTRAINT fk_g_ref_g FOREIGN KEY (g) REFERENCES public.g(i); +ALTER TABLE public.d ADD CONSTRAINT fk_e_ref_e FOREIGN KEY (e) REFERENCES public.e(i); +ALTER TABLE public.d ADD CONSTRAINT fk_f_ref_f FOREIGN KEY (f) REFERENCES public.f(i); +ALTER TABLE public.c ADD CONSTRAINT fk_i_ref_d FOREIGN KEY (i) REFERENCES public.d(i); +-- Validate foreign key constraints. These can fail if there was unvalidated data during the SHOW CREATE ALL TABLES +ALTER TABLE public.a VALIDATE CONSTRAINT fk_i_ref_b; +ALTER TABLE public.f VALIDATE CONSTRAINT fk_g_ref_g; +ALTER TABLE public.d VALIDATE CONSTRAINT fk_e_ref_e; +ALTER TABLE public.d VALIDATE CONSTRAINT fk_f_ref_f; +ALTER TABLE public.c VALIDATE CONSTRAINT fk_i_ref_d; + + +# Test that a cycle between two tables is handled correctly. +statement ok +CREATE DATABASE test_cycle; +USE test_cycle; +CREATE TABLE loop_a ( + id INT PRIMARY KEY, + b_id INT, + INDEX(b_id), + FAMILY f1 (id, b_id) +); +CREATE TABLE loop_b ( + id INT PRIMARY KEY, + a_id INT REFERENCES loop_a ON DELETE CASCADE, + FAMILY f1 (id, a_id) +); +ALTER TABLE loop_a ADD CONSTRAINT b_id_delete_constraint +FOREIGN KEY (b_id) REFERENCES loop_b (id) ON DELETE CASCADE; + +query T +SELECT crdb_internal.show_create_all_tables('test_cycle') +---- +CREATE TABLE public.loop_b ( + id INT8 NOT NULL, + a_id INT8 NULL, + CONSTRAINT "primary" PRIMARY KEY (id ASC), + FAMILY f1 (id, a_id) +); +CREATE TABLE public.loop_a ( + id INT8 NOT NULL, + b_id INT8 NULL, + CONSTRAINT "primary" PRIMARY KEY (id ASC), + INDEX loop_a_b_id_idx (b_id ASC), + FAMILY f1 (id, b_id) +); +ALTER TABLE public.loop_b ADD CONSTRAINT fk_a_id_ref_loop_a FOREIGN KEY (a_id) REFERENCES public.loop_a(id) ON DELETE CASCADE; +ALTER TABLE public.loop_a ADD CONSTRAINT b_id_delete_constraint FOREIGN KEY (b_id) REFERENCES public.loop_b(id) ON DELETE CASCADE; +-- Validate foreign key constraints. These can fail if there was unvalidated data during the SHOW CREATE ALL TABLES +ALTER TABLE public.loop_b VALIDATE CONSTRAINT fk_a_id_ref_loop_a; +ALTER TABLE public.loop_a VALIDATE CONSTRAINT b_id_delete_constraint; + + +# Test that a primary key with a non-default name works. +statement ok +CREATE DATABASE test_primary_key; +CREATE TABLE test_primary_key.t ( + i int, + CONSTRAINT pk_name PRIMARY KEY (i) +); + +query T +SELECT crdb_internal.show_create_all_tables('test_primary_key') +---- +CREATE TABLE public.t ( + i INT8 NOT NULL, + CONSTRAINT pk_name PRIMARY KEY (i ASC), + FAMILY "primary" (i) +); + +# Test that computed columns are shown correctly. +statement ok +CREATE DATABASE test_computed_column; +CREATE TABLE test_computed_column.t ( + a INT PRIMARY KEY, + b INT AS (a + 1) STORED, + FAMILY f1 (a, b) +); + +query T +SELECT crdb_internal.show_create_all_tables('test_computed_column') +---- +CREATE TABLE public.t ( + a INT8 NOT NULL, + b INT8 NULL AS (a + 1:::INT8) STORED, + CONSTRAINT "primary" PRIMARY KEY (a ASC), + FAMILY f1 (a, b) +); + + +# Test showing a table with a semicolon in the table, index, and +# column names properly escapes. +statement ok +CREATE DATABASE test_escaping; +CREATE TABLE test_escaping.";" (";" int, index (";")); +INSERT INTO test_escaping.";" VALUES (1); + +query T +SELECT crdb_internal.show_create_all_tables('test_escaping') +---- +CREATE TABLE public.";" ( + ";" INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT "primary" PRIMARY KEY (rowid ASC), + INDEX ";_;_idx" (";" ASC), + FAMILY "primary" (";", rowid) +); + + +# Ensure quotes in comments are properly escaped, also that the object names +# are properly escaped in the output of the COMMENT statements. +statement ok +CREATE DATABASE test_comment; +CREATE TABLE test_comment."t t" ("x'" INT PRIMARY KEY); +COMMENT ON TABLE test_comment."t t" IS 'has '' quotes'; +COMMENT ON INDEX test_comment."t t"@primary IS 'has '' more '' quotes'; +COMMENT ON COLUMN test_comment."t t"."x'" IS 'i '' just '' love '' quotes'; + +query T +SELECT crdb_internal.show_create_all_tables('test_comment') +---- +CREATE TABLE public."t t" ( + "x'" INT8 NOT NULL, + CONSTRAINT "primary" PRIMARY KEY ("x'" ASC), + FAMILY "primary" ("x'") +); +COMMENT ON TABLE public."t t" IS e'has \' quotes'; +COMMENT ON COLUMN public."t t"."x'" IS e'i \' just \' love \' quotes'; +COMMENT ON INDEX public."t t"@primary IS e'has \' more \' quotes'; + +# Ensure schemas are shown correctly. +statement ok +CREATE DATABASE test_schema; +USE test_schema; +CREATE SCHEMA sc1; +CREATE SCHEMA sc2; +CREATE TABLE sc1.t (x int); +CREATE TABLE sc2.t (x int); + +query T +SELECT crdb_internal.show_create_all_tables('test_schema') +---- +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) +); + + +# Ensure sequences are shown correctly. +statement ok +CREATE DATABASE test_sequence; +USE test_sequence; +CREATE SEQUENCE s1 INCREMENT 123; + +query T +SELECT crdb_internal.show_create_all_tables('test_sequence') +---- +CREATE SEQUENCE public.s1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 123 START 1; diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index 0aaeea90033c..70eab17addf2 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "math_builtins.go", "notice.go", "pg_builtins.go", + "show_create_all_tables_builtin.go", "window_builtins.go", "window_frame_builtins.go", ], @@ -56,6 +57,7 @@ go_library( "//pkg/sql/sessiondatapb", "//pkg/sql/sqlliveness", "//pkg/sql/sqltelemetry", + "//pkg/sql/sqlutil", "//pkg/sql/types", "//pkg/streaming", "//pkg/util", diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index df506eb6a3ef..d939e5274dbb 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -4856,6 +4856,21 @@ the locality flag on node startup. Returns an error if no region is set.`, Volatility: tree.VolatilityStable, }, ), + + "crdb_internal.show_create_all_tables": makeBuiltin( + tree.FunctionProperties{}, + tree.Overload{ + Types: tree.ArgTypes{ + {"dbName", types.String}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: showCreateAllTablesBuiltin, + Info: `Returns a flat log of CREATE table statements followed by +ALTER table statements that add table constraints. The flat log can be used +to recreate a database.'`, + Volatility: tree.VolatilityStable, + }, + ), } var lengthImpls = func(incBitOverload bool) builtinDefinition { diff --git a/pkg/sql/sem/builtins/show_create_all_tables_builtin.go b/pkg/sql/sem/builtins/show_create_all_tables_builtin.go new file mode 100644 index 000000000000..cacde442c115 --- /dev/null +++ b/pkg/sql/sem/builtins/show_create_all_tables_builtin.go @@ -0,0 +1,310 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package builtins + +import ( + "fmt" + "sort" + "strings" + "time" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" +) + +type tableMetadata struct { + ID int64 + name *tree.TableName + createStmt string + dependsOn []int64 + alter []string + validate []string +} + +type tableWithSchema struct { + schema string + table string +} + +// showCreateAllTablesBuiltin presents CREATE TABLE statements followed by +// ALTER TABLE statements for a database. The CREATE TABLE / ALTER TABLE +// statements are sorted such that the tables are ordered topologically to +// account for dependencies and references. The output can be used to +// recreate a database. +// The reason for ALTER TABLE statements coming after all CREATE TABLE +// statements is that we want to add constraints such as foreign keys +// after all the referenced tables are created. +func showCreateAllTablesBuiltin(evalCtx *tree.EvalContext, arg tree.Datums) (tree.Datum, error) { + mds, err := getMetadataForTablesInDB(evalCtx, arg) + if err != nil { + return nil, err + } + + if len(mds) == 0 { + return tree.NewDString(""), nil + } + + byID := make(map[int64]tableMetadata) + for _, md := range mds { + byID[md.ID] = md + } + + // First sort by name to guarantee stable output. + sort.Slice(mds, func(i, j int) bool { + return mds[i].name.String() < mds[j].name.String() + }) + + // Collect transitive dependencies in topological order into collected. + // The topological order is essential here since it captures dependencies + // for views and sequences creation, hence simple alphabetical sort won't + // be enough. + var collected []int64 + seen := make(map[int64]bool) + for _, md := range mds { + collect(md.ID, byID, seen, &collected) + } + // collectOrder maps a table ID to its collection index. This is needed + // instead of just using range over collected because collected may contain + // table IDs not present in the dump spec. It is simpler to sort mds correctly + // to skip over these referenced-but-not-dumped tables. + collectOrder := make(map[int64]int) + for i, id := range collected { + collectOrder[id] = i + } + + // Second sort dumped tables by dependency order. + sort.SliceStable(mds, func(i, j int) bool { + return collectOrder[mds[i].ID] < collectOrder[mds[j].ID] + }) + + var out []string + for _, md := range mds { + out = append(out, md.createStmt+";\n") + } + + hasRefs := false + for _, md := range mds { + for _, alter := range md.alter { + if !hasRefs { + hasRefs = true + } + out = append(out, fmt.Sprintf("%s;\n", alter)) + } + } + if hasRefs { + const alterValidateMessage = `-- Validate foreign key constraints. These can fail if there was unvalidated data during the SHOW CREATE ALL TABLES` + out = append(out, alterValidateMessage+"\n") + for _, md := range mds { + for _, validate := range md.validate { + out = append(out, fmt.Sprintf("%s;\n", validate)) + + } + } + } + + result := tree.NewDString(strings.Join(out, "")) + return result, nil +} + +// getMetadataForTablesInDB finds all the table names in a given database and +// populates the tableMetadata information for all the tables. +func getMetadataForTablesInDB(evalCtx *tree.EvalContext, arg tree.Datums) ([]tableMetadata, error) { + tsI, err := tree.MakeDTimestamp(timeutil.Now(), time.Microsecond) + if err != nil { + return nil, err + } + ts := tsI.String() + dbName := string(tree.MustBeDString(arg[0])) + tableMDs, err := getTableNames(evalCtx, dbName, ts) + if err != nil { + return nil, err + } + + mds := make([]tableMetadata, len(tableMDs)) + for i, dumpTable := range tableMDs { + tableMD, err := getTableMetadata(evalCtx, dbName, dumpTable, ts) + if err != nil { + return nil, err + } + mds[i] = tableMD + } + + return mds, nil +} + +// getTableMetadata populates the metadata for a given table by querying +// crdb_internal.create_statements. +func getTableMetadata( + evalCtx *tree.EvalContext, dbName string, table tableWithSchema, ts string, +) (tableMetadata, error) { + tn := tree.MakeTableNameWithSchema(tree.Name(dbName), tree.Name(table.schema), tree.Name(table.table)) + // Fetch table ID. + query := fmt.Sprintf(` + SELECT + schema_name, + descriptor_id, + create_nofks, + alter_statements, + validate_statements + FROM %s.crdb_internal.create_statements + AS OF SYSTEM TIME %s + WHERE database_name = $1 + AND schema_name = $2 + AND descriptor_name = $3 + `, dbName, ts) + ie := evalCtx.InternalExecutor.(sqlutil.InternalExecutor) + vals, err := ie.QueryRowEx( + evalCtx.Context, + "crdb_internal.show_create_all_tables", + evalCtx.Txn, + sessiondata.NoSessionDataOverride, + query, + dbName, + table.schema, + table.table, + ) + if err != nil { + return tableMetadata{}, err + } + + if len(vals) == 0 { + return tableMetadata{}, nil + } + + // Check the schema to disallow dumping temp tables, views and sequences. This + // will only be triggered if a user explicitly specifies a temp construct as + // one of the arguments to the `cockroach dump` command. When no table names + // are specified on the CLI, we ignore temp tables at the stage where we read + // all table names in getTableNames. + schemaName := string(tree.MustBeDString(vals[0])) + if strings.HasPrefix(schemaName, sessiondata.PgTempSchemaName) { + return tableMetadata{}, errors.Newf("cannot dump temp table %s", tn.String()) + } + + id := int64(tree.MustBeDInt(vals[1])) + createStatement := string(tree.MustBeDString(vals[2])) + alterStatements := extractArray(vals[3]) + validateStatements := extractArray(vals[4]) + + // Get dependencies. + query = fmt.Sprintf(` + SELECT dependson_id + FROM %s.crdb_internal.backward_dependencies + AS OF SYSTEM TIME %s + WHERE descriptor_id = $1 + `, dbName, ts) + it, err := ie.QueryIteratorEx( + evalCtx.Context, + "crdb_internal.show_create_all_tables", + evalCtx.Txn, + sessiondata.NoSessionDataOverride, + query, + id, + ) + if err != nil { + return tableMetadata{}, err + } + + var refs []int64 + var ok bool + for ok, err = it.Next(evalCtx.Context); ok; ok, err = it.Next(evalCtx.Context) { + id := tree.MustBeDInt(it.Cur()[0]) + refs = append(refs, int64(id)) + } + if err != nil { + return tableMetadata{}, err + } + md := tableMetadata{ + ID: id, + name: &tn, + createStmt: createStatement, + dependsOn: refs, + alter: alterStatements, + validate: validateStatements, + } + + return md, nil +} + +// extractArray ensures that a tree.Datum is a DArray and converts +// the DArray into a list of strings. +func extractArray(val tree.Datum) []string { + arr := tree.MustBeDArray(val) + res := make([]string, len(arr.Array)) + for i, v := range arr.Array { + res[i] = string(*v.(*tree.DString)) + } + return res +} + +// getTableNames retrieves all tables names in the given database. Following +// pg_dump, we ignore all descriptors which are part of the temp schema. This +// includes tables, views and sequences. +func getTableNames(evalCtx *tree.EvalContext, dbName string, ts string) ([]tableWithSchema, error) { + query := fmt.Sprintf(` + SELECT schema_name, descriptor_name + FROM "".crdb_internal.create_statements + AS OF SYSTEM TIME %s + WHERE database_name = $1 AND schema_name NOT LIKE $2 + `, ts) + ie := evalCtx.InternalExecutor.(sqlutil.InternalExecutor) + it, err := ie.QueryIteratorEx( + evalCtx.Ctx(), + "crdb_internal.show_create_all_tables", + evalCtx.Txn, + sessiondata.NoSessionDataOverride, + query, + dbName, + sessiondata.PgTempSchemaName+"%", + ) + if err != nil { + return nil, err + } + + var tableNames []tableWithSchema + + var ok bool + for ok, err = it.Next(evalCtx.Context); ok; ok, err = it.Next(evalCtx.Context) { + schema := string(tree.MustBeDString(it.Cur()[0])) + table := string(tree.MustBeDString(it.Cur()[1])) + + tableNames = append(tableNames, tableWithSchema{table: table, schema: schema}) + } + if err != nil { + return tableNames, err + } + + return tableNames, nil +} + +// collect maps a table id to it's tableMetadata and ensures tables are only +// mapped once. +func collect(tid int64, byID map[int64]tableMetadata, seen map[int64]bool, collected *[]int64) { + // has this table already been collected previously? + // We need this check because a table could be multiple times + // if it is referenced. + // For example, if a table references itself, without this check + // collect would infinitely recurse. + if seen[tid] { + return + } + // no: mark it as seen. + seen[tid] = true + for _, dep := range byID[tid].dependsOn { + // depth-first collection of dependencies + collect(dep, byID, seen, collected) + } + // Only add it after its dependencies. + *collected = append(*collected, tid) +} diff --git a/pkg/sql/show_create_all_tables_builtin_test.go b/pkg/sql/show_create_all_tables_builtin_test.go new file mode 100644 index 000000000000..7d5fd09226ea --- /dev/null +++ b/pkg/sql/show_create_all_tables_builtin_test.go @@ -0,0 +1,70 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// Use the output from crdb_internal.show_create_all_tables() to recreate the +// tables and perform another crdb_internal.show_create_all_tables() to ensure +// that the output is the same after recreating the tables. +func TestRecreateTables(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + params, _ := tests.CreateTestServerParams() + s, sqlDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(context.Background()) + + if _, err := sqlDB.Exec(` + CREATE DATABASE test; + USE test; + CREATE TABLE foo(x INT primary key); + CREATE TABLE bar(x INT, y INT, z STRING, FAMILY f1(x, y, z)) + `); err != nil { + t.Fatal(err) + } + + row := sqlDB.QueryRow("SELECT crdb_internal.show_create_all_tables('test')") + var recreateTablesStmt string + if err := row.Scan(&recreateTablesStmt); err != nil { + t.Fatal(err) + } + + // Use the recreateTablesStmt to recreate the tables, perform another + // show_create_all_tables and compare that the output is the same. + if _, err := sqlDB.Exec(` + DROP DATABASE test; + CREATE DATABASE test; + `); err != nil { + t.Fatal(err) + } + if _, err := sqlDB.Exec(recreateTablesStmt); err != nil { + t.Fatal(err) + } + + row = sqlDB.QueryRow("SELECT crdb_internal.show_create_all_tables('test')") + var recreateTablesStmt2 string + if err := row.Scan(&recreateTablesStmt2); err != nil { + t.Fatal(err) + } + + if recreateTablesStmt != recreateTablesStmt2 { + t.Fatalf("got: %s\nexpected: %s", recreateTablesStmt2, recreateTablesStmt) + } +}