Skip to content

Commit

Permalink
backupccl: remove index/bytes columns from backup/restore return stmt
Browse files Browse the repository at this point in the history
This removes the index entries and bytes columns from the return result
of a backup and restore. 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 statements
no longer return index entries and bytes backed up/restored.
  • Loading branch information
kev-cao committed Dec 12, 2024
1 parent 7575548 commit 9b67da0
Show file tree
Hide file tree
Showing 10 changed files with 42 additions and 78 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
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
2 changes: 1 addition & 1 deletion pkg/ccl/telemetryccl/telemetry_logging_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -359,7 +359,7 @@ func TestBulkJobTelemetryLogging(t *testing.T) {
if strings.Contains(tc.query, "WITH detached") {
err = db.DB.QueryRowContext(ctx, tc.query).Scan(&jobID)
} else {
err = db.DB.QueryRowContext(ctx, tc.query).Scan(&jobID, &unused, &unused, &unused, &unused, &unused)
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
29 changes: 17 additions & 12 deletions pkg/ccl/workloadccl/fixture.go
Original file line number Diff line number Diff line change
Expand Up @@ -504,16 +504,18 @@ func importFixtureTable(
return 0, err
}
} else {
if err := res.Scan(
&discard, &discard, &discard, &rows, &index, &tableBytes,
); err != nil {
if err := res.Scan(&discard, &discard, &discard, &rows); err != nil {
return 0, err
}
}
elapsed := timeutil.Since(start)
log.Infof(ctx, `imported %s in %s table (%d rows, %d index entries, took %s, %s)`,
humanizeutil.IBytes(tableBytes), table.Name, rows, index, elapsed,
humanizeutil.DataRate(tableBytes, elapsed))
if len(resCols) == 7 {
log.Infof(ctx, `imported %s in %s table (%d rows, %d index entries, took %s, %s)`,
humanizeutil.IBytes(tableBytes), table.Name, rows, index, elapsed,
humanizeutil.DataRate(tableBytes, elapsed))
} else {
log.Infof(ctx, `imported %s table (%d rows, took %s)`, table.Name, rows, elapsed)
}

// Inject pre-calculated stats.
if injectStats && len(table.Stats) > 0 {
Expand Down Expand Up @@ -652,17 +654,20 @@ func RestoreFixture(
return err
}
} else {
if err := res.Scan(
&discard, &discard, &discard, &rows, &index, &tableBytes,
); err != nil {
if err := res.Scan(&discard, &discard, &discard, &rows); err != nil {
return err
}
}
atomic.AddInt64(&bytesAtomic, tableBytes)
elapsed := timeutil.Since(start)
log.Infof(ctx, `loaded %s table %s in %s (%d rows, %d index entries, %s)`,
humanizeutil.IBytes(tableBytes), table.TableName, elapsed, rows, index,
humanizeutil.IBytes(int64(float64(tableBytes)/elapsed.Seconds())))
if len(resCols) == 7 {
log.Infof(ctx, `loaded %s table %s in %s (%d rows, %d index entries, %s)`,
humanizeutil.IBytes(tableBytes), table.TableName, elapsed, rows, index,
humanizeutil.IBytes(int64(float64(tableBytes)/elapsed.Seconds())))
} else {
log.Infof(ctx, `loaded table %s in %s (%d rows)`, table.TableName, elapsed, rows)

}
return nil
})
}
Expand Down
4 changes: 1 addition & 3 deletions pkg/jobs/resultcols.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@ var BulkJobExecutionResultHeader = colinfo.ResultColumns{
{Name: "status", Typ: types.String},
{Name: "fraction_completed", Typ: types.Float},
{Name: "rows", Typ: types.Int},
{Name: "index_entries", Typ: types.Int},
{Name: "bytes", Typ: types.Int},
}

