From d8d416fd0e2f62789c38b354dd04813befa547b4 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Tue, 28 May 2024 15:51:53 -0400 Subject: [PATCH 1/2] backupccl: indicate if manifest entry contains range keys Epic: none Release note: none --- pkg/ccl/backupccl/backuppb/backup.proto | 2 ++ pkg/ccl/backupccl/file_sst_sink.go | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/pkg/ccl/backupccl/backuppb/backup.proto b/pkg/ccl/backupccl/backuppb/backup.proto index 35d5525711f7..240d0a0861a2 100644 --- a/pkg/ccl/backupccl/backuppb/backup.proto +++ b/pkg/ccl/backupccl/backuppb/backup.proto @@ -55,6 +55,8 @@ message BackupManifest { // ApproximatePhysicalSize is the approximate size of the physical bytes in // compressed SST form. uint64 approximate_physical_size = 11; + + bool has_range_keys = 12; } message DescriptorRevision { diff --git a/pkg/ccl/backupccl/file_sst_sink.go b/pkg/ccl/backupccl/file_sst_sink.go index 1e483afef324..4169acbacf7a 100644 --- a/pkg/ccl/backupccl/file_sst_sink.go +++ b/pkg/ccl/backupccl/file_sst_sink.go @@ -266,6 +266,7 @@ func (s *fileSSTSink) write(ctx context.Context, resp exportedSpan) (roachpb.Key if err != nil { return nil, err } + hasRangeKeys := maxRange != nil // extend determines if the new span should be added to the last span. This // will occur if the previous span ended mid row, or if the new span is a @@ -299,11 +300,13 @@ func (s *fileSSTSink) write(ctx context.Context, resp exportedSpan) (roachpb.Key s.flushedFiles[l].Span.EndKey = span.EndKey s.flushedFiles[l].EntryCounts.Add(resp.metadata.EntryCounts) s.flushedFiles[l].ApproximatePhysicalSize += resp.metadata.ApproximatePhysicalSize + s.flushedFiles[l].HasRangeKeys = s.flushedFiles[l].HasRangeKeys || hasRangeKeys s.stats.spanGrows++ } else { f := resp.metadata f.Path = s.outName f.Span.EndKey = span.EndKey + f.HasRangeKeys = hasRangeKeys s.flushedFiles = append(s.flushedFiles, f) } @@ -446,6 +449,8 @@ func (s *fileSSTSink) copyPointKeys(ctx context.Context, dataSST []byte) (roachp return iter.UnsafeKey().Key.Clone(), nil } +// copyRangeKeys copies all range keys from the dataSST into the buffer and +// returns the max range key observed. func (s *fileSSTSink) copyRangeKeys(dataSST []byte) (roachpb.Key, error) { iterOpts := storage.IterOptions{ KeyTypes: storage.IterKeyTypeRangesOnly, From adbd2faeee93ad1d6cf5af7922a1dae70008ab47 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Tue, 28 May 2024 16:37:47 -0400 Subject: [PATCH 2/2] backupccl: support online restore of small chain incremental backups This patch enables online restore of incremental backup with fewer than 3 layers and that do not contain any range keys. Release note: none --- pkg/ccl/backupccl/restore_online.go | 22 ++- pkg/ccl/backupccl/restore_online_test.go | 178 ++++++++++-------- pkg/ccl/backupccl/restore_span_covering.go | 2 + .../testdata/backup-restore/rangekeys | 9 + pkg/sql/execinfrapb/processors_bulk_io.proto | 4 +- 5 files changed, 124 insertions(+), 91 deletions(-) diff --git a/pkg/ccl/backupccl/restore_online.go b/pkg/ccl/backupccl/restore_online.go index 6954b76acbaa..98d64ba73b63 100644 --- a/pkg/ccl/backupccl/restore_online.go +++ b/pkg/ccl/backupccl/restore_online.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" @@ -166,9 +167,6 @@ func sendAddRemoteSSTs( if encryption != nil { return 0, 0, errors.AssertionFailedf("encryption not supported with online restore") } - if len(uris) > 1 { - return 0, 0, errors.AssertionFailedf("online restore can only restore data from a full backup") - } const targetRangeSize = 440 << 20 @@ -282,7 +280,15 @@ func sendAddRemoteSSTWorker( return err } + var currentLayer int32 for _, file := range entry.Files { + if file.Layer < currentLayer { + return errors.AssertionFailedf("files not sorted by layer") + } + currentLayer = file.Layer + if file.HasRangeKeys { + return errors.Newf("online restore of range keys not supported") + } if err := assertCommonPrefix(file.BackupFileEntrySpan, entry.ElidedPrefix); err != nil { return err } @@ -415,19 +421,19 @@ func checkManifestsForOnlineCompat(ctx context.Context, manifests []backuppb.Bac if len(manifests) < 1 { return errors.AssertionFailedf("expected at least 1 backup manifest") } - // TODO(online-restore): Remove once we support layer ordering. - if len(manifests) > 1 { - return pgerror.Newf(pgcode.FeatureNotSupported, "experimental online restore: restoring from an incremental backup not supported") + + if len(manifests) > 1 && !clusterversion.V24_1.Version().Less(manifests[0].ClusterVersion) { + return errors.Newf("the backup must be on a cluster version greater than %s to run online restore with an incremental backup", clusterversion.V24_1.String()) } // TODO(online-restore): Remove once we support layer ordering and have tested some reasonable number of layers. - const layerLimit = 16 + const layerLimit = 3 if len(manifests) > layerLimit { return pgerror.Newf(pgcode.FeatureNotSupported, "experimental online restore: too many incremental layers %d (from backup) > %d (limit)", len(manifests), layerLimit) } for _, manifest := range manifests { - if !manifest.RevisionStartTime.IsEmpty() || !manifest.StartTime.IsEmpty() || manifest.MVCCFilter == backuppb.MVCCFilter_All { + if !manifest.RevisionStartTime.IsEmpty() || manifest.MVCCFilter == backuppb.MVCCFilter_All { return pgerror.Newf(pgcode.FeatureNotSupported, "experimental online restore: restoring from a revision history backup not supported") } } diff --git a/pkg/ccl/backupccl/restore_online_test.go b/pkg/ccl/backupccl/restore_online_test.go index cb4dcada76c1..1e519f5ac2bb 100644 --- a/pkg/ccl/backupccl/restore_online_test.go +++ b/pkg/ccl/backupccl/restore_online_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/securitytest" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/fingerprintutils" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -50,38 +51,49 @@ func TestOnlineRestoreBasic(t *testing.T) { } tc, sqlDB, dir, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, numAccounts, InitManualReplication, params) defer cleanupFn() + + rtc, rSQLDB, cleanupFnRestored := backupRestoreTestSetupEmpty(t, 1, dir, InitManualReplication, params) + defer cleanupFnRestored() + externalStorage := "nodelocal://1/backup" createStmt := `SELECT create_statement FROM [SHOW CREATE TABLE data.bank]` createStmtRes := sqlDB.QueryStr(t, createStmt) - sqlDB.Exec(t, fmt.Sprintf("BACKUP INTO '%s'", externalStorage)) + testutils.RunTrueAndFalse(t, "incremental", func(t *testing.T, incremental bool) { + sqlDB.Exec(t, fmt.Sprintf("BACKUP DATABASE data INTO '%s'", externalStorage)) - rtc, rSQLDB, cleanupFnRestored := backupRestoreTestSetupEmpty(t, 1, dir, InitManualReplication, params) - defer cleanupFnRestored() - var preRestoreTs float64 - sqlDB.QueryRow(t, `SELECT cluster_logical_timestamp()`).Scan(&preRestoreTs) + if incremental { + sqlDB.Exec(t, "UPDATE data.bank SET balance = balance+123 where true;") + sqlDB.Exec(t, fmt.Sprintf("BACKUP DATABASE data INTO LATEST IN '%s'", externalStorage)) + } - bankOnlineRestore(t, rSQLDB, numAccounts, externalStorage) + var preRestoreTs float64 + sqlDB.QueryRow(t, `SELECT cluster_logical_timestamp()`).Scan(&preRestoreTs) - fpSrc, err := fingerprintutils.FingerprintDatabase(ctx, tc.Conns[0], "data", fingerprintutils.Stripped()) - require.NoError(t, err) - fpDst, err := fingerprintutils.FingerprintDatabase(ctx, rtc.Conns[0], "data", fingerprintutils.Stripped()) - require.NoError(t, err) - require.NoError(t, fingerprintutils.CompareDatabaseFingerprints(fpSrc, fpDst)) + bankOnlineRestore(t, rSQLDB, numAccounts, externalStorage) - assertMVCCOnlineRestore(t, rSQLDB, preRestoreTs) - assertOnlineRestoreWithRekeying(t, sqlDB, rSQLDB) + fpSrc, err := fingerprintutils.FingerprintDatabase(ctx, tc.Conns[0], "data", fingerprintutils.Stripped()) + require.NoError(t, err) + fpDst, err := fingerprintutils.FingerprintDatabase(ctx, rtc.Conns[0], "data", fingerprintutils.Stripped()) + require.NoError(t, err) + require.NoError(t, fingerprintutils.CompareDatabaseFingerprints(fpSrc, fpDst)) - // Wait for the download job to complete. - var downloadJobID jobspb.JobID - rSQLDB.QueryRow(t, `SELECT job_id FROM [SHOW JOBS] WHERE description LIKE '%Background Data Download%'`).Scan(&downloadJobID) - jobutils.WaitForJobToSucceed(t, rSQLDB, downloadJobID) + assertMVCCOnlineRestore(t, rSQLDB, preRestoreTs) + assertOnlineRestoreWithRekeying(t, sqlDB, rSQLDB) - rSQLDB.CheckQueryResults(t, createStmt, createStmtRes) - sqlDB.CheckQueryResults(t, jobutils.GetExternalBytesForConnectedTenant, [][]string{{"0"}}) -} + // Wait for the download job to complete. + var downloadJobID jobspb.JobID + rSQLDB.QueryRow(t, `SELECT job_id FROM [SHOW JOBS] WHERE description LIKE '%Background Data Download%'`).Scan(&downloadJobID) + jobutils.WaitForJobToSucceed(t, rSQLDB, downloadJobID) + + rSQLDB.CheckQueryResults(t, createStmt, createStmtRes) + sqlDB.CheckQueryResults(t, jobutils.GetExternalBytesForConnectedTenant, [][]string{{"0"}}) + rSQLDB.Exec(t, "DROP DATABASE data CASCADE") + }) + +} func TestOnlineRestorePartitioned(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -209,8 +221,6 @@ func TestOnlineRestoreWaitForDownload(t *testing.T) { // TestOnlineRestoreTenant runs an online restore of a tenant and ensures the // restore is not MVCC compliant. -// -// NB: With prefix synthesis, we temporarliy do not support online restore of tenants. func TestOnlineRestoreTenant(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -246,56 +256,67 @@ func TestOnlineRestoreTenant(t *testing.T) { tenant10 := sqlutils.MakeSQLRunner(conn10) tenant10.Exec(t, `CREATE DATABASE foo; CREATE TABLE foo.bar(i int primary key); INSERT INTO foo.bar VALUES (110), (210)`) - systemDB.Exec(t, fmt.Sprintf(`BACKUP TENANT 10 INTO '%s'`, externalStorage)) + testutils.RunTrueAndFalse(t, "incremental", func(t *testing.T, incremental bool) { - restoreTC, rSQLDB, cleanupFnRestored := backupRestoreTestSetupEmpty(t, 1, dir, InitManualReplication, params) - defer cleanupFnRestored() + restoreTC, rSQLDB, cleanupFnRestored := backupRestoreTestSetupEmpty(t, 1, dir, InitManualReplication, params) + defer cleanupFnRestored() - var preRestoreTs float64 - tenant10.QueryRow(t, `SELECT cluster_logical_timestamp()`).Scan(&preRestoreTs) - - // Restore the tenant twice: once below and once above the old ID, to show - // that we can rewrite it in either direction. - rSQLDB.Exec(t, fmt.Sprintf("RESTORE TENANT 10 FROM LATEST IN '%s' WITH EXPERIMENTAL DEFERRED COPY, TENANT_NAME = 'below', TENANT = '2'", externalStorage)) - rSQLDB.Exec(t, fmt.Sprintf("RESTORE TENANT 10 FROM LATEST IN '%s' WITH EXPERIMENTAL DEFERRED COPY, TENANT_NAME = 'above', TENANT = '20'", externalStorage)) - rSQLDB.Exec(t, "ALTER TENANT below STOP SERVICE") - rSQLDB.Exec(t, "ALTER TENANT above STOP SERVICE") - rSQLDB.CheckQueryResults(t, "SELECT fingerprint FROM [SHOW EXPERIMENTAL_FINGERPRINTS FROM TENANT below]", - rSQLDB.QueryStr(t, `SELECT fingerprint FROM [SHOW EXPERIMENTAL_FINGERPRINTS FROM TENANT above]`)) - - secondaryStopper := stop.NewStopper() - _, cBelow := serverutils.StartTenant( - t, restoreTC.Server(0), base.TestTenantArgs{ - TenantName: "below", - TenantID: roachpb.MustMakeTenantID(2), - Stopper: secondaryStopper, - }) - _, cAbove := serverutils.StartTenant( - t, restoreTC.Server(0), base.TestTenantArgs{ - TenantName: "above", - TenantID: roachpb.MustMakeTenantID(20), - Stopper: secondaryStopper, - }) - - defer func() { - cBelow.Close() - cAbove.Close() - secondaryStopper.Stop(context.Background()) - }() - dbBelow, dbAbove := sqlutils.MakeSQLRunner(cBelow), sqlutils.MakeSQLRunner(cAbove) - dbBelow.CheckQueryResults(t, `select * from foo.bar`, tenant10.QueryStr(t, `select * from foo.bar`)) - dbAbove.CheckQueryResults(t, `select * from foo.bar`, tenant10.QueryStr(t, `select * from foo.bar`)) - - // Ensure the restore of a tenant was not mvcc - var maxRestoreMVCCTimestamp float64 - dbBelow.QueryRow(t, "SELECT max(crdb_internal_mvcc_timestamp) FROM foo.bar").Scan(&maxRestoreMVCCTimestamp) - require.Greater(t, preRestoreTs, maxRestoreMVCCTimestamp) - dbAbove.QueryRow(t, "SELECT max(crdb_internal_mvcc_timestamp) FROM foo.bar").Scan(&maxRestoreMVCCTimestamp) - require.Greater(t, preRestoreTs, maxRestoreMVCCTimestamp) - - dbAbove.CheckQueryResults(t, jobutils.GetExternalBytesForConnectedTenant, [][]string{{"0"}}) - dbBelow.CheckQueryResults(t, jobutils.GetExternalBytesForConnectedTenant, [][]string{{"0"}}) - rSQLDB.CheckQueryResults(t, jobutils.GetExternalBytesTenantKeySpace, [][]string{{"0"}}) + systemDB.Exec(t, fmt.Sprintf(`BACKUP TENANT 10 INTO '%s'`, externalStorage)) + + if incremental { + tenant10.Exec(t, "INSERT INTO foo.bar VALUES (111), (211)") + systemDB.Exec(t, fmt.Sprintf(`BACKUP TENANT 10 INTO LATEST IN '%s'`, externalStorage)) + } + + var preRestoreTs float64 + tenant10.QueryRow(t, `SELECT cluster_logical_timestamp()`).Scan(&preRestoreTs) + + belowID := uint64(2) + aboveID := uint64(20) + + // Restore the tenant twice: once below and once above the old ID, to show + // that we can rewrite it in either direction. + rSQLDB.Exec(t, fmt.Sprintf("RESTORE TENANT 10 FROM LATEST IN '%s' WITH EXPERIMENTAL DEFERRED COPY, TENANT_NAME = 'below', TENANT = '%d'", externalStorage, belowID)) + rSQLDB.Exec(t, fmt.Sprintf("RESTORE TENANT 10 FROM LATEST IN '%s' WITH EXPERIMENTAL DEFERRED COPY, TENANT_NAME = 'above', TENANT = '%d'", externalStorage, aboveID)) + rSQLDB.Exec(t, "ALTER TENANT below STOP SERVICE") + rSQLDB.Exec(t, "ALTER TENANT above STOP SERVICE") + rSQLDB.CheckQueryResults(t, "SELECT fingerprint FROM [SHOW EXPERIMENTAL_FINGERPRINTS FROM TENANT below]", + rSQLDB.QueryStr(t, `SELECT fingerprint FROM [SHOW EXPERIMENTAL_FINGERPRINTS FROM TENANT above]`)) + + secondaryStopper := stop.NewStopper() + _, cBelow := serverutils.StartTenant( + t, restoreTC.Server(0), base.TestTenantArgs{ + TenantName: "below", + TenantID: roachpb.MustMakeTenantID(belowID), + Stopper: secondaryStopper, + }) + _, cAbove := serverutils.StartTenant( + t, restoreTC.Server(0), base.TestTenantArgs{ + TenantName: "above", + TenantID: roachpb.MustMakeTenantID(aboveID), + Stopper: secondaryStopper, + }) + + defer func() { + cBelow.Close() + cAbove.Close() + secondaryStopper.Stop(context.Background()) + }() + dbBelow, dbAbove := sqlutils.MakeSQLRunner(cBelow), sqlutils.MakeSQLRunner(cAbove) + dbBelow.CheckQueryResults(t, `select * from foo.bar`, tenant10.QueryStr(t, `select * from foo.bar`)) + dbAbove.CheckQueryResults(t, `select * from foo.bar`, tenant10.QueryStr(t, `select * from foo.bar`)) + + // Ensure the restore of a tenant was not mvcc + var maxRestoreMVCCTimestamp float64 + dbBelow.QueryRow(t, "SELECT max(crdb_internal_mvcc_timestamp) FROM foo.bar").Scan(&maxRestoreMVCCTimestamp) + require.Greater(t, preRestoreTs, maxRestoreMVCCTimestamp) + dbAbove.QueryRow(t, "SELECT max(crdb_internal_mvcc_timestamp) FROM foo.bar").Scan(&maxRestoreMVCCTimestamp) + require.Greater(t, preRestoreTs, maxRestoreMVCCTimestamp) + + dbAbove.CheckQueryResults(t, jobutils.GetExternalBytesForConnectedTenant, [][]string{{"0"}}) + dbBelow.CheckQueryResults(t, jobutils.GetExternalBytesForConnectedTenant, [][]string{{"0"}}) + rSQLDB.CheckQueryResults(t, jobutils.GetExternalBytesTenantKeySpace, [][]string{{"0"}}) + }) } func TestOnlineRestoreErrors(t *testing.T) { @@ -318,16 +339,8 @@ func TestOnlineRestoreErrors(t *testing.T) { var ( fullBackup = "nodelocal://1/full-backup" fullBackupWithRevs = "nodelocal://1/full-backup-with-revs" - incrementalBackup = "nodelocal://1/incremental-backup" incrementalBackupWithRevs = "nodelocal://1/incremental-backup-with-revs" ) - - t.Run("incremental backups are unsupported", func(t *testing.T) { - sqlDB.Exec(t, fmt.Sprintf("BACKUP INTO '%s'", incrementalBackup)) - sqlDB.Exec(t, fmt.Sprintf("BACKUP INTO LATEST IN '%s'", incrementalBackup)) - rSQLDB.ExpectErr(t, "incremental backup not supported", - fmt.Sprintf("RESTORE TABLE data.bank FROM LATEST IN '%s' WITH EXPERIMENTAL DEFERRED COPY", incrementalBackup)) - }) t.Run("full backups with revision history are unsupported", func(t *testing.T) { var systemTime string sqlDB.QueryRow(t, "SELECT cluster_logical_timestamp()").Scan(&systemTime) @@ -338,7 +351,7 @@ func TestOnlineRestoreErrors(t *testing.T) { t.Run("incremental backups with revision history are unsupported", func(t *testing.T) { sqlDB.Exec(t, fmt.Sprintf("BACKUP INTO '%s' WITH revision_history", incrementalBackupWithRevs)) sqlDB.Exec(t, fmt.Sprintf("BACKUP INTO LATEST IN '%s' WITH revision_history", incrementalBackupWithRevs)) - rSQLDB.ExpectErr(t, "incremental backup not supported", + rSQLDB.ExpectErr(t, "revision history backup not supported", fmt.Sprintf("RESTORE TABLE data.bank FROM LATEST IN '%s' WITH EXPERIMENTAL DEFERRED COPY", incrementalBackupWithRevs)) }) t.Run("external storage locations that don't support early boot are unsupported", func(t *testing.T) { @@ -358,7 +371,7 @@ func bankOnlineRestore( t *testing.T, sqlDB *sqlutils.SQLRunner, numAccounts int, externalStorage string, ) { // Create a table in the default database to force table id rewriting. - sqlDB.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY, s STRING);") + sqlDB.Exec(t, "CREATE TABLE IF NOT EXISTS foo (i INT PRIMARY KEY, s STRING);") sqlDB.Exec(t, "CREATE DATABASE data") sqlDB.Exec(t, fmt.Sprintf("RESTORE TABLE data.bank FROM LATEST IN '%s' WITH EXPERIMENTAL DEFERRED COPY", externalStorage)) @@ -382,8 +395,9 @@ func assertMVCCOnlineRestore(t *testing.T, sqlDB *sqlutils.SQLRunner, preRestore // Check that we can write on top of OR data var maxRestoreMVCCTimestamp float64 sqlDB.QueryRow(t, "SELECT max(crdb_internal_mvcc_timestamp) FROM data.bank").Scan(&maxRestoreMVCCTimestamp) - sqlDB.Exec(t, "SET sql_safe_updates = false;") - sqlDB.Exec(t, "UPDATE data.bank SET balance = balance+1;") + + // The where true conditional avoids the need to set sql_updates to true. + sqlDB.Exec(t, "UPDATE data.bank SET balance = balance+1 where true;") var updateMVCCTimestamp float64 sqlDB.QueryRow(t, "SELECT min(crdb_internal_mvcc_timestamp) FROM data.bank").Scan(&updateMVCCTimestamp) diff --git a/pkg/ccl/backupccl/restore_span_covering.go b/pkg/ccl/backupccl/restore_span_covering.go index e6ed43aa4899..8216cb6d4804 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -346,6 +346,8 @@ func generateAndSendImportSpans( BackupFileEntryCounts: f.EntryCounts, BackingFileSize: f.BackingFileSize, ApproximatePhysicalSize: f.ApproximatePhysicalSize, + Layer: int32(layer), + HasRangeKeys: f.HasRangeKeys, } if dir, ok := backupLocalityMap[layer][f.LocalityKV]; ok { fileSpec.Dir = dir diff --git a/pkg/ccl/backupccl/testdata/backup-restore/rangekeys b/pkg/ccl/backupccl/testdata/backup-restore/rangekeys index 9cf3a2ac3a2e..55b6e6beffed 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/rangekeys +++ b/pkg/ccl/backupccl/testdata/backup-restore/rangekeys @@ -3,6 +3,9 @@ # disabled to run within tenants because the kv request cmd only works on system tenants +reset test-nodelocal +---- + new-cluster name=s1 disable-tenant ---- @@ -34,6 +37,7 @@ exec-sql RESTORE DATABASE orig FROM LATEST IN 'nodelocal://1/test-root/' with new_db_name='orig1'; ---- + query-sql SELECT count(*) from orig1.foo; ---- @@ -72,6 +76,11 @@ exec-sql RESTORE DATABASE orig FROM LATEST IN 'nodelocal://1/test-root/' with new_db_name='orig1'; ---- +exec-sql expect-error-regex=(failed to ingest into remote files: online restore of range keys not supported) +RESTORE DATABASE orig FROM LATEST IN 'nodelocal://1/test-root/' with new_db_name='orig2', experimental deferred copy; +---- +regex matches error + query-sql SELECT count(*) from orig1.foo ---- diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 60aa36a2baee..2529f7605e34 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -334,7 +334,9 @@ message RestoreFileSpec { optional roachpb.RowCount backup_file_entry_counts = 6 [(gogoproto.nullable) = false]; optional uint64 backing_file_size = 7 [(gogoproto.nullable) = false]; optional uint64 approximate_physical_size = 8 [(gogoproto.nullable) = false]; - // NEXT ID: 9. + optional int32 layer = 9 [(gogoproto.nullable) = false]; + optional bool has_range_keys = 10 [(gogoproto.nullable) = false]; + // NEXT ID: 11. } message TableRekey {