Skip to content

Commit

Permalink
server: add ./cockroach mt start-sql
Browse files Browse the repository at this point in the history
This adds a CLI command to start a SQL tenant in a standalone process.

The tenant currently communicates with the KV layer "as a node"; this
will only change with cockroachdb#47898. As is, the tenant has full access to the
KV layer and so a few things may work that shouldn't as a result of
that.

Additionally, the tenant runs a Gossip client as a temporary stopgap
until we have removed the remaining dependencies on it (cockroachdb#49693 has
the details on what those are).

Apart from that, though, this is the real thing and can be used to
start setting up end-to-end testing of ORMs as well as the deploy-
ments.

A word on the choice of the `mt` command: `mt` is an abbreviation for
`multi-tenant` which was considered too long; just `tenant` was
considered misleading - there will be multiple additional subcommands
housing the other services required for running the whole infrastructure
including certs generation, the KV auth broker server, and the SQL
proxy. Should a lively bikeshed around naming break out we can rename
the commands later when consensus has been reached.

For those curious to try it out the following will be handy:

```bash
rm -rf ~/.cockroach-certs cockroach-data &&
killall -9 cockroach && \
export COCKROACH_CA_KEY=$HOME/.cockroach-certs/ca.key && \
./cockroach cert create-ca && \
./cockroach cert create-client root && \
./cockroach cert create-node 127.0.0.1 && \
./cockroach start --host 127.0.0.1 --background && \
./cockroach sql --host 127.0.0.1 -e 'select crdb_internal.create_tenant(123);'
./cockroach mt start-sql --tenant-id 123 --kv-addrs 127.0.0.1:26257 --sql-addr :5432 &
sleep 1 && \
./cockroach sql --host 127.0.0.1 --port 5432
```

This roughly matches what the newly introduced `acceptance/multitenant`
roachtest does as well.

Release note: None
  • Loading branch information
tbg committed Jun 5, 2020
1 parent fd1bccf commit 0dbe16f
Show file tree
Hide file tree
Showing 8 changed files with 305 additions and 24 deletions.
26 changes: 26 additions & 0 deletions pkg/cli/cli_mt.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package cli

import "github.com/spf13/cobra"

func init() {
cockroachCmd.AddCommand(mtCmd)
mtCmd.AddCommand(mtStartSQLCmd)
}

// mtCmd is the base command for functionality related to multi-tenancy.
var mtCmd = &cobra.Command{
Use: "mt [command]",
Short: "commands related to multi-tenancy",
RunE: usageAndErr,
Hidden: true,
}
89 changes: 89 additions & 0 deletions pkg/cli/cli_mt_start_sql.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package cli

import (
"context"
"os"
"os/signal"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/errors"
"github.com/spf13/cobra"
)

var mtStartSQLCmd = &cobra.Command{
Use: "start-sql",
Short: "start a standalone SQL server",
Args: cobra.NoArgs,
RunE: MaybeDecorateGRPCError(runStartSQL),
}

func init() {
mtCmd.AddCommand(
mtStartSQLCmd,
)
}

func runStartSQL(cmd *cobra.Command, args []string) error {
ctx := context.Background()
const clusterName = ""
stopper := stop.NewStopper()
defer stopper.Stop(ctx)

st := serverCfg.BaseConfig.Settings

// TODO(tbg): this has to be passed in. See the upgrade strategy in:
// https://github.com/cockroachdb/cockroach/issues/47919
if err := clusterversion.Initialize(ctx, st.Version.BinaryVersion(), &st.SV); err != nil {
return err
}

tempStorageMaxSizeBytes := int64(base.DefaultInMemTempStorageMaxSizeBytes)
if err := diskTempStorageSizeValue.Resolve(
&tempStorageMaxSizeBytes, memoryPercentResolver,
); err != nil {
return err
}

serverCfg.SQLConfig.TempStorageConfig = base.TempStorageConfigFromEnv(
ctx,
st,
base.StoreSpec{InMemory: true},
"", // parentDir
tempStorageMaxSizeBytes,
)

addr, err := server.StartTenant(
ctx,
stopper,
clusterName,
serverCfg.BaseConfig,
serverCfg.SQLConfig,
)
if err != nil {
return err
}
log.Infof(ctx, "SQL server for tenant %s listening at %s", serverCfg.SQLConfig.TenantID, addr)

// TODO(tbg): make the other goodies in `./cockroach start` reusable, such as
// logging to files, periodic memory output, heap and goroutine dumps, debug
// server, graceful drain. Then use them here.

ch := make(chan os.Signal)
signal.Notify(ch, drainSignals...)
sig := <-ch
return errors.Newf("received signal %v", sig)
}
26 changes: 26 additions & 0 deletions pkg/cli/cliflags/flags_mt.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package cliflags

