Skip to content

Commit

Permalink
Merge pull request #17486 from tschottdorf/settings-refactor
Browse files Browse the repository at this point in the history
*: un-singletonize the cluster settings
  • Loading branch information
tbg authored Aug 8, 2017
2 parents e1b502b + fc7bfdf commit a5d4193
Show file tree
Hide file tree
Showing 145 changed files with 2,012 additions and 1,378 deletions.
3 changes: 2 additions & 1 deletion pkg/acceptance/cluster/localcluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

Expand Down Expand Up @@ -815,7 +816,7 @@ func (l *LocalCluster) stop(ctx context.Context) {
// NewClient implements the Cluster interface.
func (l *LocalCluster) NewClient(ctx context.Context, i int) (*roachClient.DB, error) {
clock := hlc.NewClock(hlc.UnixNano, 0)
rpcContext := rpc.NewContext(log.AmbientContext{}, &base.Config{
rpcContext := rpc.NewContext(log.AmbientContext{Tracer: tracing.NewTracer()}, &base.Config{
User: security.NodeUser,
SSLCertsDir: l.CertsDir,
}, clock, l.stopper)
Expand Down
3 changes: 2 additions & 1 deletion pkg/acceptance/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
)

type keepClusterVar string
Expand Down Expand Up @@ -266,7 +267,7 @@ func MakeFarmer(t testing.TB, prefix string, stopper *stop.Stopper) *terrafarm.F
// may control a different acceptable offset for the nodes in the cluster. We
// should stop creating transaction outside of the cluster.
clientClock := hlc.NewClock(hlc.UnixNano, base.DefaultMaxClockOffset)
rpcContext := rpc.NewContext(log.AmbientContext{}, &base.Config{
rpcContext := rpc.NewContext(log.AmbientContext{Tracer: tracing.NewTracer()}, &base.Config{
Insecure: true,
User: security.NodeUser,
// Set a bogus address, to be used by the clock skew checks as the ID of
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/sqlccl/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -588,7 +588,7 @@ func backupPlanHook(
}

if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().ClusterID(), p.ExecCfg().Organization.Get(), "BACKUP",
p.ExecCfg().Settings, p.ExecCfg().ClusterID(), p.ExecCfg().Organization.Get(), "BACKUP",
); err != nil {
return nil, nil, err
}
Expand Down Expand Up @@ -792,7 +792,7 @@ func showBackupPlanHook(
}

if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().ClusterID(), p.ExecCfg().Organization.Get(), "SHOW BACKUP",
p.ExecCfg().Settings, p.ExecCfg().ClusterID(), p.ExecCfg().Organization.Get(), "SHOW BACKUP",
); err != nil {
return nil, nil, err
}
Expand Down
9 changes: 6 additions & 3 deletions pkg/ccl/sqlccl/backup_cloud_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,14 @@ import (
"github.com/rlmcpherson/s3gof3r"

"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)

func initNone(_ *cluster.Settings) {}

// The tests in this file talk to remote APIs which require credentials.
// To run these tests, you need to supply credentials via env vars (the tests
// skip themselves if they are not set). Obtain these credentials from the
Expand All @@ -47,7 +50,7 @@ func TestCloudBackupRestoreS3(t *testing.T) {
defer sql.TestDisableTableLeases()()
const numAccounts = 1000

ctx, _, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts)
ctx, _, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts, initNone)
defer cleanupFn()
prefix := fmt.Sprintf("TestBackupRestoreS3-%d", timeutil.Now().UnixNano())
uri := url.URL{Scheme: "s3", Host: bucket, Path: prefix}
Expand Down Expand Up @@ -80,7 +83,7 @@ func TestCloudBackupRestoreGoogleCloudStorage(t *testing.T) {
}(http.DefaultTransport.(*http.Transport).DisableKeepAlives)
http.DefaultTransport.(*http.Transport).DisableKeepAlives = true

ctx, _, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts)
ctx, _, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts, initNone)
defer cleanupFn()
prefix := fmt.Sprintf("TestBackupRestoreGoogleCloudStorage-%d", timeutil.Now().UnixNano())
uri := url.URL{Scheme: "gs", Host: bucket, Path: prefix}
Expand Down Expand Up @@ -112,7 +115,7 @@ func TestCloudBackupRestoreAzure(t *testing.T) {
}(http.DefaultTransport.(*http.Transport).DisableKeepAlives)
http.DefaultTransport.(*http.Transport).DisableKeepAlives = true

ctx, _, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts)
ctx, _, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, 1, numAccounts, initNone)
defer cleanupFn()
prefix := fmt.Sprintf("TestBackupRestoreAzure-%d", timeutil.Now().UnixNano())
uri := url.URL{Scheme: "azure", Host: bucket, Path: prefix}
Expand Down
74 changes: 44 additions & 30 deletions pkg/ccl/sqlccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,12 +36,11 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/sqlccl"
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl/sampledataccl"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/jobs"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
Expand All @@ -65,7 +64,11 @@ const (
)

func backupRestoreTestSetupWithParams(
t testing.TB, clusterSize int, numAccounts int, params base.TestClusterArgs,
t testing.TB,
clusterSize int,
numAccounts int,
init func(*cluster.Settings),
params base.TestClusterArgs,
) (
ctx context.Context,
tempDir string,
Expand All @@ -79,6 +82,10 @@ func backupRestoreTestSetupWithParams(

tc = testcluster.StartTestCluster(t, clusterSize, params)

for _, server := range tc.Servers {
init(server.ClusterSettings())
}

const payloadSize = 100
splits := 10
if numAccounts == 0 {
Expand Down Expand Up @@ -123,15 +130,15 @@ func backupRestoreTestSetupWithParams(
}

func backupRestoreTestSetup(
t testing.TB, clusterSize int, numAccounts int,
t testing.TB, clusterSize int, numAccounts int, init func(*cluster.Settings),
) (
ctx context.Context,
tempDir string,
tc *testcluster.TestCluster,
sqlDB *sqlutils.SQLRunner,
cleanup func(),
) {
return backupRestoreTestSetupWithParams(t, clusterSize, numAccounts, base.TestClusterArgs{})
return backupRestoreTestSetupWithParams(t, clusterSize, numAccounts, init, base.TestClusterArgs{})
}

func verifyBackupRestoreStatementResult(
Expand Down Expand Up @@ -189,7 +196,7 @@ func TestBackupRestoreStatementResult(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 1
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

if err := verifyBackupRestoreStatementResult(
Expand All @@ -211,18 +218,25 @@ func TestBackupRestoreLocal(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 1000
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts)
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, initNone)
defer cleanupFn()

backupAndRestore(ctx, t, sqlDB, dir, numAccounts)
}

func enableAddSSTable(st *cluster.Settings) {
st.AddSSTableEnabled.Override(true)
}

func disableAddSSTable(st *cluster.Settings) {
st.AddSSTableEnabled.Override(false)
}

func TestBackupRestoreAddSSTable(t *testing.T) {
defer leaktest.AfterTest(t)()
defer settings.TestingSetBool(&storageccl.AddSSTableEnabled, true)()

const numAccounts = 1000
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts)
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, enableAddSSTable)
defer cleanupFn()

backupAndRestore(ctx, t, sqlDB, dir, numAccounts)
Expand All @@ -232,7 +246,7 @@ func TestBackupRestoreEmpty(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 0
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts)
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, initNone)
defer cleanupFn()

backupAndRestore(ctx, t, sqlDB, dir, numAccounts)
Expand All @@ -245,7 +259,7 @@ func TestBackupRestoreNegativePrimaryKey(t *testing.T) {

const numAccounts = 1000

ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts)
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, initNone)
defer cleanupFn()

// Give half the accounts negative primary keys.
Expand Down Expand Up @@ -394,7 +408,7 @@ func TestBackupRestoreSystemJobs(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 0
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, initNone)
defer cleanupFn()

sanitizedIncDir := dir + "/inc"
Expand Down Expand Up @@ -508,7 +522,7 @@ func checkInProgressBackupRestore(
const numAccounts = 1000
const totalExpectedResponses = backupRestoreDefaultRanges

ctx, dir, _, sqlDB, cleanup := backupRestoreTestSetupWithParams(t, multiNode, numAccounts, params)
ctx, dir, _, sqlDB, cleanup := backupRestoreTestSetupWithParams(t, multiNode, numAccounts, initNone, params)
defer cleanup()

sqlDB.Exec(`CREATE DATABASE restoredb`)
Expand Down Expand Up @@ -706,7 +720,7 @@ func TestBackupRestoreResume(t *testing.T) {
ctx := context.Background()

const numAccounts = 1000
_, dir, tc, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts)
_, dir, tc, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, initNone)
defer cleanupFn()

backupTableDesc := sqlbase.GetTableDescriptor(tc.Servers[0].DB(), "data", "bank")
Expand Down Expand Up @@ -839,7 +853,7 @@ func TestBackupRestoreControlJob(t *testing.T) {
}

const numAccounts = 1000
_, dir, _, sqlDB, cleanup := backupRestoreTestSetupWithParams(t, multiNode, numAccounts, params)
_, dir, _, sqlDB, cleanup := backupRestoreTestSetupWithParams(t, multiNode, numAccounts, initNone, params)
defer cleanup()

run := func(op, query string, args ...interface{}) (int64, error) {
Expand Down Expand Up @@ -903,7 +917,7 @@ func TestBackupRestoreInterleaved(t *testing.T) {
defer leaktest.AfterTest(t)()
const numAccounts = 10

_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

// TODO(dan): The INTERLEAVE IN PARENT clause currently doesn't allow the
Expand Down Expand Up @@ -1005,7 +1019,7 @@ func TestBackupRestoreCrossTableReferences(t *testing.T) {
const createStore = "CREATE DATABASE store"
const createStoreStats = "CREATE DATABASE storestats"

_, dir, _, origDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, _, origDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

// Generate some testdata and back it up.
Expand Down Expand Up @@ -1312,7 +1326,7 @@ func TestBackupRestoreIncremental(t *testing.T) {
const numBackups = 4
windowSize := int(numAccounts / 3)

_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, 0)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, initNone)
defer cleanupFn()
rng, _ := randutil.NewPseudoRand()

Expand Down Expand Up @@ -1438,7 +1452,7 @@ func TestBackupRestoreWithConcurrentWrites(t *testing.T) {
const rows = 10
const numBackgroundTasks = multiNode

_, baseDir, tc, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, rows)
_, baseDir, tc, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, rows, initNone)
defer cleanupFn()

bgActivity := make(chan struct{})
Expand Down Expand Up @@ -1490,7 +1504,7 @@ func TestConcurrentBackupRestores(t *testing.T) {

const numAccounts = 10
const concurrency, numIterations = 2, 3
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts)
ctx, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, initNone)
defer cleanupFn()

g, gCtx := errgroup.WithContext(ctx)
Expand Down Expand Up @@ -1537,7 +1551,7 @@ func TestBackupAsOfSystemTime(t *testing.T) {

const numAccounts = 1000

_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, initNone)
defer cleanupFn()

var beforeTs, equalTs string
Expand Down Expand Up @@ -1577,7 +1591,7 @@ func TestBackupAsOfSystemTime(t *testing.T) {
func TestAsOfSystemTimeOnRestoredData(t *testing.T) {
defer leaktest.AfterTest(t)()

_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, 0)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, multiNode, 0, initNone)
defer cleanupFn()
sqlDB.Exec(`DROP TABLE data.bank`)

Expand Down Expand Up @@ -1612,7 +1626,7 @@ func TestBackupRestoreChecksum(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 1000
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

// The helper helpfully prefixes it, but we're going to do direct file IO.
Expand Down Expand Up @@ -1660,7 +1674,7 @@ func TestTimestampMismatch(t *testing.T) {
defer leaktest.AfterTest(t)()
const numAccounts = 1

_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()
sqlDB.Exec(`CREATE TABLE data.t2 (a INT PRIMARY KEY)`)
sqlDB.Exec(`INSERT INTO data.t2 VALUES (1)`)
Expand Down Expand Up @@ -1745,7 +1759,7 @@ func TestTimestampMismatch(t *testing.T) {
func TestBackupLevelDB(t *testing.T) {
defer leaktest.AfterTest(t)()

_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, 0)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, 0, initNone)
defer cleanupFn()

_ = sqlDB.Exec(`BACKUP DATABASE data TO $1`, dir)
Expand Down Expand Up @@ -1778,7 +1792,7 @@ func TestRestoredPrivileges(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 1
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

rootOnly := sqlDB.QueryStr(`SHOW GRANTS ON data.bank`)
Expand Down Expand Up @@ -1816,7 +1830,7 @@ func TestRestoreInto(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 1
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

sqlDB.Exec(`BACKUP DATABASE data TO $1`, dir)
Expand All @@ -1839,7 +1853,7 @@ func TestBackupRestorePermissions(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 1
_, dir, tc, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, tc, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

sqlDB.Exec(`CREATE USER testuser`)
Expand Down Expand Up @@ -1884,7 +1898,7 @@ func TestShowBackup(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 1
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, dir, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

now := timeutil.Now()
Expand All @@ -1909,7 +1923,7 @@ func TestBackupAzureAccountName(t *testing.T) {
defer leaktest.AfterTest(t)()

const numAccounts = 1
_, _, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts)
_, _, _, sqlDB, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, initNone)
defer cleanupFn()

values := url.Values{}
Expand Down
Loading

0 comments on commit a5d4193

Please sign in to comment.