Skip to content

Commit

Permalink
Merge #49693 #49724
Browse files Browse the repository at this point in the history
49693: server: use "read-only" Gossip for tenants r=asubiotto,nvanbenschoten a=tbg

We were previously using the Gossip instance of the TestServer against
which the tenant was initialized.

This commit trims the dependency further by initializing its own Gossip
instance which is never written to (i.e. `AddInfo` is never called) and
which does not accept incoming connections.

As a reminder, the remaining problematic uses of Gossip as of this
commit are:

- making a `nodeDialer` (for `DistSender`), tracked in:
  #47909
- access to the system config:
  - `(schemaChangeGCResumer).Resume`, tracked:
    #49691
  - `database.Cache`, tracked:
    #49692
- `(physicalplan).spanResolver` (for replica oracle).
  This is likely not a blocker as we can use a "dumber" oracle in this case;
  the oracle is used for distsql physical planning of which tenants will
  do none. Tracked in: #48432

cc @ajwerner 

Release note: None

49724: sql: clean up of scanNode and some other things r=yuzefovich a=yuzefovich

**sql: unify PlanningCtx constructors into one**

Release note: None

**sql: remove separate scanVisilibity struct**

This commit removes `sql.scanVisibility` in favor of protobuf-generated
`execinfrapb.ScanVisibility`. It also introduces prettier aliases for
the two values into `execinfra` package that are now used throughout the
code.

Release note: None

**sql: clean up of scan node and a few other things**

This commit does the following cleanups of `scanNode`:
1. refactors `scanNode.initCols` method to be standalone (it will
probably be reused later by distsql spec exec factory).
2. removes `numBackfillColumns`, `specifiedIndexReverse`, and
`isSecondaryIndex` fields since they are no longer used.
3. refactors the code to remove `valNeededForCols` field which was
always consecutive numbers in range `[0, len(n.cols)-1]`.
4. refactors `getIndexIdx` method to not depend on `scanNode`.

Additionally, this commit removes `planDependencies` business
from `planTop` since optimizer now handles CREATE VIEW and handles
the plan dependencies on its own (and CREATE VIEW was the single
user of that struct in the plan top).

Also, it removes (which seems like) unnecessary call to `planColumns`
when creating distinct spec and an unused parameter from
`createTableReaders` method.

Addresses: #47474.

Release note: None

Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
3 people committed Jun 2, 2020
3 parents b4b7d45 + d94b356 + e2ac346 commit 3b2245f
Show file tree
Hide file tree
Showing 40 changed files with 382 additions and 403 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ func distChangefeedFlow(
}
dsp := phs.DistSQLPlanner()
evalCtx := phs.ExtendedEvalContext()
planCtx := dsp.NewPlanningCtx(ctx, evalCtx, noTxn)
planCtx := dsp.NewPlanningCtx(ctx, evalCtx, noTxn, true /* distribute */)

