From f57a6b8f26eb18a634b1d2764f570a1e818a2f9c Mon Sep 17 00:00:00 2001 From: Anne Zhu Date: Fri, 25 Jun 2021 15:18:48 -0400 Subject: [PATCH] backupccl: fully qualify target tables during scheduled backup This change adds logic to fully qualify the table names that are specified as targets during a scheduled backup. It does this so that the backup statement that is created and stored during planning of the scheduled job has FQ table names. This is necessary because the actual execution of the backup job occurs in a background, schedueled job session with different resolution configurations. Fixes: #66450 Release note: None Release justification: bug fixes and low-risk updates to new functionality --- pkg/ccl/backupccl/BUILD.bazel | 1 + pkg/ccl/backupccl/create_scheduled_backup.go | 102 ++++++++++++- .../backupccl/create_scheduled_backup_test.go | 135 ++++++++++++++++-- 3 files changed, 226 insertions(+), 12 deletions(-) diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 08c21d4e7890..a3a4b7c4b392 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -68,6 +68,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/multiregion", + "//pkg/sql/catalog/resolver", "//pkg/sql/catalog/schemadesc", "//pkg/sql/catalog/schemaexpr", "//pkg/sql/catalog/systemschema", diff --git a/pkg/ccl/backupccl/create_scheduled_backup.go b/pkg/ccl/backupccl/create_scheduled_backup.go index 61f3b635820a..b2225ddf265b 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup.go +++ b/pkg/ccl/backupccl/create_scheduled_backup.go @@ -26,7 +26,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -693,13 +696,110 @@ func dryRunInvokeBackup(ctx context.Context, p sql.PlanHookState, backupNode *tr return invokeBackup(ctx, backupFn) } +func fullyQualifyScheduledBackupTargetTables( + ctx context.Context, p sql.PlanHookState, tables tree.TablePatterns, +) ([]tree.TablePattern, error) { + fqTablePatterns := make([]tree.TablePattern, len(tables)) + for i, target := range tables { + tablePattern, err := target.NormalizeTablePattern() + if err != nil { + return nil, err + } + switch tp := tablePattern.(type) { + case *tree.TableName: + if err := sql.DescsTxn(ctx, p.ExecCfg(), func(ctx context.Context, txn *kv.Txn, + col *descs.Collection) error { + // Resolve the table. + un := tp.ToUnresolvedObjectName() + found, _, tableDesc, err := resolver.ResolveExisting(ctx, un, p, tree.ObjectLookupFlags{}, + p.CurrentDatabase(), p.CurrentSearchPath()) + if err != nil { + return err + } + if !found { + return errors.Newf("target table %s could not be resolved", tp.String()) + } + + // Resolve the database. + found, dbDesc, err := col.GetImmutableDatabaseByID(ctx, txn, tableDesc.GetParentID(), + tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return err + } + if !found { + return errors.Newf("database of target table %s could not be resolved", tp.String()) + } + + // Resolve the schema. + schemaDesc, err := col.GetImmutableSchemaByID(ctx, txn, tableDesc.GetParentSchemaID(), + tree.SchemaLookupFlags{Required: true}) + if err != nil { + return err + } + tn := tree.NewTableNameWithSchema( + tree.Name(dbDesc.GetName()), + tree.Name(schemaDesc.GetName()), + tree.Name(tableDesc.GetName()), + ) + fqTablePatterns[i] = tn + return nil + }); err != nil { + return nil, err + } + case *tree.AllTablesSelector: + if !tp.ExplicitSchema { + tp.ExplicitSchema = true + tp.SchemaName = tree.Name(p.CurrentDatabase()) + } else if tp.ExplicitSchema && !tp.ExplicitCatalog { + // The schema field could either be a schema or a database. If we can + // successfully resolve the schema, we will add the DATABASE prefix. + // Otherwise, no updates are needed since the schema field refers to the + // database. + var resolvedSchema bool + if err := sql.DescsTxn(ctx, p.ExecCfg(), func(ctx context.Context, txn *kv.Txn, + col *descs.Collection) error { + dbDesc, err := col.GetImmutableDatabaseByName(ctx, txn, p.CurrentDatabase(), + tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return err + } + resolvedSchema, _, err = catalogkv.ResolveSchemaID(ctx, txn, p.ExecCfg().Codec, + dbDesc.GetID(), tp.SchemaName.String()) + return err + }); err != nil { + return nil, err + } + + if resolvedSchema { + tp.ExplicitCatalog = true + tp.CatalogName = tree.Name(p.CurrentDatabase()) + } + } + fqTablePatterns[i] = tp + } + } + return fqTablePatterns, nil +} + // makeScheduleBackupEval prepares helper scheduledBackupEval struct to assist in evaluation // of various schedule and backup specific components. func makeScheduledBackupEval( ctx context.Context, p sql.PlanHookState, schedule *tree.ScheduledBackup, ) (*scheduledBackupEval, error) { - eval := &scheduledBackupEval{ScheduledBackup: schedule} var err error + if schedule.Targets != nil && schedule.Targets.Tables != nil { + // Table backup targets must be fully qualified during scheduled backup + // planning. This is because the actual execution of the backup job occurs + // in a background, scheduled job session, that does not have the same + // resolution configuration as during planning. + schedule.Targets.Tables, err = fullyQualifyScheduledBackupTargetTables(ctx, p, + schedule.Targets.Tables) + if err != nil { + return nil, errors.Wrap(err, "qualifying backup target tables") + } + } + + eval := &scheduledBackupEval{ScheduledBackup: schedule} if schedule.ScheduleLabelSpec.Label != nil { eval.scheduleLabel, err = p.TypeAsString(ctx, schedule.ScheduleLabelSpec.Label, scheduleBackupOp) diff --git a/pkg/ccl/backupccl/create_scheduled_backup_test.go b/pkg/ccl/backupccl/create_scheduled_backup_test.go index 3f7d345c2ddc..0875c18e7b27 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup_test.go +++ b/pkg/ccl/backupccl/create_scheduled_backup_test.go @@ -193,6 +193,114 @@ func (t userType) String() string { return "enterprise user" } +func TestScheduledTableBackupNameQualification(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + th, cleanup := newTestHelper(t) + defer cleanup() + + th.sqlDB.Exec(t, ` +CREATE DATABASE mydb; +USE mydb; + +CREATE TABLE t1(a int); +INSERT INTO t1 values (1), (10), (100); + +CREATE TABLE t2(b int); +INSERT INTO t2 VALUES (3), (2), (1); + +CREATE TABLE t3(c int); +INSERT INTO t3 VALUES (5), (5), (7); + +CREATE TABLE "my.tbl"(d int); + +CREATE SCHEMA myschema; +CREATE TABLE myschema.mytbl(a int); + +CREATE DATABASE other_db; +CREATE TABLE other_db.t1(a int); +`) + + testCases := []struct { + name string + query string + expectedBackupStmt string + }{ + { + name: "fully-qualified-table-name", + query: "CREATE SCHEDULE FOR BACKUP mydb.public.t1 INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.public.t1 INTO '%s' WITH detached", + }, + { + name: "schema-qualified-table-name", + query: "CREATE SCHEDULE FOR BACKUP public.t1 INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.public.t1 INTO '%s' WITH detached", + }, + { + name: "uds-qualified-table-name", + query: "CREATE SCHEDULE FOR BACKUP myschema.mytbl INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.myschema.mytbl INTO '%s' WITH detached", + }, + { + name: "db-qualified-table-name", + query: "CREATE SCHEDULE FOR BACKUP mydb.t1 INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.public.t1 INTO '%s' WITH detached", + }, + { + name: "unqualified-table-name", + query: "CREATE SCHEDULE FOR BACKUP t1 INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.public.t1 INTO '%s' WITH detached", + }, + { + name: "unqualified-table-name-with-symbols", + query: `CREATE SCHEDULE FOR BACKUP "my.tbl" INTO $1 RECURRING '@hourly'`, + expectedBackupStmt: `BACKUP TABLE mydb.public."my.tbl" INTO '%s' WITH detached`, + }, + { + name: "table-names-from-different-db", + query: "CREATE SCHEDULE FOR BACKUP t1, other_db.t1 INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.public.t1, other_db.public.t1 INTO '%s' WITH detached", + }, + { + name: "unqualified-all-tables-selectors", + query: "CREATE SCHEDULE FOR BACKUP * INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.public.* INTO '%s' WITH detached", + }, + { + name: "all-tables-selectors-with-user-defined-schema", + query: "CREATE SCHEDULE FOR BACKUP myschema.* INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.myschema.* INTO '%s' WITH detached", + }, + { + name: "partially-qualified-all-tables-selectors-with-different-db", + query: "CREATE SCHEDULE FOR BACKUP other_db.* INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE other_db.public.* INTO '%s' WITH detached", + }, + { + name: "fully-qualified-all-tables-selectors-with-multiple-dbs", + query: "CREATE SCHEDULE FOR BACKUP *, other_db.* INTO $1 RECURRING '@hourly'", + expectedBackupStmt: "BACKUP TABLE mydb.public.*, other_db.public.* INTO '%s' WITH detached", + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + defer th.clearSchedules(t) + defer utilccl.TestingDisableEnterprise()() + + destination := "nodelocal://0/backup/" + tc.name + schedules, err := th.createBackupSchedule(t, tc.query, destination) + require.NoError(t, err) + + for _, s := range schedules { + stmt := getScheduledBackupStatement(t, s.ExecutionArgs()) + require.Equal(t, fmt.Sprintf(tc.expectedBackupStmt, destination), stmt) + } + }) + } +} + // This test examines serialized representation of backup schedule arguments // when the scheduled backup statement executes. This test does not concern // itself with the actual scheduling and the execution of those backups. @@ -331,15 +439,18 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { WITH revision_history RECURRING '@hourly'`, expectedSchedules: []expectedSchedule{ { - nameRe: "BACKUP .*", - backupStmt: "BACKUP TABLE system.jobs, system.scheduled_jobs INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached", + nameRe: "BACKUP .*", + backupStmt: "BACKUP TABLE system.public.jobs, " + + "system.public.scheduled_jobs INTO LATEST IN 'nodelocal://0/backup' WITH" + + " revision_history, detached", period: time.Hour, paused: true, chainProtectedTimestampRecord: true, }, { - nameRe: "BACKUP .+", - backupStmt: "BACKUP TABLE system.jobs, system.scheduled_jobs INTO 'nodelocal://0/backup' WITH revision_history, detached", + nameRe: "BACKUP .+", + backupStmt: "BACKUP TABLE system.public.jobs, " + + "system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH revision_history, detached", period: 24 * time.Hour, runsNow: true, chainProtectedTimestampRecord: true, @@ -376,13 +487,13 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { expectedSchedules: []expectedSchedule{ { nameRe: "BACKUP .*", - backupStmt: "BACKUP TABLE system.* INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP TABLE system.public.* INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached", period: time.Hour, paused: true, }, { nameRe: "BACKUP .+", - backupStmt: "BACKUP TABLE system.* INTO 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP TABLE system.public.* INTO 'nodelocal://0/backup' WITH revision_history, detached", period: 24 * time.Hour, runsNow: true, }, @@ -437,10 +548,12 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { expectedSchedules: []expectedSchedule{ { nameRe: "BACKUP .*", - backupStmt: "BACKUP TABLE system.jobs, " + - "system.scheduled_jobs INTO 'nodelocal://0/backup' WITH revision_history, encryption_passphrase = 'secret', detached", - shownStmt: "BACKUP TABLE system.jobs, " + - "system.scheduled_jobs INTO 'nodelocal://0/backup' WITH revision_history, encryption_passphrase = '*****', detached", + backupStmt: "BACKUP TABLE system.public.jobs, " + + "system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH" + + " revision_history, encryption_passphrase = 'secret', detached", + shownStmt: "BACKUP TABLE system.public.jobs, " + + "system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH" + + " revision_history, encryption_passphrase = '*****', detached", period: 7 * 24 * time.Hour, }, }, @@ -469,7 +582,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { }, { name: "missing-destination-placeholder", - query: `CREATE SCHEDULE FOR BACKUP TABLE t INTO $1 RECURRING '@hourly'`, + query: `CREATE SCHEDULE FOR BACKUP TABLE system.public.jobs INTO $1 RECURRING '@hourly'`, errMsg: "failed to evaluate backup destination paths", }, {