Skip to content

Commit e6d3d9d

Browse files
craig[bot]yuzefovichmsbutlerajstorm
committed
155555: roachtest: increase server.sqlliveness.ttl in copyfrom r=yuzefovich a=yuzefovich `copyfrom/atomic` often fails with `RETRY_COMMIT_DEADLINE_EXCEEDED` error. Based on some guidance from KV folks, it sounds like (apart from the closed TS system that has already been adjusted) the most likely explanation is that the lease on a descriptor has expired while the txn was running. Before 24.1 we used expiry based leasing, then we introduced session based leasing, and the migration to only use that has been completed in the beginning of the year. If I'm reading the code right, then the lease duration depends on the SQL liveness TTL, so this commit bumps the relevant cluster setting from 40s to 5m to give enough time for atomic COPY to finish. I did 50 runs of the test, and all of them passed. Fixes: #155300. Release note: None 155563: importer: unskip all tests with test tenants r=yuzefovich a=yuzefovich This mostly involved ensuring that the right ApplicationLayerInterface is being used, or the test just worked. Also a couple of tests explicitly control the tenants, so mark those accordingly. My hypothesis is that some tests have been fixed long time ago in 4fd9f70 (`tc.ServerConn` now returns the right connection), but I didn't verify it. Only `TestImportIntoCSV` needs a special callout. In that test we have some expected errors, and some of them when encountered by the tenants can be retried. In order to speed up the test we reduce the retry duration from 2 minutes to 2 seconds. Additionally remove some SQLMemoryPoolSize overrides that now equal the default of 256MiB (which increased from 128MiB a couple years ago). Fixes: #107141. Fixes: #142803. Release note: None 155624: backup: move backup index logic to backupinfo pkg r=kev-cao a=msbutler The backup index is part of the backup metadata, whose logic lives in the backupinfo package. This refactor also enables subsequent refactor to unify backup metadata writing for regular backups and compacted backups. Epic: none Release note: none 155626: dev-inf: Fix jq command to handle JSON array format r=rickystewart a=ajstorm The execution file is a JSON array, not JSONL format. Updated all extraction steps to use '.[]' to iterate over the array before filtering by type. Changed from: jq -r 'select(.type == "result") | .result' Changed to: jq -r '.[] | select(.type == "result") | .result' This fixes the error: "Cannot index array with string 'type'" Tested locally with both array and JSONL formats to verify the fix works. Epic: none Release note: none Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com> Co-authored-by: Michael Butler <butler@cockroachlabs.com> Co-authored-by: Adam Storm <storm@cockroachlabs.com>
5 parents 6cabe06 + 407771f + 9d0779b + ebfe440 + 5e1f078 commit e6d3d9d

File tree

18 files changed

+106
-146
lines changed

18 files changed

+106
-146
lines changed

