Skip to content

Commit

Permalink
roachprod: reference virtual clusters by name instead of ID
Browse files Browse the repository at this point in the history
This changes how virtual clusters are referenced in roachprod commands
(`start-sql` and `stop-sql`). Specifically, where before a
`cluster-id` was requested, we now request a virtual cluster
name. This makes the API more user-friendly and removes the need for
the caller to keep track of virtual cluster IDs.

With this change we now remove the "default virtual cluster name"
logic, which is no longer necessary.

Epic: none

Release note: None
  • Loading branch information
renatolabs committed Oct 18, 2023
1 parent 325b3a0 commit 42d27f7
Show file tree
Hide file tree
Showing 5 changed files with 88 additions and 64 deletions.
8 changes: 1 addition & 7 deletions pkg/cmd/roachprod/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@ var (
secure = false
virtualClusterName string
sqlInstance int
virtualClusterID int
extraSSHOptions = ""
nodeEnv []string
tag string
Expand Down Expand Up @@ -214,8 +213,6 @@ Default is "RECURRING '*/15 * * * *' FULL BACKUP '@hourly' WITH SCHEDULE OPTIONS

startInstanceCmd.Flags().StringVarP(&storageCluster, "storage-cluster", "S", "", "storage cluster")
_ = startInstanceCmd.MarkFlagRequired("storage-cluster")
startInstanceCmd.Flags().IntVarP(&startOpts.VirtualClusterID,
"cluster-id", "i", startOpts.VirtualClusterID, "internal ID for the virtual cluster")
startInstanceCmd.Flags().IntVar(&startOpts.SQLInstance,
"sql-instance", 0, "specific SQL/HTTP instance to connect to (this is a roachprod abstraction for separate-process deployments distinct from the internal instance ID)")
startInstanceCmd.Flags().StringVar(&externalProcessNodes, "external-cluster", externalProcessNodes, "start service in external mode, as a separate process in the given nodes")
Expand All @@ -227,9 +224,6 @@ Default is "RECURRING '*/15 * * * *' FULL BACKUP '@hourly' WITH SCHEDULE OPTIONS
stopProcessesCmd.Flags().IntVar(&maxWait, "max-wait", maxWait, "approx number of seconds to wait for processes to exit")
}

stopInstanceCmd.Flags().IntVarP(&virtualClusterID, "cluster-id", "t", virtualClusterID, "internal ID for the virtual cluster")
stopInstanceCmd.Flags().IntVar(&sqlInstance, "sql-instance", 0, "specific SQL/HTTP instance to stop")

syncCmd.Flags().BoolVar(&listOpts.IncludeVolumes, "include-volumes", false, "Include volumes when syncing")

