Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
69947: backupccl: fully qualify target tables during scheduled backup r=dt a=adityamaru

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

69983: sql: ensure subzones inherit `global_reads` from parent correctly r=nvanbenschoten a=arulajmani

Previously, an index/partition subzone would not inherit the
`global_reads` attribute correctly from the parent zone. This patch
fixes that.

Closes #69647

Release justification: low-risk bug fix for existing functionality
Release note (bug fix): Fixed a bug where index/partition subzones may
not have inherited the `global_reads` field correctly in some cases
from their parent.

69996: sql: fix sql stats not flushing with default flush interval r=maryliag a=Azhng

Previously, if the sql.stats.flush.interval is not set explicitly
via SQL CLI, the flusher would not function properly. This is because
timer.Reset() method is called in the post-clause in the for-loop,
hence it was not properly initialized.
This commit fixes this issue by properly initializing the timer.

Release note: None

Co-authored-by: Anne Zhu <anne.zhu@cockroachlabs.com>
Co-authored-by: arulajmani <arulajmani@gmail.com>
Co-authored-by: Azhng <archer.xn@gmail.com>
  • Loading branch information
4 people committed Sep 10, 2021
4 parents a321f6d + b68d37a + 4c002cf + 9c69fca commit fd070d5
Show file tree
Hide file tree
Showing 9 changed files with 351 additions and 15 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
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 @@ -197,6 +197,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 @@ -335,15 +443,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 @@ -380,13 +491,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 @@ -441,10 +552,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 @@ -473,7 +586,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
Original file line number Diff line number Diff line change
Expand Up @@ -997,6 +997,7 @@ INDEX tbl2@tbl2_i_idx ALTER INDEX tbl2@tbl2_i_idx CONFIGURE ZONE USING
range_min_bytes = 134217728,
range_max_bytes = 536870912,
gc.ttlseconds = 10,
global_reads = true,
num_replicas = 3,
num_voters = 3,
constraints = '{+region=us-east-1: 1}',
Expand Down Expand Up @@ -1355,6 +1356,7 @@ INDEX tbl6@tbl6_i_idx ALTER INDEX tbl6@tbl6_i_idx CONFIGURE ZONE USING
range_min_bytes = 134217728,
range_max_bytes = 536870912,
gc.ttlseconds = 10,
global_reads = true,
num_replicas = 10,
num_voters = 3,
constraints = '{+region=us-east-1: 1}',
Expand Down
69 changes: 69 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/zone
Original file line number Diff line number Diff line change
Expand Up @@ -1155,3 +1155,72 @@ TABLE global ALTER TABLE global CONFIGURE ZONE USING
num_replicas = 7,
constraints = '[]',
lease_preferences = '[]'

# Ensure the global_reads field has correct inheritance semantics for index
# subzones.
subtest regression_69647

statement ok
CREATE table t_69647(pk INT PRIMARY KEY, i INT);
CREATE INDEX i_69647 ON t_69647(i);
ALTER TABLE t_69647 CONFIGURE ZONE USING global_reads=true;

query TT
SHOW ZONE CONFIGURATION FOR INDEX i_69647
----
TABLE test.public.t_69647 ALTER TABLE test.public.t_69647 CONFIGURE ZONE USING
range_min_bytes = 134217728,
range_max_bytes = 536870912,
gc.ttlseconds = 90000,
global_reads = true,
num_replicas = 7,
constraints = '[]',
lease_preferences = '[]'

statement ok
ALTER INDEX i_69647 CONFIGURE ZONE USING num_replicas=5

query TT
SHOW ZONE CONFIGURATION FOR INDEX i_69647
----
INDEX test.public.t_69647@i_69647 ALTER INDEX test.public.t_69647@i_69647 CONFIGURE ZONE USING
range_min_bytes = 134217728,
range_max_bytes = 536870912,
gc.ttlseconds = 90000,
global_reads = true,
num_replicas = 5,
constraints = '[]',
lease_preferences = '[]'

statement ok
ALTER INDEX i_69647 CONFIGURE ZONE USING global_reads = false

query TT
SHOW ZONE CONFIGURATION FOR INDEX i_69647
----
INDEX test.public.t_69647@i_69647 ALTER INDEX test.public.t_69647@i_69647 CONFIGURE ZONE USING
range_min_bytes = 134217728,
range_max_bytes = 536870912,
gc.ttlseconds = 90000,
global_reads = false,
num_replicas = 5,
constraints = '[]',
lease_preferences = '[]'


# Same test as above, but this time for partitions instead of indexes.
statement ok
ALTER TABLE t_69647 PARTITION BY LIST (pk) (PARTITION "one" VALUES IN (1), PARTITION "rest" VALUES IN (DEFAULT));
ALTER PARTITION "one" OF TABLE t_69647 CONFIGURE ZONE USING num_replicas=3;

query TT
SHOW ZONE CONFIGURATION FOR PARTITION "one" OF TABLE t_69647
----
PARTITION one OF TABLE t_69647 ALTER PARTITION one OF TABLE t_69647 CONFIGURE ZONE USING
range_min_bytes = 134217728,
range_max_bytes = 536870912,
gc.ttlseconds = 90000,
global_reads = true,
num_replicas = 3,
constraints = '[]',
lease_preferences = '[]'
Loading

0 comments on commit fd070d5

Please sign in to comment.