Skip to content

Commit

Permalink
server: only run tenants with service mode 'shared'
Browse files Browse the repository at this point in the history
Prior to this patch, services for secondary tenants would be started
automatically upon first use by a client.

This commit changes this to auto-start services upfront for all
tenants with service mode SHARED. (And shut down services for tenants
with another service mode configured.)

Release note: None
  • Loading branch information
knz committed Jan 26, 2023
1 parent 32b5345 commit 67c573c
Show file tree
Hide file tree
Showing 7 changed files with 294 additions and 54 deletions.
70 changes: 68 additions & 2 deletions pkg/ccl/serverccl/server_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ package serverccl

import (
"context"
gosql "database/sql"
"fmt"
"io"
"net/http"
Expand All @@ -22,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -57,12 +59,22 @@ func TestServerControllerHTTP(t *testing.T) {
require.NoError(t, row.Scan(&id, &secret, &username, &created, &expires))

// Create our own test tenant with a known name.
_, err = db.Exec("SELECT crdb_internal.create_tenant(10, 'hello')")
_, err = db.Exec("CREATE TENANT hello; ALTER TENANT hello START SERVICE SHARED")
require.NoError(t, err)

// Get a SQL connection to the test tenant.
sqlAddr := s.ServingSQLAddr()
db2 := serverutils.OpenDBConn(t, sqlAddr, "cluster:hello/defaultdb", false, s.Stopper())
var db2 *gosql.DB
testutils.SucceedsSoon(t, func() error {
var err error
db2, err = serverutils.OpenDBConnE(sqlAddr, "cluster:hello/defaultdb", false, s.Stopper())
// Expect no error yet: the connection is opened lazily; an
// error here means the parameters were incorrect.
require.NoError(t, err)

// This actually uses the connection.
return db2.Ping()
})

// Instantiate the HTTP test username and privileges into the test tenant.
_, err = db2.Exec(fmt.Sprintf(`CREATE USER %s`, lexbase.EscapeSQLIdent(username)))
Expand Down Expand Up @@ -170,3 +182,57 @@ VALUES($1, $2, $3, $4, $5)`, id, secret, username, created, expires)
require.Equal(t, len(body.Sessions), 1)
require.Equal(t, body.Sessions[0].ApplicationName, "hello system")
}

func TestServerStartStop(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()

s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
DisableDefaultTestTenant: true,
})
defer s.Stopper().Stop(ctx)

sqlAddr := s.ServingSQLAddr()

// Create our own test tenant with a known name.
_, err := db.Exec("CREATE TENANT hello")
require.NoError(t, err)

// Make the service alive.
_, err = db.Exec("ALTER TENANT hello START SERVICE SHARED")
require.NoError(t, err)

// Check the liveness.
testutils.SucceedsSoon(t, func() error {
db2, err := serverutils.OpenDBConnE(sqlAddr, "cluster:hello/defaultdb", false, s.Stopper())
// Expect no error yet: the connection is opened lazily; an
// error here means the parameters were incorrect.
require.NoError(t, err)

defer db2.Close()
if err := db2.Ping(); err != nil {
return err
}
return nil
})

// Stop the service. .
_, err = db.Exec("ALTER TENANT hello STOP SERVICE")
require.NoError(t, err)

// Verify that the service is indeed stopped.
testutils.SucceedsSoon(t, func() error {
db2, err := serverutils.OpenDBConnE(sqlAddr, "cluster:hello/defaultdb", false, s.Stopper())
// Expect no error yet: the connection is opened lazily; an
// error here means the parameters were incorrect.
require.NoError(t, err)
defer db2.Close()
if err := db2.Ping(); err != nil {
// Connection error: success.
return nil
}
return errors.New("server still alive")
})
}
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,7 @@ go_library(
"//pkg/kv/kvserver/rangelog",
"//pkg/kv/kvserver/reports",
"//pkg/multitenant",
"//pkg/multitenant/mtinfopb",
"//pkg/multitenant/multitenantcpu",
"//pkg/multitenant/multitenantio",
"//pkg/multitenant/tenantcostmodel",
Expand Down
30 changes: 12 additions & 18 deletions pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -4219,33 +4219,27 @@ func (s *adminServer) RecoveryVerify(
return nil, errors.AssertionFailedf("To be implemented by #93043")
}

// ListTenants returns a list of active tenants in the cluster. Calling this
// function will start in-process tenants if they are not already running.
// ListTenants returns a list of tenants that are served
// by shared-process services in this server.
func (s *systemAdminServer) ListTenants(
ctx context.Context, _ *serverpb.ListTenantsRequest,
) (*serverpb.ListTenantsResponse, error) {
ie := s.internalExecutor
rowIter, err := ie.QueryIterator(ctx, "list-tenants", nil, /* txn */
`SELECT name FROM system.tenants WHERE active = true AND name IS NOT NULL`)
tenantNames, err := s.server.serverController.getExpectedRunningTenants(ctx, s.internalExecutor)
if err != nil {
return nil, err
}
defer func() { _ = rowIter.Close() }()

var tenantNames []roachpb.TenantName
var hasNext bool
for hasNext, err = rowIter.Next(ctx); hasNext && err == nil; hasNext, err = rowIter.Next(ctx) {
row := rowIter.Cur()
tenantName := tree.MustBeDString(row[0])
tenantNames = append(tenantNames, roachpb.TenantName(tenantName))
}

var tenantList []*serverpb.Tenant
tenantList := make([]*serverpb.Tenant, 0, len(tenantNames))
for _, tenantName := range tenantNames {
server, err := s.server.serverController.getOrCreateServer(ctx, tenantName)
server, err := s.server.serverController.getServer(ctx, tenantName)
if err != nil {
log.Errorf(ctx, "unable to get or create a tenant server: %v", err)
continue
if errors.Is(err, errNoTenantServerRunning) {
// The service for this tenant is not started yet. This is not
// an error - the services are started asynchronously. The
// client can try again later.
continue
}
return nil, err
}
tenantID := server.getTenantID()
tenantList = append(tenantList, &serverpb.Tenant{
Expand Down
9 changes: 8 additions & 1 deletion pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1033,7 +1033,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
// https://github.com/cockroachdb/cockroach/issues/84585 is
// implemented.
func(ctx context.Context, name roachpb.TenantName) error {
d, err := sc.getOrCreateServer(ctx, name)
d, err := sc.getServer(ctx, name)
if err != nil {
return err
}
Expand Down Expand Up @@ -1897,6 +1897,13 @@ func (s *Server) PreStart(ctx context.Context) error {
s.stopper,
)

// Let the server controller start watching tenant service mode changes.
if err := s.serverController.start(workersCtx,
s.node.execCfg.InternalDB.Executor(),
); err != nil {
return errors.Wrap(err, "failed to start the server controller")
}

log.Event(ctx, "server initialized")

// Begin recording time series data collected by the status monitor.
Expand Down
Loading

0 comments on commit 67c573c

Please sign in to comment.