Skip to content

Commit

Permalink
storage: move ExternalIODir into StorageConfig
Browse files Browse the repository at this point in the history
Previously the ExternalIODir was stored as a variable inside the
BaseConfig struct and updated based on the StoreSpecs if it wasn't
explicitly set. This commit moves it into the StorageConfig and removes
it from the BaseConfig.

Epic: CRDB-41111

Release note: None
  • Loading branch information
andrewbaptist committed Jan 31, 2025
1 parent 281a25b commit 523550f
Show file tree
Hide file tree
Showing 69 changed files with 282 additions and 198 deletions.
1 change: 1 addition & 0 deletions pkg/backup/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -289,6 +289,7 @@ go_test(
"//pkg/sql/sqlliveness/slbase",
"//pkg/sql/stats",
"//pkg/storage",
"//pkg/storage/storagepb",
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/fingerprintutils",
Expand Down
3 changes: 2 additions & 1 deletion pkg/backup/alter_backup_schedule_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobstest"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
Expand Down Expand Up @@ -71,7 +72,7 @@ func newAlterSchedulesTestHelper(
}

args := base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
Knobs: base.TestingKnobs{
JobsTestingKnobs: knobs,
},
Expand Down
75 changes: 43 additions & 32 deletions pkg/backup/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/fingerprintutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
Expand Down Expand Up @@ -457,28 +458,28 @@ func TestBackupRestoreExecLocality(t *testing.T) {
},
ServerArgsPerNode: map[int]base.TestServerArgs{
0: {
ExternalIODir: "/west0",
StorageConfig: storagepb.NodeConfig{ExternalIODir: "/west0"},
Locality: roachpb.Locality{Tiers: []roachpb.Tier{
{Key: "tier", Value: "0"},
{Key: "region", Value: "west"},
}},
},
1: {
ExternalIODir: "/west1",
StorageConfig: storagepb.NodeConfig{ExternalIODir: "/west1"},
Locality: roachpb.Locality{Tiers: []roachpb.Tier{
{Key: "tier", Value: "1"},
{Key: "region", Value: "west"},
}},
},
2: {
ExternalIODir: "/east0",
StorageConfig: storagepb.NodeConfig{ExternalIODir: "/east0"},
Locality: roachpb.Locality{Tiers: []roachpb.Tier{
{Key: "tier", Value: "0"},
{Key: "region", Value: "east"},
}},
},
3: {
ExternalIODir: "/east1",
StorageConfig: storagepb.NodeConfig{ExternalIODir: "/east1"},
Locality: roachpb.Locality{Tiers: []roachpb.Tier{
{Key: "tier", Value: "1"},
{Key: "region", Value: "east"},
Expand Down Expand Up @@ -2781,7 +2782,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) {
_, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()
args := base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
}

// Generate some testdata and back it up.
Expand Down Expand Up @@ -3141,7 +3142,9 @@ func TestBackupRestoreIncremental(t *testing.T) {

tc, sqlDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, InitManualReplication)
defer cleanupFn()
args := base.TestServerArgs{ExternalIODir: dir}
args := base.TestServerArgs{
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
}
rng, _ := randutil.NewTestRand()

backupDir := "nodelocal://1/backup"
Expand Down Expand Up @@ -4028,11 +4031,14 @@ func TestNonLinearChain(t *testing.T) {
dir, cleanup := testutils.TempDir(t)
defer cleanup()

tc := testcluster.NewTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{
DefaultTestTenant: base.TODOTestTenantDisabled, ExternalIODir: dir, Knobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
},
}})
tc := testcluster.NewTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
DefaultTestTenant: base.TODOTestTenantDisabled,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
Knobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
},
}})