// Flags specific to multi-tenancy commands.
var (
TenantID = FlagInfo{
Name: "tenant-id",
EnvVar: "COCKROACH_TENANT_ID",
Description: `The tenant ID under which to start the SQL server.`,
}

KVAddrs = FlagInfo{
Name: "kv-addrs",
Shorthand: "",
Description: `A comma-separated list of KV endpoints (load balancers allowed).`,
}
)
89 changes: 73 additions & 16 deletions pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,13 @@ import (
"net"
"path/filepath"
"regexp"
"strconv"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/cli/cliflags"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
Expand Down Expand Up @@ -107,6 +109,13 @@ func IntFlag(f *pflag.FlagSet, valPtr *int, flagInfo cliflags.FlagInfo, defaultV
registerEnvVarDefault(f, flagInfo)
}

// UInt64Flag creates a uint64 flag and registers it with the FlagSet.
func Uint64Flag(f *pflag.FlagSet, valPtr *uint64, flagInfo cliflags.FlagInfo, defaultVal uint64) {
f.Uint64VarP(valPtr, flagInfo.Name, flagInfo.Shorthand, defaultVal, flagInfo.Usage())

registerEnvVarDefault(f, flagInfo)
}

// BoolFlag creates a bool flag and registers it with the FlagSet.
func BoolFlag(f *pflag.FlagSet, valPtr *bool, flagInfo cliflags.FlagInfo, defaultVal bool) {
f.BoolVarP(valPtr, flagInfo.Name, flagInfo.Shorthand, defaultVal, flagInfo.Usage())
Expand Down Expand Up @@ -246,8 +255,11 @@ func init() {
return setDefaultStderrVerbosity(cmd, log.Severity_WARNING)
})

// Add a pre-run command for `start` and `start-single-node`.
for _, cmd := range StartCmds {
// Add a pre-run command for `start` and `start-single-node`, as well as the
// multi-tenancy related commands that start long-running servers.
allStartCmds := append([]*cobra.Command(nil), StartCmds...)
allStartCmds = append(allStartCmds, mtStartSQLCmd)
for _, cmd := range allStartCmds {
AddPersistentPreRunE(cmd, func(cmd *cobra.Command, _ []string) error {
// Finalize the configuration of network and logging settings.
if err := extraServerFlagInit(cmd); err != nil {
Expand Down Expand Up @@ -704,6 +716,41 @@ func init() {
f := debugBallastCmd.Flags()
VarFlag(f, &debugCtx.ballastSize, cliflags.Size)
}

// Multi-tenancy commands.
{
f := mtStartSQLCmd.Flags()
VarFlag(f, &tenantIDWrapper{&serverCfg.SQLConfig.TenantID}, cliflags.TenantID)
// NB: serverInsecure populates baseCfg.{Insecure,SSLCertsDir} in this the following method
// (which is a PreRun for this command):
_ = extraServerFlagInit
BoolFlag(f, &startCtx.serverInsecure, cliflags.ServerInsecure, startCtx.serverInsecure)
StringFlag(f, &startCtx.serverSSLCertsDir, cliflags.ServerCertsDir, startCtx.serverSSLCertsDir)
// NB: this also gets PreRun treatment via extraServerFlagInit to populate BaseCfg.SQLAddr.
VarFlag(f, addrSetter{&serverSQLAddr, &serverSQLPort}, cliflags.ListenSQLAddr)

StringSlice(f, &serverCfg.SQLConfig.TenantKVAddrs, cliflags.KVAddrs, []string{"127.0.0.1:26257"})
}
}

type tenantIDWrapper struct {
tenID *roachpb.TenantID
}

func (w *tenantIDWrapper) String() string {
return w.tenID.String()
}
func (w *tenantIDWrapper) Set(s string) error {
tenID, err := strconv.ParseUint(s, 10, 64)
if err != nil {
return errors.Wrap(err, "invalid tenant ID")
}
*w.tenID = roachpb.MakeTenantID(tenID)
return nil
}

func (w *tenantIDWrapper) Type() string {
return "number"
}

// processEnvVarDefaults injects the current value of flag-related
Expand Down Expand Up @@ -786,14 +833,24 @@ func extraServerFlagInit(cmd *cobra.Command) error {

fs := flagSetForCmd(cmd)

// Construct the socket name, if requested.
if !fs.Lookup(cliflags.Socket.Name).Changed && fs.Lookup(cliflags.SocketDir.Name).Changed {
// If --socket (DEPRECATED) was set, then serverCfg.SocketFile is
// already set and we don't want to change it.
// However, if --socket-dir is set, then we'll use that.
// There are two cases:
// --socket-dir is set and is empty; in this case the user is telling us "disable the socket".
// is set and non-empty. Then it should be used as specified.
// Helper for .Changed that is nil-aware as not all of the `cmd`s may have
// all of the flags.
changed := func(set *pflag.FlagSet, name string) bool {
f := set.Lookup(name)
return f != nil && f.Changed
}

// Construct the socket name, if requested. The flags may not be defined for
// `cmd` so be cognizant of that.
//
// If --socket (DEPRECATED) was set, then serverCfg.SocketFile is
// already set and we don't want to change it.
// However, if --socket-dir is set, then we'll use that.
// There are two cases:
// 1. --socket-dir is set and is empty; in this case the user is telling us
// "disable the socket".
// 2. is set and non-empty. Then it should be used as specified.
if !changed(fs, cliflags.Socket.Name) && changed(fs, cliflags.SocketDir.Name) {
if serverSocketDir == "" {
serverCfg.SocketFile = ""
} else {
Expand All @@ -820,9 +877,9 @@ func extraServerFlagInit(cmd *cobra.Command) error {
serverCfg.SQLAddr = net.JoinHostPort(serverSQLAddr, serverSQLPort)
serverCfg.SplitListenSQL = fs.Lookup(cliflags.ListenSQLAddr.Name).Changed

// Fill in the defaults for --advertise-sql-addr.
advSpecified := fs.Lookup(cliflags.AdvertiseAddr.Name).Changed ||
fs.Lookup(cliflags.AdvertiseHost.Name).Changed
// Fill in the defaults for --advertise-sql-addr, if the flag exists on `cmd`.
advSpecified := changed(fs, cliflags.AdvertiseAddr.Name) ||
changed(fs, cliflags.AdvertiseHost.Name)
if serverSQLAdvertiseAddr == "" {
if advSpecified {
serverSQLAdvertiseAddr = serverAdvertiseAddr
Expand Down Expand Up @@ -851,9 +908,9 @@ func extraServerFlagInit(cmd *cobra.Command) error {
// Before we do so, we'll check whether the user explicitly
// specified something contradictory, and tell them that's no
// good.
if (fs.Lookup(cliflags.ListenHTTPAddr.Name).Changed ||
fs.Lookup(cliflags.ListenHTTPAddrAlias.Name).Changed) &&
(serverHTTPAddr != "" && serverHTTPAddr != "localhost") {
if changed(fs, cliflags.ListenHTTPAddr.Name) ||
changed(fs, cliflags.ListenHTTPAddrAlias.Name) &&
(serverHTTPAddr != "" && serverHTTPAddr != "localhost") {
return errors.WithHintf(
errors.Newf("--unencrypted-localhost-http is incompatible with --http-addr=%s:%s",
serverHTTPAddr, serverHTTPPort),
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/acceptance.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ func registerAcceptance(r *testRegistry) {
{name: "gossip/restart", fn: runGossipRestart},
{name: "gossip/restart-node-one", fn: runGossipRestartNodeOne},
{name: "gossip/locality-address", fn: runCheckLocalityIPAddress},
{name: "multitenant", fn: runAcceptanceMultitenant},
{name: "rapid-restart", fn: runRapidRestart},
{
name: "many-splits", fn: runManySplits,
Expand Down
74 changes: 74 additions & 0 deletions pkg/cmd/roachtest/multitenant.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package main

import (
"context"
gosql "database/sql"
"net/url"
"strings"
"time"

"github.com/stretchr/testify/require"
)

func runAcceptanceMultitenant(ctx context.Context, t *test, c *cluster) {
c.Put(ctx, cockroach, "./cockroach")
c.Start(ctx, t, c.All())

_, err := c.Conn(ctx, 1).Exec(`SELECT crdb_internal.create_tenant(123)`)
require.NoError(t, err)

kvAddrs := c.ExternalAddr(ctx, c.All())

errCh := make(chan error)
go func() {
errCh <- c.RunE(ctx, c.Node(1),
"./cockroach", "mt", "start-sql",
// TODO(tbg): make this test secure.
// "--certs-dir", "certs",
"--insecure",
"--tenant-id", "123",
"--kv-addrs", strings.Join(kvAddrs, ","),
"--sql-addr", "0.0.0.0:36257",
)
}()
u, err := url.Parse(c.ExternalPGUrl(ctx, c.Node(1))[0])
require.NoError(t, err)
u.Host = c.ExternalIP(ctx, c.Node(1))[0] + ":36257"
url := u.String()
c.l.Printf("sql server should be running at %s", url)

time.Sleep(time.Second)

select {
case err := <-errCh:
t.Fatal(err)
default:
}

db, err := gosql.Open("postgres", url)
if err != nil {
t.Fatal(err)
}
defer db.Close()
_, err = db.Exec(`CREATE TABLE foo (id INT PRIMARY KEY, v STRING)`)
require.NoError(t, err)

_, err = db.Exec(`INSERT INTO foo VALUES($1, $2)`, 1, "bar")
require.NoError(t, err)

var id int
var v string
require.NoError(t, db.QueryRow(`SELECT * FROM foo LIMIT 1`).Scan(&id, &v))
require.Equal(t, 1, id)
require.Equal(t, "bar", v)
}
5 changes: 5 additions & 0 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -327,6 +327,11 @@ type SQLConfig struct {

// QueryCacheSize is the memory size (in bytes) of the query plan cache.
QueryCacheSize int64

// TenantKVAddrs are the entry points to the KV layer.
//
// Only applies when the SQL server is deployed individually.
TenantKVAddrs []string
}

// MakeSQLConfig returns a SQLConfig with default values.
Expand Down
Loading

0 comments on commit 0dbe16f

Please sign in to comment.