Skip to content

Commit

Permalink
server: trim deps in StartTenant
Browse files Browse the repository at this point in the history
`(*TestServer).StartTenant` now uses only the Gossip instance(*) from
the TestServer and sets everything else up from scratch. This
demonstrates that mod the Gossip dependency, we're en route to
running the SQL server in a standalone process.

Informs #49104.

(*): and the Stopper, but that's fine.

Release note: None
  • Loading branch information
tbg committed May 28, 2020
1 parent 807dc36 commit 7b7fcb0
Show file tree
Hide file tree
Showing 3 changed files with 87 additions and 24 deletions.
2 changes: 1 addition & 1 deletion pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ type TestServerArgs struct {

// Fields copied to the server.Config.
Insecure bool
RetryOptions retry.Options
RetryOptions retry.Options // TODO(tbg): make testing knob.
SocketFile string
ScanInterval time.Duration
ScanMinIdleTime time.Duration
Expand Down
2 changes: 2 additions & 0 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,8 @@ type Config struct {
JoinPreferSRVRecords bool

// RetryOptions controls the retry behavior of the server.
//
// TODO(tbg): this is only ever used in one test. Make it a testing knob.
RetryOptions retry.Options

// CacheSize is the amount of memory in bytes to use for caching data.
Expand Down
107 changes: 84 additions & 23 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -439,25 +439,93 @@ func (d dummyProtectedTSProvider) Protect(context.Context, *kv.Txn, *ptpb.Record
}

func testSQLServerArgs(ts *TestServer) sqlServerArgs {
st := cluster.MakeTestingClusterSettings()
stopper := ts.Stopper()
// If we used a dummy gossip, DistSQL and random other things won't work.
// Just use the test server's for now.
//
// TODO(tbg): drop the Gossip dependency.
g := ts.Gossip()
ts = nil // prevent usage below

cfg := makeTestConfig(st)
st := cluster.MakeTestingClusterSettings()

clock := hlc.NewClock(hlc.UnixNano, 1)
rpcContext := rpc.NewInsecureTestingContext(clock, stopper)
var sqlConfig SQLConfig
var bothConfig BothConfig
{
cfg := makeTestConfig(st)
sqlConfig = cfg.SQLConfig
bothConfig = cfg.BothConfig
}

nl := allErrorsFakeLiveness{}
clock := hlc.NewClock(hlc.UnixNano, time.Duration(bothConfig.MaxOffset))

ds := ts.DistSender()
var rpcTestingKnobs rpc.ContextTestingKnobs
if p, ok := bothConfig.TestingKnobs.Server.(*TestingKnobs); ok {
rpcTestingKnobs = p.ContextTestingKnobs
}
rpcContext := rpc.NewContextWithTestingKnobs(
bothConfig.AmbientCtx,
bothConfig.Config,
clock,
stopper,
st,
rpcTestingKnobs,
)

var dsKnobs kvcoord.ClientTestingKnobs
if dsKnobsP, ok := bothConfig.TestingKnobs.DistSQL.(*kvcoord.ClientTestingKnobs); ok {
dsKnobs = *dsKnobsP
}
rpcRetryOptions := base.DefaultRetryOptions()
resolver := gossip.AddressResolver(g) // TODO(tbg): break gossip dep
nodeDialer := nodedialer.New(rpcContext, resolver)
dsCfg := kvcoord.DistSenderConfig{
AmbientCtx: bothConfig.AmbientCtx,
Settings: st,
Clock: clock,
RPCRetryOptions: &rpcRetryOptions,
RPCContext: rpcContext,
RangeDescriptorDB: nil, // use DistSender itself
NodeDialer: nodeDialer,
TestingKnobs: dsKnobs,
}
ds := kvcoord.NewDistSender(dsCfg, g)

var clientKnobs kvcoord.ClientTestingKnobs
if p, ok := bothConfig.TestingKnobs.KVClient.(*kvcoord.ClientTestingKnobs); ok {
clientKnobs = *p
}
// TODO(tbg): expose this registry via prometheus. See:
// https://github.com/cockroachdb/cockroach/issues/47905
registry := metric.NewRegistry()
txnMetrics := kvcoord.MakeTxnMetrics(bothConfig.HistogramWindowInterval())
registry.AddMetricStruct(txnMetrics)
tcsFactory := kvcoord.NewTxnCoordSenderFactory(
kvcoord.TxnCoordSenderFactoryConfig{
AmbientCtx: bothConfig.AmbientCtx,
Settings: st,
Clock: clock,
Stopper: stopper,
HeartbeatInterval: base.DefaultTxnHeartbeatInterval,
Linearizable: false,
Metrics: txnMetrics,
TestingKnobs: clientKnobs,
},
ds,
)
db := kv.NewDB(
bothConfig.AmbientCtx,
tcsFactory,
clock,
)

circularInternalExecutor := &sql.InternalExecutor{}
// Protected timestamps won't be available (at first) in multi-tenant
// clusters.
var protectedTSProvider protectedts.Provider
{
pp, err := ptprovider.New(ptprovider.Config{
DB: ts.DB(),
DB: db,
InternalExecutor: circularInternalExecutor,
Settings: st,
})
Expand All @@ -467,15 +535,6 @@ func testSQLServerArgs(ts *TestServer) sqlServerArgs {
protectedTSProvider = dummyProtectedTSProvider{pp}
}

registry := metric.NewRegistry()

// If we used a dummy gossip, DistSQL and random other things won't work.
// Just use the test server's for now.
// g := gossip.NewTest(nodeID, nil, nil, stopper, registry, nil)
g := ts.Gossip()

nd := nodedialer.New(rpcContext, gossip.AddressResolver(ts.Gossip()))

dummyRecorder := &status.MetricsRecorder{}

// TODO(asubiotto): Jobs don't play well with a weird node ID in a multitenant
Expand All @@ -497,9 +556,9 @@ func testSQLServerArgs(ts *TestServer) sqlServerArgs {
rpcContext: rpcContext,
distSender: ds,
statusServer: noStatusServer,
nodeLiveness: nl,
nodeLiveness: allErrorsFakeLiveness{},
gossip: gossip.MakeUnexposedGossip(g),
nodeDialer: nd,
nodeDialer: nodeDialer,
grpcServer: dummyRPCServer,
recorder: dummyRecorder,
isMeta1Leaseholder: func(timestamp hlc.Timestamp) (bool, error) {
Expand All @@ -513,13 +572,13 @@ func testSQLServerArgs(ts *TestServer) sqlServerArgs {
return nil, errors.New("fake external uri storage")
},
},
SQLConfig: &cfg.SQLConfig,
BothConfig: &cfg.BothConfig,
SQLConfig: &sqlConfig,
BothConfig: &bothConfig,
stopper: stopper,
clock: clock,
runtime: status.NewRuntimeStatSampler(context.Background(), clock),
tenantID: roachpb.SystemTenantID,
db: ts.DB(),
db: db,
registry: registry,
sessionRegistry: sql.NewSessionRegistry(),
circularInternalExecutor: circularInternalExecutor,
Expand All @@ -545,6 +604,8 @@ func (ts *TestServer) StartTenant(params base.TestTenantArgs) (pgAddr string, _
ClusterSettingsUpdater: args.Settings.MakeUpdater(),
}
}
ts = nil // proves we're not using it below

s, err := newSQLServer(ctx, args)
if err != nil {
return "", err
Expand All @@ -566,8 +627,8 @@ func (ts *TestServer) StartTenant(params base.TestTenantArgs) (pgAddr string, _
if err != nil {
return "", err
}
ts.Stopper().RunWorker(ctx, func(ctx context.Context) {
<-ts.Stopper().ShouldQuiesce()
args.stopper.RunWorker(ctx, func(ctx context.Context) {
<-args.stopper.ShouldQuiesce()
// NB: we can't do this as a Closer because (*Server).ServeWith is
// running in a worker and usually sits on accept(pgL) which unblocks
// only when pgL closes. In other words, pgL needs to close when
Expand Down

0 comments on commit 7b7fcb0

Please sign in to comment.