tc.Start(t)
defer tc.Stopper().Stop(context.Background())
Expand Down Expand Up @@ -4647,7 +4653,7 @@ func TestRestoredPrivileges(t *testing.T) {
const numAccounts = 1
_, sqlDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()
args := base.TestServerArgs{ExternalIODir: dir}
args := base.TestServerArgs{StorageConfig: storagepb.NodeConfig{ExternalIODir: dir}}

rootOnly := sqlDB.QueryStr(t, `SHOW GRANTS ON data.bank`)

Expand Down Expand Up @@ -4725,7 +4731,9 @@ func TestRestoreDatabaseVersusTable(t *testing.T) {
tc, origDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()
s := tc.ApplicationLayer(0)
args := base.TestServerArgs{ExternalIODir: s.ExternalIODir()}
args := base.TestServerArgs{
StorageConfig: storagepb.NodeConfig{ExternalIODir: s.ExternalIODir()},
}

for _, q := range []string{
`CREATE DATABASE d2`,
Expand Down Expand Up @@ -5201,7 +5209,7 @@ func TestBackupRestoreSequence(t *testing.T) {
_, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()
args := base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
}

backupLoc := localFoo
Expand Down Expand Up @@ -5416,7 +5424,9 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) {
const numAccounts = 1
_, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication)
defer cleanupFn()
args := base.TestServerArgs{ExternalIODir: dir}
args := base.TestServerArgs{
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
}

// Setup for sequence ownership backup/restore tests in the same database.
backupLoc := localFoo + `/d`
Expand Down Expand Up @@ -6026,7 +6036,7 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) {
dir, dirCleanupFn := testutils.TempDir(t)
defer dirCleanupFn()
params := base.TestClusterArgs{}
params.ServerArgs.ExternalIODir = dir
params.ServerArgs.StorageConfig.ExternalIODir = dir
params.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
params.ServerArgs.DefaultTestTenant = base.TestControlsTenantsExplicitly
tc := testcluster.StartTestCluster(t, 1, params)
Expand Down Expand Up @@ -6344,7 +6354,7 @@ func TestRestoreErrorPropagates(t *testing.T) {

params := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error {
Expand Down Expand Up @@ -6391,7 +6401,7 @@ func TestProtectedTimestampsFailDueToLimits(t *testing.T) {
dir, dirCleanupFn := testutils.TempDir(t)
defer dirCleanupFn()
params := base.TestClusterArgs{}
params.ServerArgs.ExternalIODir = dir
params.ServerArgs.StorageConfig.ExternalIODir = dir
params.ServerArgs.Knobs.ProtectedTS = &protectedts.TestingKnobs{
// The meta table is used to track limits.
UseMetaTable: true,
Expand All @@ -6412,7 +6422,7 @@ func TestProtectedTimestampsFailDueToLimits(t *testing.T) {
// TODO(adityamaru): Remove in 22.2 once no records protect spans.
t.Run("deprecated-spans-limit", func(t *testing.T) {
params := base.TestClusterArgs{}
params.ServerArgs.ExternalIODir = dir
params.ServerArgs.StorageConfig.ExternalIODir = dir
params.ServerArgs.Knobs.ProtectedTS = &protectedts.TestingKnobs{
DisableProtectedTimestampForMultiTenant: true,
// The meta table is used to track limits.
Expand Down Expand Up @@ -7092,7 +7102,7 @@ func TestBackupRestoreTenant(t *testing.T) {
t.Run("restore-tenant10-to-latest", func(t *testing.T) {
restoreTC := testcluster.StartTestCluster(
t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
DefaultTestTenant: base.TestControlsTenantsExplicitly,
Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()},
}},
Expand Down Expand Up @@ -7271,7 +7281,7 @@ func TestBackupRestoreTenant(t *testing.T) {
t.Run("restore-t10-from-cluster-backup", func(t *testing.T) {
restoreTC := testcluster.StartTestCluster(
t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
DefaultTestTenant: base.TestControlsTenantsExplicitly,
}},
)
Expand Down Expand Up @@ -7329,7 +7339,7 @@ func TestBackupRestoreTenant(t *testing.T) {
t.Run("restore-tenant10-to-ts1", func(t *testing.T) {
restoreTC := testcluster.StartTestCluster(
t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
DefaultTestTenant: base.TestControlsTenantsExplicitly,
}},
)
Expand Down Expand Up @@ -7365,7 +7375,7 @@ func TestBackupRestoreTenant(t *testing.T) {
t.Run("restore-tenant20-to-latest", func(t *testing.T) {
restoreTC := testcluster.StartTestCluster(
t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
DefaultTestTenant: base.TestControlsTenantsExplicitly,
}},
)
Expand Down Expand Up @@ -7540,7 +7550,7 @@ func TestBackupExportRequestTimeout(t *testing.T) {
params := base.TestClusterArgs{}
dir, dirCleanupFn := testutils.TempDir(t)
defer dirCleanupFn()
params.ServerArgs.ExternalIODir = dir
params.ServerArgs.StorageConfig.ExternalIODir = dir
const numAccounts = 10
ctx := context.Background()
tc, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, 2 /* nodes */, numAccounts,
Expand Down Expand Up @@ -8627,7 +8637,7 @@ func TestRestoreJobEventLogging(t *testing.T) {

baseDir := "testdata"
args := base.TestServerArgs{
ExternalIODir: baseDir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: baseDir},
Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}}
params := base.TestClusterArgs{ServerArgs: args}
tc, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, 1,
Expand Down Expand Up @@ -9196,7 +9206,7 @@ func TestGCDropIndexSpanExpansion(t *testing.T) {
defer cleanup()
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{
ExternalIODir: baseDir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: baseDir},
// This test hangs when run within a tenant. It's likely that
// the cause of the hang is the fact that we're waiting on the GC to
// complete, and we don't have visibility into the GC completing from
Expand Down Expand Up @@ -9485,7 +9495,7 @@ func TestExportRequestBelowGCThresholdOnDataExcludedFromBackup(t *testing.T) {
DisableLastProcessedCheck: true,
}
args.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
args.ServerArgs.ExternalIODir = localExternalDir
args.ServerArgs.StorageConfig.ExternalIODir = localExternalDir
tc := testcluster.StartTestCluster(t, 1, args)
defer tc.Stopper().Stop(ctx)

Expand Down Expand Up @@ -9579,7 +9589,7 @@ func TestExcludeDataFromBackupDoesNotHoldupGC(t *testing.T) {
dir, dirCleanupFn := testutils.TempDir(t)
defer dirCleanupFn()
params := base.TestClusterArgs{}
params.ServerArgs.ExternalIODir = dir
params.ServerArgs.StorageConfig.ExternalIODir = dir
// Test fails when run within a tenant. More investigation is
// required. Tracked with #76378.
params.ServerArgs.DefaultTestTenant = base.TODOTestTenantDisabled
Expand Down Expand Up @@ -10424,9 +10434,9 @@ func TestBackupRestoreTelemetryEvents(t *testing.T) {

baseDir := "testdata"
args := base.TestServerArgs{

ExternalIODir: baseDir,
Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}}
StorageConfig: storagepb.NodeConfig{ExternalIODir: baseDir},
Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()},
}
params := base.TestClusterArgs{ServerArgs: args}
tc, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, 1,
InitManualReplication, params)
Expand Down Expand Up @@ -11043,7 +11053,8 @@ func TestExportResponseDataSizeZeroCPUPagination(t *testing.T) {
externalDir, dirCleanup := testutils.TempDir(t)
defer dirCleanup()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
ExternalIODir: externalDir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: externalDir},

Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{
TestingRequestFilter: func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error {
for _, ru := range request.Requests {
Expand Down
1 change: 1 addition & 0 deletions pkg/backup/backuprand/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_test(
"//pkg/server",
"//pkg/sql/randgen",
"//pkg/sql/sem/tree",
"//pkg/storage/storagepb",
"//pkg/testutils",
"//pkg/testutils/fingerprintutils",
"//pkg/testutils/jobutils",
Expand Down
3 changes: 2 additions & 1 deletion pkg/backup/backuprand/backup_rand_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/fingerprintutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
Expand Down Expand Up @@ -51,7 +52,7 @@ func TestBackupRestoreRandomDataRoundtrips(t *testing.T) {
// with #76378.
DefaultTestTenant: base.TODOTestTenantDisabled,
UseDatabase: "rand",
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
},
}
const localFoo = "nodelocal://1/foo/"
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 @@ -186,10 +186,10 @@ func setTestClusterDefaults(params *base.TestClusterArgs, dataDir string, useDat
}
}

params.ServerArgs.ExternalIODir = dataDir
params.ServerArgs.StorageConfig.ExternalIODir = dataDir
for i := range params.ServerArgsPerNode {
param := params.ServerArgsPerNode[i]
param.ExternalIODir = dataDir + param.ExternalIODir
param.StorageConfig.ExternalIODir = dataDir + param.StorageConfig.ExternalIODir
params.ServerArgsPerNode[i] = param
}

Expand Down
3 changes: 2 additions & 1 deletion pkg/backup/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
Expand Down Expand Up @@ -93,7 +94,7 @@ func newTestHelper(t *testing.T) (*testHelper, func()) {
args := base.TestServerArgs{
Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "region", Value: "of-france"}}},
Settings: cluster.MakeClusterSettings(),
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
// Some scheduled backup tests fail when run within a tenant. More
// investigation is required. Tracked with #76378.
DefaultTestTenant: base.TODOTestTenantDisabled,
Expand Down
5 changes: 3 additions & 2 deletions pkg/backup/restore_data_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand Down Expand Up @@ -241,7 +242,7 @@ func runTestIngest(t *testing.T, init func(*cluster.Settings)) {
DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(107812),

Knobs: knobs,
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
Settings: cs,
}
// TODO(dan): This currently doesn't work with AddSSTable on in-memory
Expand All @@ -261,7 +262,7 @@ func runTestIngest(t *testing.T, init func(*cluster.Settings)) {
DB: s.InternalDB().(descs.DB),
ExternalStorage: func(ctx context.Context, dest cloudpb.ExternalStorage, opts ...cloud.ExternalStorageOption) (cloud.ExternalStorage, error) {
return cloud.MakeExternalStorage(ctx, dest, base.ExternalIODirConfig{},
s.ClusterSettings(), blobs.TestBlobServiceClient(args.ExternalIODir),
s.ClusterSettings(), blobs.TestBlobServiceClient(args.StorageConfig.ExternalIODir),
nil, /* db */
nil, /* limiters */
cloud.NilMetrics,
Expand Down
3 changes: 2 additions & 1 deletion pkg/backup/restore_mid_schema_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
Expand Down Expand Up @@ -238,7 +239,7 @@ func restoreMidSchemaChange(
dir, dirCleanupFn := testutils.TempDir(t)
params := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
ExternalIODir: dir,
StorageConfig: storagepb.NodeConfig{ExternalIODir: dir},
Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()},
},
}
Expand Down
Loading

0 comments on commit 523550f

Please sign in to comment.