.github/workflows/pr-analyzer-threestage.yml

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -66,7 +66,7 @@ jobs:
6666
id: stage1_result
6767
if: steps.stage1.conclusion == 'success'
6868
run: |
69-
RESULT=$(jq -r 'select(.type == "result") | .result' "${{ steps.stage1.outputs.execution_file }}")
69+
RESULT=$(jq -r '.[] | select(.type == "result") | .result' "${{ steps.stage1.outputs.execution_file }}")
7070
{
7171
echo 'result<<EOF'
7272
echo "$RESULT"
@@ -116,7 +116,7 @@ jobs:
116116
id: stage2_result
117117
if: steps.stage2.conclusion == 'success'
118118
run: |
119-
RESULT=$(jq -r 'select(.type == "result") | .result' "${{ steps.stage2.outputs.execution_file }}")
119+
RESULT=$(jq -r '.[] | select(.type == "result") | .result' "${{ steps.stage2.outputs.execution_file }}")
120120
{
121121
echo 'result<<EOF'
122122
echo "$RESULT"
@@ -181,7 +181,7 @@ jobs:
181181
id: stage3_result
182182
if: steps.stage3.conclusion == 'success'
183183
run: |
184-
RESULT=$(jq -r 'select(.type == "result") | .result' "${{ steps.stage3.outputs.execution_file }}")
184+
RESULT=$(jq -r '.[] | select(.type == "result") | .result' "${{ steps.stage3.outputs.execution_file }}")
185185
{
186186
echo 'result<<EOF'
187187
echo "$RESULT"

pkg/backup/backup_job.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -444,7 +444,7 @@ func backup(
444444
return roachpb.RowCount{}, 0, err
445445
}
446446

447-
if err := backupdest.WriteBackupIndexMetadata(
447+
if err := backupinfo.WriteBackupIndexMetadata(
448448
ctx,
449449
execCtx.ExecCfg(),
450450
execCtx.User(),

pkg/backup/backupdest/BUILD.bazel

Lines changed: 1 addition & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,6 @@ go_library(
44
name = "backupdest",
55
srcs = [
66
"backup_destination.go",
7-
"backup_index.go",
87
"incrementals.go",
98
],
109
importpath = "github.com/cockroachdb/cockroach/pkg/backup/backupdest",
@@ -17,24 +16,19 @@ go_library(
1716
"//pkg/backup/backuputils",
1817
"//pkg/cloud",
1918
"//pkg/cloud/cloudpb",
20-
"//pkg/clusterversion",
2119
"//pkg/jobs/jobspb",
2220
"//pkg/roachpb",
2321
"//pkg/security/username",
24-
"//pkg/settings",
2522
"//pkg/settings/cluster",
2623
"//pkg/sql",
2724
"//pkg/sql/pgwire/pgcode",
2825
"//pkg/sql/pgwire/pgerror",
2926
"//pkg/util",
30-
"//pkg/util/ctxgroup",
3127
"//pkg/util/encoding",
3228
"//pkg/util/hlc",
3329
"//pkg/util/ioctx",
3430
"//pkg/util/log",
35-
"//pkg/util/metamorphic",
3631
"//pkg/util/mon",
37-
"//pkg/util/protoutil",
3832
"//pkg/util/timeutil",
3933
"//pkg/util/tracing",
4034
"@com_github_cockroachdb_errors//:errors",
@@ -46,7 +40,6 @@ go_test(
4640
name = "backupdest_test",
4741
srcs = [
4842
"backup_destination_test.go",
49-
"backup_index_test.go",
5043
"incrementals_test.go",
5144
"main_test.go",
5245
],
@@ -57,35 +50,29 @@ go_test(
5750
}),
5851
deps = [
5952
"//pkg/backup/backupbase",
53+
"//pkg/backup/backupinfo",
6054
"//pkg/backup/backuppb",
6155
"//pkg/backup/backuptestutils",
6256
"//pkg/backup/backuputils",
6357
"//pkg/base",
64-
"//pkg/blobs",
6558
"//pkg/ccl",
6659
"//pkg/cloud",
67-
"//pkg/cloud/cloudpb",
6860
"//pkg/cloud/impl:cloudimpl",
69-
"//pkg/clusterversion",
7061
"//pkg/jobs/jobspb",
7162
"//pkg/security/securityassets",
7263
"//pkg/security/securitytest",
7364
"//pkg/security/username",
7465
"//pkg/server",
75-
"//pkg/settings/cluster",
7666
"//pkg/sql",
77-
"//pkg/testutils",
7867
"//pkg/testutils/jobutils",
7968
"//pkg/testutils/serverutils",
8069
"//pkg/testutils/testcluster",
8170
"//pkg/util",
8271
"//pkg/util/hlc",
83-
"//pkg/util/ioctx",
8472
"//pkg/util/leaktest",
8573
"//pkg/util/log",
8674
"//pkg/util/protoutil",
8775
"//pkg/util/randutil",
88-
"@com_github_cockroachdb_errors//:errors",
8976
"@com_github_stretchr_testify//require",
9077
],
9178
)

pkg/backup/backupdest/backup_destination.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -237,7 +237,7 @@ func ResolveDest(
237237
}
238238
}
239239

240-
partName := ConstructDateBasedIncrementalFolderName(startTime.GoTime(), endTime.GoTime())
240+
partName := backupinfo.ConstructDateBasedIncrementalFolderName(startTime.GoTime(), endTime.GoTime())
241241
defaultIncrementalsURI, urisByLocalityKV, err := GetURIsByLocalityKV(fullyResolvedIncrementalsLocation, partName)
242242
if err != nil {
243243
return ResolvedDestination{}, err
@@ -478,7 +478,7 @@ func ListFullBackupsInCollection(
478478
ctx context.Context, store cloud.ExternalStorage, useIndex bool,
479479
) ([]string, error) {
480480
if useIndex {
481-
return ListSubdirsFromIndex(ctx, store)
481+
return backupinfo.ListSubdirsFromIndex(ctx, store)
482482
}
483483

484484
var backupPaths []string
@@ -539,15 +539,15 @@ func ResolveBackupManifests(
539539
}
540540
defer rootStore.Close()
541541

542-
if !ReadBackupIndexEnabled.Get(&execCfg.Settings.SV) || isCustomIncLocation {
542+
if !backupinfo.ReadBackupIndexEnabled.Get(&execCfg.Settings.SV) || isCustomIncLocation {
543543
return legacyResolveBackupManifests(
544544
ctx, execCfg, mem, defaultCollectionURI, mkStore,
545545
resolvedSubdir, fullyResolvedBaseDirectory, fullyResolvedIncrementalsDirectory,
546546
endTime, encryption, kmsEnv, user, includeSkipped, includeCompacted,
547547
)
548548
}
549549

550-
exists, err := IndexExists(ctx, rootStore, resolvedSubdir)
550+
exists, err := backupinfo.IndexExists(ctx, rootStore, resolvedSubdir)
551551
if err != nil {
552552
return nil, nil, nil, 0, err
553553
}
@@ -759,7 +759,7 @@ func indexedResolveBackupManifests(
759759
}
760760
}()
761761

762-
indexes, err := GetBackupTreeIndexMetadata(
762+
indexes, err := backupinfo.GetBackupTreeIndexMetadata(
763763
ctx, rootStores[0], resolvedSubdir,
764764
)
765765
if err != nil {

pkg/backup/backupdest/incrementals.go

Lines changed: 4 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -7,14 +7,13 @@ package backupdest
77

88
import (
99
"context"
10-
"fmt"
1110
"path"
1211
"regexp"
1312
"sort"
1413
"strings"
15-
"time"
1614

1715
"github.com/cockroachdb/cockroach/pkg/backup/backupbase"
16+
"github.com/cockroachdb/cockroach/pkg/backup/backupinfo"
1817
"github.com/cockroachdb/cockroach/pkg/backup/backuputils"
1918
"github.com/cockroachdb/cockroach/pkg/cloud"
2019
"github.com/cockroachdb/cockroach/pkg/security/username"
@@ -92,11 +91,11 @@ func FindAllIncrementalPaths(
9291
defer sp.Finish()
9392

9493
// Backup indexes do not support custom incremental locations.
95-
if customIncLocation || !ReadBackupIndexEnabled.Get(&execCfg.Settings.SV) {
94+
if customIncLocation || !backupinfo.ReadBackupIndexEnabled.Get(&execCfg.Settings.SV) {
9695
return LegacyFindPriorBackups(ctx, incStore, OmitManifest)
9796
}
9897

99-
indexes, err := ListIndexes(ctx, rootStore, subdir)
98+
indexes, err := backupinfo.ListIndexes(ctx, rootStore, subdir)
10099
if err != nil {
101100
return nil, err
102101
}
@@ -110,7 +109,7 @@ func FindAllIncrementalPaths(
110109
return util.MapE(
111110
indexes[1:], // We skip the full backup
112111
func(indexFilename string) (string, error) {
113-
return parseBackupFilePathFromIndexFileName(subdir, indexFilename)
112+
return backupinfo.ParseBackupFilePathFromIndexFileName(subdir, indexFilename)
114113
},
115114
)
116115
}
@@ -317,18 +316,3 @@ func ResolveDefaultBaseIncrementalStorageLocation(
317316

318317
return defaultURI, nil
319318
}
320-
321-
// ConstructDateBasedIncrementalFolderName constructs the name of a date-based
322-
// incremental backup folder relative to the full subdirectory it belongs to.
323-
//
324-
// /2025/07/30-120000.00/20250730/130000.00-20250730-120000.00
325-
//
326-
// └─────────────────────────────────────┘
327-
// returns this
328-
func ConstructDateBasedIncrementalFolderName(start, end time.Time) string {
329-
return fmt.Sprintf(
330-
"%s-%s",
331-
end.Format(backupbase.DateBasedIncFolderName),
332-
start.Format(backupbase.DateBasedIncFolderNameSuffix),
333-
)
334-
}

pkg/backup/backupdest/incrementals_test.go

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ import (
1515

1616
"github.com/cockroachdb/cockroach/pkg/backup/backupbase"
1717
"github.com/cockroachdb/cockroach/pkg/backup/backupdest"
18+
"github.com/cockroachdb/cockroach/pkg/backup/backupinfo"
1819
"github.com/cockroachdb/cockroach/pkg/backup/backuptestutils"
1920
"github.com/cockroachdb/cockroach/pkg/backup/backuputils"
2021
"github.com/cockroachdb/cockroach/pkg/cloud"
@@ -250,7 +251,7 @@ func TestFindAllIncrementalPaths(t *testing.T) {
250251

251252
var expectedPaths []string
252253
for _, b := range targetChain[1:] {
253-
expectedPaths = append(expectedPaths, backupdest.ConstructDateBasedIncrementalFolderName(
254+
expectedPaths = append(expectedPaths, backupinfo.ConstructDateBasedIncrementalFolderName(
254255
toTime(b.start), toTime(b.end),
255256
))
256257
}
@@ -439,7 +440,7 @@ func writeEmptyBackupManifest(
439440
}
440441
backupPath = backuputils.JoinURLPath(
441442
subdir,
442-
backupdest.ConstructDateBasedIncrementalFolderName(start, end),
443+
backupinfo.ConstructDateBasedIncrementalFolderName(start, end),
443444
)
444445
}
445446

@@ -486,7 +487,7 @@ func writeEmptyBackupManifest(
486487

487488
require.NoError(
488489
t,
489-
backupdest.WriteBackupIndexMetadata(
490+
backupinfo.WriteBackupIndexMetadata(
490491
context.Background(), execCfg, username.RootUserName(),
491492
execCfg.DistSQLSrv.ExternalStorageFromURI, backupDetails, hlc.Timestamp{},
492493
),

pkg/backup/backupinfo/BUILD.bazel

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,7 @@ load("//pkg/testutils:buildutil/buildutil.bzl", "disallowed_imports_test")
44
go_library(
55
name = "backupinfo",
66
srcs = [
7+
"backup_index.go",
78
"desc_sst.go",
89
"external_sst_util.go",
910
"file_sst.go",
@@ -19,11 +20,13 @@ go_library(
1920
"//pkg/ccl/storageccl",
2021
"//pkg/cloud",
2122
"//pkg/cloud/cloudpb",
23+
"//pkg/clusterversion",
2224
"//pkg/jobs/jobspb",
2325
"//pkg/keys",
2426
"//pkg/kv/kvpb",
2527
"//pkg/roachpb",
2628
"//pkg/security/username",
29+
"//pkg/settings",
2730
"//pkg/sql",
2831
"//pkg/sql/catalog",
2932
"//pkg/sql/catalog/dbdesc",
@@ -43,6 +46,7 @@ go_library(
4346
"//pkg/util/hlc",
4447
"//pkg/util/ioctx",
4548
"//pkg/util/log",
49+
"//pkg/util/metamorphic",
4650
"//pkg/util/mon",
4751
"//pkg/util/protoutil",
4852
"//pkg/util/syncutil",
@@ -57,17 +61,22 @@ go_library(
5761
go_test(
5862
name = "backupinfo_test",
5963
srcs = [
64+
"backup_index_test.go",
6065
"desc_sst_test.go",
6166
"main_test.go",
6267
"manifest_handling_test.go",
6368
],
6469
embed = [":backupinfo"],
6570
deps = [
71+
"//pkg/backup/backupbase",
6672
"//pkg/backup/backuppb",
73+
"//pkg/backup/backuptestutils",
6774
"//pkg/base",
6875
"//pkg/blobs",
6976
"//pkg/ccl",
7077
"//pkg/cloud",
78+
"//pkg/cloud/cloudpb",
79+
"//pkg/clusterversion",
7180
"//pkg/jobs/jobspb",
7281
"//pkg/keys",
7382
"//pkg/multitenant/mtinfopb",
@@ -76,15 +85,20 @@ go_test(
7685
"//pkg/security/securitytest",
7786
"//pkg/security/username",
7887
"//pkg/server",
88+
"//pkg/settings/cluster",
89+
"//pkg/sql",
7990
"//pkg/sql/catalog/descpb",
8091
"//pkg/sql/isql",
92+
"//pkg/testutils",
8193
"//pkg/testutils/serverutils",
8294
"//pkg/testutils/testcluster",
8395
"//pkg/util",
8496
"//pkg/util/bulk",
8597
"//pkg/util/hlc",
98+
"//pkg/util/ioctx",
8699
"//pkg/util/leaktest",
87100
"//pkg/util/log",
101+
"//pkg/util/protoutil",
88102
"//pkg/util/randutil",
89103
"@com_github_cockroachdb_errors//:errors",
90104
"@com_github_stretchr_testify//require",

pkg/backup/backupdest/backup_index.go renamed to pkg/backup/backupinfo/backup_index.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,7 @@
33
// Use of this software is governed by the CockroachDB Software License
44
// included in the /LICENSE file.
55

6-
package backupdest
6+
package backupinfo
77

88
import (
99
"bytes"
@@ -279,7 +279,7 @@ func GetBackupTreeIndexMetadata(
279279
// Note: While the path is stored in the index file, we can take a shortcut here
280280
// and derive it from the filename solely because backup paths are
281281
// millisecond-precise and so are the timestamps encoded in the filename.
282-
func parseBackupFilePathFromIndexFileName(subdir, basename string) (string, error) {
282+
func ParseBackupFilePathFromIndexFileName(subdir, basename string) (string, error) {
283283
start, end, err := parseIndexFilename(basename)
284284
if err != nil {
285285
return "", err

pkg/backup/backupdest/backup_index_test.go renamed to pkg/backup/backupinfo/backup_index_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,7 @@
33
// Use of this software is governed by the CockroachDB Software License
44
// included in the /LICENSE file.
55

6-
package backupdest
6+
package backupinfo
77

88
import (
99
"bufio"

pkg/backup/backupinfo/manifest_handling.go

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@ import (
1616
"sort"
1717
"strconv"
1818
"strings"
19+
"time"
1920

2021
"github.com/cockroachdb/cockroach/pkg/backup/backupbase"
2122
"github.com/cockroachdb/cockroach/pkg/backup/backupencryption"
@@ -1760,3 +1761,18 @@ func GetBackupManifestIterFactories(
17601761

17611762
return layerToFileIterFactory, nil
17621763
}
1764+
1765+
// ConstructDateBasedIncrementalFolderName constructs the name of a date-based
1766+
// incremental backup folder relative to the full subdirectory it belongs to.
1767+
//
1768+
// /2025/07/30-120000.00/20250730/130000.00-20250730-120000.00
1769+
//
1770+
// └─────────────────────────────────────┘
1771+
// returns this
1772+
func ConstructDateBasedIncrementalFolderName(start, end time.Time) string {
1773+
return fmt.Sprintf(
1774+
"%s-%s",
1775+
end.Format(backupbase.DateBasedIncFolderName),
1776+
start.Format(backupbase.DateBasedIncFolderNameSuffix),
1777+
)
1778+
}

0 commit comments

Comments
 (0)