var spanPartitions []sql.SpanPartition
if details.SinkURI == `` {
Expand Down
6 changes: 6 additions & 0 deletions pkg/gossip/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -1659,6 +1659,12 @@ type DeprecatedGossip struct {
w errorutil.TenantSQLDeprecatedWrapper
}

// Start calls .Start() on the underlying Gossip instance, which is assumed to
// be non-nil.
func (dg DeprecatedGossip) Start(advertAddr net.Addr, resolvers []resolver.Resolver) {
dg.w.Deprecated(0).(*Gossip).Start(advertAddr, resolvers)
}

// deprecated trades a Github issue tracking the removal of the call for the
// wrapped Gossip instance.
func (dg DeprecatedGossip) deprecated(issueNo int) *Gossip {
Expand Down
109 changes: 68 additions & 41 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,14 +138,31 @@ type BaseConfig struct {
TestingKnobs base.TestingKnobs
}

// Config holds parameters needed to setup a (combined KV and SQL) server.
//
// TODO(tbg): this should end up being just SQLConfig union KVConfig union BaseConfig.
// MakeBaseConfig returns a BaseConfig with default values.
func MakeBaseConfig(st *cluster.Settings) BaseConfig {
baseCfg := BaseConfig{
AmbientCtx: log.AmbientContext{Tracer: st.Tracer},
Config: new(base.Config),
Settings: st,
MaxOffset: MaxOffsetType(base.DefaultMaxClockOffset),
DefaultZoneConfig: zonepb.DefaultZoneConfig(),
StorageEngine: storage.DefaultStorageEngine,
}
baseCfg.InitDefaults()
return baseCfg
}

// Config holds the parameters needed to set up a combined KV and SQL server.
type Config struct {
// Embed the base context.
BaseConfig
base.RaftConfig
KVConfig
SQLConfig
}

// KVConfig holds the parameters that (together with a BaseConfig) allow setting
// up a KV server.
type KVConfig struct {
base.RaftConfig

// Stores is specified to enable durable key-value storage.
Stores base.StoreSpecList
Expand Down Expand Up @@ -257,8 +274,31 @@ type Config struct {
enginesCreated bool
}

// SQLConfig holds the parameters to setup a SQL server.
// MakeKVConfig returns a KVConfig with default values.
func MakeKVConfig(storeSpec base.StoreSpec) KVConfig {
disableWebLogin := envutil.EnvOrDefaultBool("COCKROACH_DISABLE_WEB_LOGIN", false)
kvCfg := KVConfig{
DefaultSystemZoneConfig: zonepb.DefaultSystemZoneConfig(),
CacheSize: DefaultCacheSize,
ScanInterval: defaultScanInterval,
ScanMinIdleTime: defaultScanMinIdleTime,
ScanMaxIdleTime: defaultScanMaxIdleTime,
EventLogEnabled: defaultEventLogEnabled,
EnableWebSessionAuthentication: !disableWebLogin,
Stores: base.StoreSpecList{
Specs: []base.StoreSpec{storeSpec},
},
}
kvCfg.RaftConfig.SetDefaults()
return kvCfg
}

// SQLConfig holds the parameters that (together with a BaseConfig) allow
// setting up a SQL server.
type SQLConfig struct {
// The tenant that the SQL server runs on the behalf of.
TenantID roachpb.TenantID

// LeaseManagerConfig holds configuration values specific to the LeaseManager.
LeaseManagerConfig *base.LeaseManagerConfig

Expand Down Expand Up @@ -289,6 +329,19 @@ type SQLConfig struct {
QueryCacheSize int64
}

// MakeSQLConfig returns a SQLConfig with default values.
func MakeSQLConfig(tenID roachpb.TenantID, tempStorageCfg base.TempStorageConfig) SQLConfig {
sqlCfg := SQLConfig{
TenantID: tenID,
MemoryPoolSize: defaultSQLMemoryPoolSize,
TableStatCacheSize: defaultSQLTableStatCacheSize,
QueryCacheSize: defaultSQLQueryCacheSize,
TempStorageConfig: tempStorageCfg,
LeaseManagerConfig: base.NewLeaseManagerConfig(),
}
return sqlCfg
}

// setOpenFileLimit sets the soft limit for open file descriptors to the hard
// limit if needed. Returns an error if the hard limit is too low. Returns the
// value to set maxOpenFiles to for each store.
Expand Down Expand Up @@ -317,50 +370,24 @@ func SetOpenFileLimitForOneStore() (uint64, error) {
return setOpenFileLimit(1)
}

// MakeConfig returns a Config with default values.
// MakeConfig returns a Config for the system tenant with default values.
func MakeConfig(ctx context.Context, st *cluster.Settings) Config {
storeSpec, err := base.NewStoreSpec(defaultStorePath)
if err != nil {
panic(err)
}
tempStorageCfg := base.TempStorageConfigFromEnv(
ctx, st, storeSpec, "" /* parentDir */, base.DefaultTempStorageMaxSizeBytes)

disableWebLogin := envutil.EnvOrDefaultBool("COCKROACH_DISABLE_WEB_LOGIN", false)

sqlCfg := SQLConfig{
MemoryPoolSize: defaultSQLMemoryPoolSize,
TableStatCacheSize: defaultSQLTableStatCacheSize,
QueryCacheSize: defaultSQLQueryCacheSize,
TempStorageConfig: base.TempStorageConfigFromEnv(
ctx, st, storeSpec, "" /* parentDir */, base.DefaultTempStorageMaxSizeBytes),
LeaseManagerConfig: base.NewLeaseManagerConfig(),
}

bothCfg := BaseConfig{
Config: new(base.Config),
Settings: st,
MaxOffset: MaxOffsetType(base.DefaultMaxClockOffset),
DefaultZoneConfig: zonepb.DefaultZoneConfig(),
StorageEngine: storage.DefaultStorageEngine,
}
sqlCfg := MakeSQLConfig(roachpb.SystemTenantID, tempStorageCfg)
baseCfg := MakeBaseConfig(st)
kvCfg := MakeKVConfig(storeSpec)

cfg := Config{
BaseConfig: bothCfg,
SQLConfig: sqlCfg,
DefaultSystemZoneConfig: zonepb.DefaultSystemZoneConfig(),
CacheSize: DefaultCacheSize,
ScanInterval: defaultScanInterval,
ScanMinIdleTime: defaultScanMinIdleTime,
ScanMaxIdleTime: defaultScanMaxIdleTime,
EventLogEnabled: defaultEventLogEnabled,
EnableWebSessionAuthentication: !disableWebLogin,
Stores: base.StoreSpecList{
Specs: []base.StoreSpec{storeSpec},
},
BaseConfig: baseCfg,
KVConfig: kvCfg,
SQLConfig: sqlCfg,
}
cfg.AmbientCtx.Tracer = st.Tracer

cfg.Config.InitDefaults()
cfg.RaftConfig.SetDefaults()

return cfg
}
Expand Down
3 changes: 1 addition & 2 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -549,12 +549,11 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
stopper: stopper,
clock: clock,
runtime: runtimeSampler,
tenantID: roachpb.SystemTenantID,
db: db,
registry: registry,
sessionRegistry: sessionRegistry,
circularInternalExecutor: internalExecutor,
jobRegistry: jobRegistry,
circularJobRegistry: jobRegistry,
jobAdoptionStopFile: jobAdoptionStopFile,
protectedtsProvider: protectedtsProvider,
})
Expand Down
9 changes: 3 additions & 6 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,9 +140,6 @@ type sqlServerArgs struct {
// samplerProcessor.
runtime execinfra.RuntimeStats

// The tenant that the SQL server runs on the behalf of.
tenantID roachpb.TenantID

// SQL uses KV, both for non-DistSQL and DistSQL execution.
db *kv.DB

Expand All @@ -161,7 +158,7 @@ type sqlServerArgs struct {
// The protected timestamps KV subsystem depends on this, so we pass a
// pointer to an empty struct in this configuration, which newSQLServer
// fills.
jobRegistry *jobs.Registry
circularJobRegistry *jobs.Registry
jobAdoptionStopFile string

// The executorConfig uses the provider.
Expand All @@ -170,7 +167,7 @@ type sqlServerArgs struct {

func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
execCfg := &sql.ExecutorConfig{}
codec := keys.MakeSQLCodec(cfg.tenantID)
codec := keys.MakeSQLCodec(cfg.SQLConfig.TenantID)

// Create blob service for inter-node file sharing.
blobService, err := blobs.NewBlobService(cfg.Settings.ExternalIODir)
Expand All @@ -179,7 +176,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
}
blobspb.RegisterBlobServer(cfg.grpcServer, blobService)

jobRegistry := cfg.jobRegistry
jobRegistry := cfg.circularJobRegistry

{
regLiveness := cfg.nodeLiveness
Expand Down
Loading

0 comments on commit 3b2245f

Please sign in to comment.