Skip to content

Commit

Permalink
Merge pull request #70397 from adityamaru/backport21.2-69947
Browse files Browse the repository at this point in the history
release-21.2: backupccl: fully qualify target tables during scheduled backup
  • Loading branch information
adityamaru authored Nov 16, 2021
2 parents b161573 + f57a6b8 commit a060e9b
Show file tree
Hide file tree
Showing 3 changed files with 226 additions and 12 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,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",
Expand Down
102 changes: 101 additions & 1 deletion pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
Expand Down
135 changes: 124 additions & 11 deletions pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
},
Expand Down Expand Up @@ -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,
},
},
Expand Down Expand Up @@ -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",
},
{
Expand Down

0 comments on commit a060e9b

Please sign in to comment.