wipeCmd.Flags().BoolVar(&wipePreserveCerts, "preserve-certs", false, "do not wipe certificates")
Expand Down Expand Up @@ -367,7 +361,7 @@ Default is "RECURRING '*/15 * * * *' FULL BACKUP '@hourly' WITH SCHEDULE OPTIONS
cmd.Flags().BoolVar(&secure,
"secure", false, "use a secure cluster")
}
for _, cmd := range []*cobra.Command{pgurlCmd, sqlCmd, adminurlCmd} {
for _, cmd := range []*cobra.Command{pgurlCmd, sqlCmd, adminurlCmd, stopInstanceCmd} {
cmd.Flags().StringVar(&virtualClusterName,
"cluster", "", "specific virtual cluster to connect to")
cmd.Flags().IntVar(&sqlInstance,
Expand Down
19 changes: 9 additions & 10 deletions pkg/cmd/roachprod/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -526,7 +526,7 @@ SIGHUP), unless you also configure --max-wait.
}

var startInstanceCmd = &cobra.Command{
Use: "start-sql --storage-cluster <storage-cluster> [--external-cluster <virtual-cluster-nodes]",
Use: "start-sql <name> --storage-cluster <storage-cluster> [--external-cluster <virtual-cluster-nodes]",
Short: "start the SQL/HTTP service for a virtual cluster as a separate process",
Long: `Start SQL/HTTP instances for a virtual cluster as separate processes.
Expand All @@ -536,8 +536,6 @@ will create the virtual cluster on the storage cluster if it does not
exist already. If creating multiple virtual clusters on the same
node, the --sql-instance flag must be passed to differentiate them.
The --cluster-id flag can be used to specify the tenant ID; it defaults to 2.
The instance is started in shared process (in memory) mode by
default. To start an external process instance, pass the
--external-cluster flag indicating where the SQL server processes
Expand All @@ -560,7 +558,7 @@ The --args and --env flags can be used to pass arbitrary command line flags and
environment variables to the cockroach process.
` + tagHelp + `
`,
Args: cobra.NoArgs,
Args: cobra.ExactArgs(1),
Run: wrap(func(cmd *cobra.Command, args []string) error {
clusterSettingsOpts := []install.ClusterSettingOption{
install.TagOption(tag),
Expand All @@ -586,13 +584,14 @@ environment variables to the cockroach process.
startOpts.Target = install.StartServiceForVirtualCluster
}

startOpts.VirtualClusterName = args[0]
return roachprod.StartServiceForVirtualCluster(context.Background(),
config.Logger, externalProcessNodes, storageCluster, startOpts, clusterSettingsOpts...)
}),
}

var stopInstanceCmd = &cobra.Command{
Use: "stop-sql <cluster> --cluster-id <id> --sql-instance <instance> [--sig] [--wait]",
Use: "stop-sql <cluster> --cluster <name> --sql-instance <instance> [--sig] [--wait]",
Short: "stop sql instances on a cluster",
Long: `Stop sql instances on a cluster.
Expand All @@ -614,11 +613,11 @@ non-terminating signal (e.g. SIGHUP), unless you also configure --max-wait.
wait = true
}
stopOpts := roachprod.StopOpts{
Wait: wait,
MaxWait: maxWait,
Sig: sig,
VirtualClusterID: virtualClusterID,
SQLInstance: sqlInstance,
Wait: wait,
MaxWait: maxWait,
Sig: sig,
VirtualClusterName: virtualClusterName,
SQLInstance: sqlInstance,
}
clusterName := args[0]
return roachprod.StopServiceForVirtualCluster(context.Background(), config.Logger, clusterName, stopOpts)
Expand Down
12 changes: 8 additions & 4 deletions pkg/roachprod/install/cluster_synced.go
Original file line number Diff line number Diff line change
Expand Up @@ -1693,7 +1693,7 @@ tar cvf %[3]s certs
// DistributeTenantCerts will generate and distribute certificates to all of the
// nodes, using the host cluster to generate tenant certificates.
func (c *SyncedCluster) DistributeTenantCerts(
ctx context.Context, l *logger.Logger, hostCluster *SyncedCluster, tenantID int,
ctx context.Context, l *logger.Logger, hostCluster *SyncedCluster, virtualClusterID int,
) error {
if hostCluster.checkForTenantCertificates(ctx, l) {
return nil
Expand All @@ -1708,7 +1708,7 @@ func (c *SyncedCluster) DistributeTenantCerts(
return err
}

if err := hostCluster.createTenantCertBundle(ctx, l, tenantCertsTarName, tenantID, nodeNames); err != nil {
if err := hostCluster.createTenantCertBundle(ctx, l, tenantCertsTarName, virtualClusterID, nodeNames); err != nil {
return err
}

Expand All @@ -1727,7 +1727,11 @@ func (c *SyncedCluster) DistributeTenantCerts(
// This function assumes it is running on a host cluster node that already has
// had the main cert bundle created.
func (c *SyncedCluster) createTenantCertBundle(
ctx context.Context, l *logger.Logger, bundleName string, tenantID int, nodeNames []string,
ctx context.Context,
l *logger.Logger,
bundleName string,
virtualClusterID int,
nodeNames []string,
) error {
display := fmt.Sprintf("%s: initializing tenant certs", c.Name)
return c.Parallel(ctx, l, c.Nodes[0:1], func(ctx context.Context, node Node) (*RunResultDetails, error) {
Expand Down Expand Up @@ -1757,7 +1761,7 @@ tar cvf %[4]s $CERT_DIR
`,
cockroachNodeBinary(c, node),
strings.Join(nodeNames, " "),
tenantID,
virtualClusterID,
bundleName,
)

Expand Down
98 changes: 70 additions & 28 deletions pkg/roachprod/install/cockroach.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package install
import (
"context"
_ "embed" // required for go:embed
"encoding/json"
"fmt"
"net/url"
"os"
Expand Down Expand Up @@ -355,13 +356,20 @@ func (c *SyncedCluster) Start(ctx context.Context, l *logger.Logger, startOpts S
}

if startOpts.IsVirtualCluster() {
if err := c.createVirtualClusterMetadata(ctx, l, startOpts); err != nil {
startOpts.VirtualClusterID, err = c.upsertVirtualClusterMetadata(ctx, l, startOpts)
if err != nil {
return err
}
}

if err := c.distributeTenantCerts(ctx, l, startOpts.KVCluster, startOpts.VirtualClusterID); err != nil {
return err
l.Printf("virtual cluster ID: %d", startOpts.VirtualClusterID)

if err := c.distributeTenantCerts(ctx, l, startOpts.KVCluster, startOpts.VirtualClusterID); err != nil {
return err
}
} else {
if err := c.distributeCerts(ctx, l); err != nil {
return err
}
}

nodes := c.TargetNodes()
Expand Down Expand Up @@ -1078,15 +1086,16 @@ func (c *SyncedCluster) distributeCerts(ctx context.Context, l *logger.Logger) e
return nil
}

// createVirtualClusterMetadata creates the virtual cluster, if
// necessary. We only need to run the statements in this function
// against a single connection to the storage cluster.
func (c *SyncedCluster) createVirtualClusterMetadata(
// upsertVirtualClusterMetadata creates the virtual cluster metadata,
// if necessary, and marks the service as started internally. We only
// need to run the statements in this function against a single
// connection to the storage cluster.
func (c *SyncedCluster) upsertVirtualClusterMetadata(
ctx context.Context, l *logger.Logger, startOpts StartOpts,
) error {
) (int, error) {
runSQL := func(stmt string) (string, error) {
results, err := startOpts.KVCluster.ExecSQL(ctx, l, startOpts.KVCluster.Nodes[:1], "", 0, []string{
"--format", "raw", "-e", stmt,
"--format", "json", "-e", stmt,
})
if err != nil {
return "", err
Expand All @@ -1098,42 +1107,75 @@ func (c *SyncedCluster) createVirtualClusterMetadata(
return results[0].CombinedOut, nil
}

tenantExistsQuery := fmt.Sprintf(
"SELECT 1 FROM system.tenants WHERE name = '%s'", startOpts.VirtualClusterName,
)
virtualClusterIDByName := func(name string) (int, error) {
type tenantRow struct {
ID string `json:"id"`
}

existsOut, err := runSQL(tenantExistsQuery)
if err != nil {
return err
query := fmt.Sprintf(
"SELECT id FROM system.tenants WHERE name = '%s'", startOpts.VirtualClusterName,
)

existsOut, err := runSQL(query)
if err != nil {
return -1, err
}

var tenants []tenantRow
if err := json.Unmarshal([]byte(existsOut), &tenants); err != nil {
return -1, fmt.Errorf("failed to unmarshal system.tenants output: %w", err)
}

if len(tenants) == 0 {
return -1, nil
}

n, err := strconv.Atoi(tenants[0].ID)
if err != nil {
return -1, fmt.Errorf("failed to parse virtual cluster ID: %w", err)
}

return n, nil
}

// Check if virtual cluster already exists, in which case there is
// nothing to do.
if !strings.Contains(existsOut, "0 rows") {
return nil
virtualClusterID, err := virtualClusterIDByName(startOpts.VirtualClusterName)
if err != nil {
return -1, err
}

l.Printf("Creating virtual cluster metadata")
l.Printf("Starting virtual cluster")
serviceMode := "SHARED"
if startOpts.Target == StartServiceForVirtualCluster {
serviceMode = "EXTERNAL"
}

createTenantStmts := []string{
fmt.Sprintf("CREATE TENANT '%s'", startOpts.VirtualClusterName),
fmt.Sprintf("ALTER TENANT '%s' START SERVICE %s", startOpts.VirtualClusterName, serviceMode),
var virtualClusterStmts []string
if virtualClusterID <= 0 {
// If the virtual cluster metadata does not exist yet, create it.
virtualClusterStmts = append(virtualClusterStmts,
fmt.Sprintf("CREATE TENANT '%s'", startOpts.VirtualClusterName),
)
}

virtualClusterStmts = append(virtualClusterStmts, fmt.Sprintf(
"ALTER TENANT '%s' START SERVICE %s",
startOpts.VirtualClusterName, serviceMode),
)

_, err = runSQL(strings.Join(virtualClusterStmts, "; "))
if err != nil {
return -1, err
}

_, err = runSQL(strings.Join(createTenantStmts, "; "))
return err
return virtualClusterIDByName(startOpts.VirtualClusterName)
}

// distributeCerts distributes certs if it's a secure cluster.
func (c *SyncedCluster) distributeTenantCerts(
ctx context.Context, l *logger.Logger, storageCluster *SyncedCluster, tenantID int,
ctx context.Context, l *logger.Logger, storageCluster *SyncedCluster, virtualClusterID int,
) error {
if c.Secure {
return c.DistributeTenantCerts(ctx, l, storageCluster, tenantID)
return c.DistributeTenantCerts(ctx, l, storageCluster, virtualClusterID)
}
return nil
}
Expand Down
15 changes: 0 additions & 15 deletions pkg/roachprod/multitenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/errors"
)

// StartServiceForVirtualCluster starts SQL/HTTP instances for a
Expand Down Expand Up @@ -67,11 +66,6 @@ func StartServiceForVirtualCluster(
startCluster = ec
}

if startOpts.VirtualClusterID < 2 {
return errors.Errorf("invalid virtual cluster ID %d (must be 2 or higher)", startOpts.VirtualClusterID)
}
startOpts.VirtualClusterName = defaultVirtualClusterName(startOpts.VirtualClusterID)

if startOpts.Target == install.StartServiceForVirtualCluster {
l.Printf("Starting SQL/HTTP instances for the virtual cluster")
}
Expand All @@ -87,15 +81,6 @@ func StopServiceForVirtualCluster(
return err
}

stopOpts.VirtualClusterName = defaultVirtualClusterName(stopOpts.VirtualClusterID)
label := install.VirtualClusterLabel(stopOpts.VirtualClusterName, stopOpts.SQLInstance)
return c.Stop(ctx, l, stopOpts.Sig, stopOpts.Wait, stopOpts.MaxWait, label)
}

// defaultVirtualClusterName returns the virtual cluster name used for
// the virtual cluster with ID given.
//
// TODO(herko): Allow users to pass in a virtual cluster name.
func defaultVirtualClusterName(virtualClusterID int) string {
return fmt.Sprintf("virtual-cluster-%d", virtualClusterID)
}

0 comments on commit 42d27f7

Please sign in to comment.