Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
134516: backupccl: remove index/bytes columns from backup/restore return stmt r=msbutler a=kev-cao

This removes the index entries and bytes columns from the return result of a backup/restore/import. This columns do not inform the user of anything useful and only serve to mislead them.

Fixes: #133053

Epic: None

Release note (backward-incompatible change): Backup/restore/import statements no longer return index entries and bytes backed up/restored.

137584: sql: avoid invalid SQL in `raw_config_sql` in `crdb_internal.zones` r=yuzefovich a=yuzefovich

Previously, we could include an invalid `ALTER ... CONFIGURE ZONE USING;` stmt in the `raw_config_sql` column of `crdb_internal.zones`. This was the case when some of the config was inherited and we didn't want to include it for the corresponding object. The contract of this virtual tables allows NULLs in this column, so this commit returns NULL in such cases. Note that I spent non-trivial amount of time trying to figure out how we could end up in this situation but was unsuccessful.

This invalid stmt was included into the output of SHOW CREATE TABLE which, among other things, we use when recreating the bundle, so we should now have easier time recreating multi-region bundles.

Fixes: #123904.

Release note: None

Co-authored-by: Kevin Cao <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
3 people committed Dec 17, 2024
3 parents ed5e1fe + 03e2409 + e862503 commit 8e24b34
Show file tree
Hide file tree
Showing 10 changed files with 50 additions and 61 deletions.
2 changes: 0 additions & 2 deletions pkg/backup/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1023,8 +1023,6 @@ func (b *backupResumer) ReportResults(ctx context.Context, resultsCh chan<- tree
tree.NewDString(string(jobs.StatusSucceeded)),
tree.NewDFloat(tree.DFloat(1.0)),
tree.NewDInt(tree.DInt(b.backupStats.Rows)),
tree.NewDInt(tree.DInt(b.backupStats.IndexEntries)),
tree.NewDInt(tree.DInt(b.backupStats.DataSize)),
}:
return nil
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/backup/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -376,7 +376,7 @@ func backupTypeCheck(
if detached {
header = jobs.DetachedJobExecutionResultHeader
} else {
header = jobs.BulkJobExecutionResultHeader
header = jobs.BackupRestoreJobResultHeader
}
if err := exprutil.TypeCheck(
ctx, "BACKUP", p.SemaCtx(),
Expand Down Expand Up @@ -740,7 +740,7 @@ func backupPlanHook(
if detached {
return fn, jobs.DetachedJobExecutionResultHeader, nil, false, nil
}
return fn, jobs.BulkJobExecutionResultHeader, nil, false, nil
return fn, jobs.BackupRestoreJobResultHeader, nil, false, nil
}

func logAndSanitizeKmsURIs(ctx context.Context, kmsURIs ...string) error {
Expand Down
47 changes: 11 additions & 36 deletions pkg/backup/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -853,17 +853,13 @@ func TestBackupRestoreNegativePrimaryKey(t *testing.T) {
sqlDB.Exec(t, `CREATE UNIQUE INDEX id2 ON data.bank (id)`)

var unused string
var exportedRows, exportedIndexEntries int
var exportedRows int
sqlDB.QueryRow(t, `BACKUP DATABASE data INTO $1`, localFoo+"/alteredPK").Scan(
&unused, &unused, &unused, &exportedRows, &exportedIndexEntries, &unused,
&unused, &unused, &unused, &exportedRows,
)
if exportedRows != numAccounts {
t.Fatalf("expected %d rows, got %d", numAccounts, exportedRows)
}
expectedIndexEntries := numAccounts * 2 // Indexes id2 and balance_idx
if exportedIndexEntries != expectedIndexEntries {
t.Fatalf("expected %d index entries, got %d", expectedIndexEntries, exportedIndexEntries)
}
}

func backupAndRestore(
Expand Down Expand Up @@ -893,9 +889,7 @@ func backupAndRestore(
})

var unused string
var exported struct {
rows, idx, bytes int64
}
var exportedRows int64

backupURIFmtString, backupURIArgs := uriFmtStringAndArgs(backupURIs, 0)
backupQuery := fmt.Sprintf("BACKUP DATABASE data INTO %s", backupURIFmtString)
Expand All @@ -907,19 +901,10 @@ func backupAndRestore(
}
queryArgs := append(backupURIArgs, kmsURIArgs...)
sqlDB.QueryRow(t, backupQuery, queryArgs...).Scan(
&unused, &unused, &unused, &exported.rows, &exported.idx, &exported.bytes,
&unused, &unused, &unused, &exportedRows,
)
// When numAccounts == 0, our approxBytes formula breaks down because
// backups of no data still contain the system.users and system.descriptor
// tables. Just skip the check in this case.
if numAccounts > 0 {
approxBytes := int64(backupRestoreRowPayloadSize * numAccounts)
if max := approxBytes * 3; exported.bytes < approxBytes || exported.bytes > max {
t.Errorf("expected data size in [%d,%d] but was %d", approxBytes, max, exported.bytes)
}
}
if expected := int64(numAccounts * 1); exported.rows != expected {
t.Fatalf("expected %d rows for %d accounts, got %d", expected, numAccounts, exported.rows)
if expected := int64(numAccounts * 1); exportedRows != expected {
t.Fatalf("expected %d rows for %d accounts, got %d", expected, numAccounts, exportedRows)
}

found := false
Expand Down Expand Up @@ -1011,22 +996,13 @@ func verifyRestoreData(
) {

var unused string
var restored struct {
rows, idx, bytes int64
}
var restoredRows int64
sqlDB.QueryRow(t, restoreQuery, restoreURIArgs...).Scan(
&unused, &unused, &unused, &restored.rows, &restored.idx, &restored.bytes,
&unused, &unused, &unused, &restoredRows,
)

approxBytes := int64(backupRestoreRowPayloadSize * numAccounts)
if max := approxBytes * 3; restored.bytes < approxBytes || restored.bytes > max {
t.Errorf("expected data size in [%d,%d] but was %d", approxBytes, max, restored.bytes)
}
if expected := int64(numAccounts); restored.rows != expected {
t.Fatalf("expected %d rows for %d accounts, got %d", expected, numAccounts, restored.rows)
}
if expected := int64(numAccounts); restored.idx != expected {
t.Fatalf("expected %d idx rows for %d accounts, got %d", expected, numAccounts, restored.idx)
if expected := int64(numAccounts); restoredRows != expected {
t.Fatalf("expected %d rows for %d accounts, got %d", expected, numAccounts, restoredRows)
}

var rowCount int64
Expand Down Expand Up @@ -8675,8 +8651,7 @@ func TestRestoreJobEventLogging(t *testing.T) {

var jobID int64
var unused interface{}
sqlDB.QueryRow(t, restoreQuery).Scan(&jobID, &unused, &unused, &unused, &unused,
&unused)
sqlDB.QueryRow(t, restoreQuery).Scan(&jobID, &unused, &unused, &unused)

expectedStatus := []string{string(jobs.StatusSucceeded), string(jobs.StatusRunning)}
expectedRecoveryEvent := eventpb.RecoveryEvent{
Expand Down
4 changes: 2 additions & 2 deletions pkg/backup/backuptestutils/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -225,7 +225,7 @@ func VerifyBackupRestoreStatementResult(
return err
}
if a, e := columns, []string{
"job_id", "status", "fraction_completed", "rows", "index_entries", "bytes",
"job_id", "status", "fraction_completed", "rows",
}; !reflect.DeepEqual(e, a) {
return errors.Errorf("unexpected columns:\n%s", strings.Join(pretty.Diff(e, a), "\n"))
}
Expand All @@ -247,7 +247,7 @@ func VerifyBackupRestoreStatementResult(
return errors.New("zero rows in result")
}
if err := rows.Scan(
&actualJob.id, &actualJob.status, &actualJob.fractionCompleted, &unused, &unused, &unused,
&actualJob.id, &actualJob.status, &actualJob.fractionCompleted, &unused,
); err != nil {
return err
}
Expand Down
2 changes: 0 additions & 2 deletions pkg/backup/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -2195,8 +2195,6 @@ func (r *restoreResumer) ReportResults(ctx context.Context, resultsCh chan<- tre
tree.NewDString(string(jobs.StatusSucceeded)),
tree.NewDFloat(tree.DFloat(1.0)),
tree.NewDInt(tree.DInt(r.restoreStats.Rows)),
tree.NewDInt(tree.DInt(r.restoreStats.IndexEntries)),
tree.NewDInt(tree.DInt(r.restoreStats.DataSize)),
}
}
}():
Expand Down
2 changes: 1 addition & 1 deletion pkg/backup/restore_old_sequences_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ func restoreOldSequencesTest(exportDir string, isSchemaOnly bool) func(t *testin
restoreQuery = restoreQuery + ", schema_only"
}
sqlDB.QueryRow(t, restoreQuery, localFoo).Scan(
&unused, &unused, &unused, &importedRows, &unused, &unused,
&unused, &unused, &unused, &importedRows,
)
totalRows := 4
if isSchemaOnly {
Expand Down
4 changes: 2 additions & 2 deletions pkg/backup/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -1189,7 +1189,7 @@ func restoreTypeCheck(
} else if restoreStmt.Options.ExperimentalOnline {
header = jobs.OnlineRestoreJobExecutionResultHeader
} else {
header = jobs.BulkJobExecutionResultHeader
header = jobs.BackupRestoreJobResultHeader
}
return true, header, nil
}
Expand Down Expand Up @@ -1426,7 +1426,7 @@ func restorePlanHook(
} else if restoreStmt.Options.ExperimentalOnline {
header = jobs.OnlineRestoreJobExecutionResultHeader
} else {
header = jobs.BulkJobExecutionResultHeader
header = jobs.BackupRestoreJobResultHeader
}
return fn, header, nil, false, nil
}
Expand Down
4 changes: 3 additions & 1 deletion pkg/ccl/telemetryccl/telemetry_logging_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -358,8 +358,10 @@ func TestBulkJobTelemetryLogging(t *testing.T) {

if strings.Contains(tc.query, "WITH detached") {
err = db.DB.QueryRowContext(ctx, tc.query).Scan(&jobID)
} else {
} else if strings.HasPrefix(tc.query, "IMPORT") {
err = db.DB.QueryRowContext(ctx, tc.query).Scan(&jobID, &unused, &unused, &unused, &unused, &unused)
} else {
err = db.DB.QueryRowContext(ctx, tc.query).Scan(&jobID, &unused, &unused, &unused)
}
if err != nil {
t.Errorf("unexpected error executing query `%s`: %v", tc.query, err)
Expand Down
9 changes: 8 additions & 1 deletion pkg/jobs/resultcols.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,13 @@ var BulkJobExecutionResultHeader = colinfo.ResultColumns{
{Name: "bytes", Typ: types.Int},
}

var BackupRestoreJobResultHeader = colinfo.ResultColumns{
{Name: "job_id", Typ: types.Int},
{Name: "status", Typ: types.String},
{Name: "fraction_completed", Typ: types.Float},
{Name: "rows", Typ: types.Int},
}

// OnlineRestoreJobExecutionResultHeader is the header for an online restore
// job, which provides a header different from the usual bulk job execution
var OnlineRestoreJobExecutionResultHeader = colinfo.ResultColumns{
Expand All @@ -31,7 +38,7 @@ var OnlineRestoreJobExecutionResultHeader = colinfo.ResultColumns{
{Name: "background_download_job_id", Typ: types.Int},
}

// DetachedJobExecutionResultHeader is a the header for various job commands when
// DetachedJobExecutionResultHeader is the header for various job commands when
// job executes in detached mode (i.e. the caller doesn't wait for job completion).
var DetachedJobExecutionResultHeader = colinfo.ResultColumns{
{Name: "job_id", Typ: types.Int},
Expand Down
33 changes: 21 additions & 12 deletions pkg/sql/show_zone_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,34 +158,35 @@ func getShowZoneConfigRow(
}

// zoneConfigToSQL pretty prints a zone configuration as a SQL string.
func zoneConfigToSQL(zs *tree.ZoneSpecifier, zone *zonepb.ZoneConfig) (string, error) {
func zoneConfigToSQL(zs *tree.ZoneSpecifier, zone *zonepb.ZoneConfig) (tree.Datum, error) {
constraints, err := yamlMarshalFlow(zonepb.ConstraintsList{
Constraints: zone.Constraints,
Inherited: zone.InheritedConstraints})
if err != nil {
return "", err
return tree.DNull, err
}
constraints = strings.TrimSpace(constraints)
voterConstraints, err := yamlMarshalFlow(zonepb.ConstraintsList{
Constraints: zone.VoterConstraints,
Inherited: zone.InheritedVoterConstraints(),
})
if err != nil {
return "", err
return tree.DNull, err
}
voterConstraints = strings.TrimSpace(voterConstraints)
prefs, err := yamlMarshalFlow(zone.LeasePreferences)
if err != nil {
return "", err
return tree.DNull, err
}
prefs = strings.TrimSpace(prefs)

useComma := false
first := true
maybeWriteComma := func(f *tree.FmtCtx) {
if useComma {
if !first {
f.Printf(",\n")
} else {
first = false
}
useComma = true
}

f := tree.NewFmtCtx(tree.FmtParsable)
Expand Down Expand Up @@ -228,7 +229,13 @@ func zoneConfigToSQL(zs *tree.ZoneSpecifier, zone *zonepb.ZoneConfig) (string, e
maybeWriteComma(f)
f.Printf("\tlease_preferences = %s", lexbase.EscapeSQLString(prefs))
}
return f.String(), nil
if first {
// We didn't include any zone config parameters, so rather than
// returning an invalid 'ALTER ... CONFIGURE ZONE USING;' stmt we'll
// return NULL.
return tree.DNull, nil
}
return tree.NewDString(f.String()), nil
}

// generateZoneConfigIntrospectionValues creates a result row
Expand Down Expand Up @@ -293,11 +300,12 @@ func generateZoneConfigIntrospectionValues(
if zs == nil {
values[rawConfigSQLCol] = tree.DNull
} else {
sqlStr, err := zoneConfigToSQL(zs, zone)
var d tree.Datum
d, err = zoneConfigToSQL(zs, zone)
if err != nil {
return err
}
values[rawConfigSQLCol] = tree.NewDString(sqlStr)
values[rawConfigSQLCol] = d
}

// Populate the protobuf column.
Expand All @@ -322,11 +330,12 @@ func generateZoneConfigIntrospectionValues(
if zs == nil {
values[fullConfigSQLCol] = tree.DNull
} else {
sqlStr, err := zoneConfigToSQL(zs, inheritedConfig)
var d tree.Datum
d, err = zoneConfigToSQL(zs, inheritedConfig)
if err != nil {
return err
}
values[fullConfigSQLCol] = tree.NewDString(sqlStr)
values[fullConfigSQLCol] = d
}
return nil
}
Expand Down

0 comments on commit 8e24b34

Please sign in to comment.