// OnlineRestoreJobExecutionResultHeader is the header for an online restore
Expand All @@ -31,7 +29,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
2 changes: 0 additions & 2 deletions pkg/sql/importer/import_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1809,8 +1809,6 @@ func (r *importResumer) ReportResults(ctx context.Context, resultsCh chan<- tree
tree.NewDString(string(jobs.StatusSucceeded)),
tree.NewDFloat(tree.DFloat(1.0)),
tree.NewDInt(tree.DInt(r.res.Rows)),
tree.NewDInt(tree.DInt(r.res.IndexEntries)),
tree.NewDInt(tree.DInt(r.res.DataSize)),
}:
return nil
case <-ctx.Done():
Expand Down
26 changes: 9 additions & 17 deletions pkg/sql/importer/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2406,9 +2406,7 @@ func TestImportCSVStmt(t *testing.T) {
sqlDB.Exec(t, fmt.Sprintf(`SET DATABASE = %s`, intodb))

var unused string
var restored struct {
rows, idx, bytes int
}
var rows int

if tc.createQuery != "" {
sqlDB.Exec(t, tc.createQuery)
Expand All @@ -2421,9 +2419,7 @@ func TestImportCSVStmt(t *testing.T) {
sqlDB.ExpectErr(t, tc.err, query)
return
}
sqlDB.QueryRow(t, query).Scan(
&unused, &unused, &unused, &restored.rows, &restored.idx, &restored.bytes,
)
sqlDB.QueryRow(t, query).Scan(&unused, &unused, &unused, &rows)

jobPrefix := fmt.Sprintf(`IMPORT INTO %s.public.t`, intodb)

Expand Down Expand Up @@ -2455,7 +2451,7 @@ func TestImportCSVStmt(t *testing.T) {
return
}

if expected, actual := expectedRows, restored.rows; expected != actual {
if expected, actual := expectedRows, rows; expected != actual {
t.Fatalf("expected %d rows, got %d", expected, actual)
}

Expand Down Expand Up @@ -3234,9 +3230,7 @@ func TestImportIntoCSV(t *testing.T) {
sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 't'`).Scan(&tableID)

var unused string
var restored struct {
rows, idx, bytes int
}
var rows int

// Insert the test data
insert := []string{"''", "'text'", "'a'", "'e'", "'l'", "'t'", "'z'"}
Expand All @@ -3255,7 +3249,7 @@ func TestImportIntoCSV(t *testing.T) {
}

sqlDB.QueryRow(t, query).Scan(
&unused, &unused, &unused, &restored.rows, &restored.idx, &restored.bytes,
&unused, &unused, &unused, &rows,
)

jobPrefix := `IMPORT INTO defaultdb.public.t(a, b)`
Expand All @@ -3276,7 +3270,7 @@ func TestImportIntoCSV(t *testing.T) {
return
}

if expected, actual := insertedRows, restored.rows; expected != actual {
if expected, actual := insertedRows, rows; expected != actual {
t.Fatalf("expected %d rows, got %d", expected, actual)
}

Expand Down Expand Up @@ -3391,7 +3385,7 @@ func TestImportIntoCSV(t *testing.T) {
g.GoCtx(func(ctx context.Context) error {
defer close(importBodyFinished)
return sqlDB.DB.QueryRowContext(ctx, fmt.Sprintf(`IMPORT INTO t (a, b) CSV DATA (%s)`,
testFiles.files[1])).Scan(&jobID, &unused, &unused, &unused, &unused, &unused)
testFiles.files[1])).Scan(&jobID, &unused, &unused, &unused)
})
g.GoCtx(func(ctx context.Context) error {
defer close(delayImportFinish)
Expand Down Expand Up @@ -6033,8 +6027,7 @@ func TestImportPgDumpIgnoredStmts(t *testing.T) {
var importJobID int
var unused interface{}
sqlDB.QueryRow(t, "IMPORT PGDUMP ($1) WITH ignore_unsupported_statements, "+
"log_ignored_statements=$2", srv.URL, ignoredLog).Scan(&importJobID, &unused, &unused,
&unused, &unused, &unused)
"log_ignored_statements=$2", srv.URL, ignoredLog).Scan(&importJobID, &unused, &unused, &unused)
// Check that statements which are not expected to be ignored, are still
// processed.
sqlDB.CheckQueryResults(t, "SELECT * FROM foo", [][]string{{"1"}, {"2"}, {"3"}})
Expand Down Expand Up @@ -7077,8 +7070,7 @@ func TestImportJobEventLogging(t *testing.T) {
var jobID int64
var unused interface{}
sqlDB.Exec(t, createQuery)
sqlDB.QueryRow(t, importQuery, simpleOcf).Scan(&jobID, &unused, &unused, &unused, &unused,
&unused)
sqlDB.QueryRow(t, importQuery, simpleOcf).Scan(&jobID, &unused, &unused, &unused)

expectedStatus := []string{string(jobs.StatusSucceeded), string(jobs.StatusRunning)}
expectedRecoveryEvent := eventpb.RecoveryEvent{
Expand Down

0 comments on commit 9b67da0

Please sign in to comment.