Skip to content

Commit

Permalink
cli/start: use the same format for server details in multi-tenant
Browse files Browse the repository at this point in the history
This ensures that the server details are printed using the same format
as regular nodes for multi-tenant SQL servers.

Before:

```
SQL server for tenant 123 listening at 127.0.0.1:26258, http at 127.0.0.1:8081
```

After:
```
CockroachDB SQL server starting at 2021-12-10 12:11:57.684183707 +0000 UTC (took 0.1s)
build:               CCL v21.2.0-alpha.00000000-6604-g6f243c67e2-dirty @  (go1.17.4)
webui:               http://kenax:8081
sql:                 postgresql://root@kenax:26258/defaultdb?sslmode=disable
sql (JDBC):          jdbc:postgresql://kenax:26258/defaultdb?sslmode=disable&user=root
RPC client flags:    ./cockroach <client cmd> --host=kenax:26258 --insecure
temp dir:            /data/home/kena/src/go/src/github.com/cockroachdb/cockroach/mt-data/cockroach-temp981776782
external I/O path:   <disabled>
store[0]:            path=/data/home/kena/src/go/src/github.com/cockroachdb/cockroach/mt-data
storage engine:      pebble
clusterID:           6622219f-89b7-4d0d-919f-22db8fa83dfd
tenantID:            123
instanceID:          3
```

Release note: None
  • Loading branch information
knz committed Dec 10, 2021
1 parent 4fcec2d commit c631d86
Show file tree
Hide file tree
Showing 4 changed files with 134 additions and 83 deletions.
1 change: 1 addition & 0 deletions docs/generated/redact_safe.md
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ pkg/roachpb/metadata.go | `ReplicaID`
pkg/roachpb/metadata.go | `ReplicaType`
pkg/roachpb/metadata.go | `StoreID`
pkg/roachpb/method.go | `Method`
pkg/roachpb/tenant.go | `TenantID`
pkg/rpc/connection_class.go | `ConnectionClass`
pkg/sql/catalog/descpb/structured.go | `ColumnID`
pkg/sql/catalog/descpb/structured.go | `ConstraintType`
Expand Down
9 changes: 7 additions & 2 deletions pkg/cli/mt_start_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/sdnotify"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/spf13/cobra"
)
Expand Down Expand Up @@ -62,6 +63,8 @@ well unless it can be verified using a trusted root certificate store. That is,
}

func runStartSQL(cmd *cobra.Command, args []string) error {
tBegin := timeutil.Now()

// First things first: if the user wants background processing,
// relinquish the terminal ASAP by forking and exiting.
//
Expand Down Expand Up @@ -108,7 +111,7 @@ func runStartSQL(cmd *cobra.Command, args []string) error {

initGEOS(ctx)

sqlServer, addr, httpAddr, err := server.StartTenant(
sqlServer, _, _, err := server.StartTenant(
ctx,
stopper,
clusterName,
Expand Down Expand Up @@ -145,7 +148,9 @@ func runStartSQL(cmd *cobra.Command, args []string) error {
sqlServer.StartDiagnostics(ctx)
}

log.Infof(ctx, "SQL server for tenant %s listening at %s, http at %s", serverCfg.SQLConfig.TenantID, addr, httpAddr)
// Report the server identifiers and other server details
// in the same format as 'cockroach start'.
reportServerInfo(ctx, tBegin, &serverCfg, st, false /* isHostNode */, false /* initialStart */)

// TODO(tbg): make the other goodies in `./cockroach start` reusable, such as
// logging to files, periodic memory output, heap and goroutine dumps, debug
Expand Down
204 changes: 123 additions & 81 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -676,91 +676,15 @@ If problems persist, please see %s.`
return err
}

// Now inform the user that the server is running and tell the
// user about its run-time derived parameters.
var buf redact.StringBuilder
info := build.GetInfo()
buf.Printf("CockroachDB node starting at %s (took %0.1fs)\n", timeutil.Now(), timeutil.Since(tBegin).Seconds())
buf.Printf("build:\t%s %s @ %s (%s)\n",
redact.Safe(info.Distribution), redact.Safe(info.Tag), redact.Safe(info.Time), redact.Safe(info.GoVersion))
buf.Printf("webui:\t%s\n", serverCfg.AdminURL())

// (Re-)compute the client connection URL. We cannot do this
// earlier (e.g. above, in the runStart function) because
// at this time the address and port have not been resolved yet.
sCtx := rpc.MakeSecurityContext(serverCfg.Config, security.ClusterTLSSettings(serverCfg.Settings), roachpb.SystemTenantID)
pgURL, err := sCtx.PGURL(url.User(security.RootUser))
if err != nil {
log.Ops.Errorf(ctx, "failed computing the URL: %v", err)
return err
}
buf.Printf("sql:\t%s\n", pgURL.ToPQ())
buf.Printf("sql (JDBC):\t%s\n", pgURL.ToJDBC())

buf.Printf("RPC client flags:\t%s\n", clientFlagsRPC())
if len(serverCfg.SocketFile) != 0 {
buf.Printf("socket:\t%s\n", serverCfg.SocketFile)
}
logNum := 1
_ = cliCtx.logConfig.IterateDirectories(func(d string) error {
if logNum == 1 {
// Backward-compatibility.
buf.Printf("logs:\t%s\n", d)
} else {
buf.Printf("logs[%d]:\t%s\n", logNum, d)
}
logNum++
return nil
})
if serverCfg.Attrs != "" {
buf.Printf("attrs:\t%s\n", serverCfg.Attrs)
}
if len(serverCfg.Locality.Tiers) > 0 {
buf.Printf("locality:\t%s\n", serverCfg.Locality)
}
if s.TempDir() != "" {
buf.Printf("temp dir:\t%s\n", s.TempDir())
}
if ext := s.ClusterSettings().ExternalIODir; ext != "" {
buf.Printf("external I/O path: \t%s\n", ext)
} else {
buf.Printf("external I/O path: \t<disabled>\n")
}
for i, spec := range serverCfg.Stores.Specs {
buf.Printf("store[%d]:\t%s\n", i, spec)
}
buf.Printf("storage engine: \t%s\n", &serverCfg.StorageEngine)
// Remember the server identifiers for logging.
// TODO(knz): Remove this.
nodeID := s.NodeID()
if initialStart {
if nodeID == kvserver.FirstNodeID {
buf.Printf("status:\tinitialized new cluster\n")
} else {
buf.Printf("status:\tinitialized new node, joined pre-existing cluster\n")
}
} else {
buf.Printf("status:\trestarted pre-existing node\n")
}

if baseCfg.ClusterName != "" {
buf.Printf("cluster name:\t%s\n", baseCfg.ClusterName)
}

// Remember the cluster ID for log file rotation.
clusterID := s.ClusterID().String()
log.SetNodeIDs(clusterID, int32(nodeID))
buf.Printf("clusterID:\t%s\n", clusterID)
buf.Printf("nodeID:\t%d\n", nodeID)

// Collect the formatted string and show it to the user.
msg, err := expandTabsInRedactableBytes(buf.RedactableBytes())
if err != nil {
return err
}
msgS := msg.ToString()
log.Ops.Infof(ctx, "node startup completed:\n%s", msgS)
if !startCtx.inBackground && !log.LoggingToStderr(severity.INFO) {
fmt.Print(msgS.StripMarkers())
}
// Now inform the user that the server is running and tell the
// user about its run-time derived parameters.
reportServerInfo(ctx, tBegin, &serverCfg, s.ClusterSettings(), true /* isHostNode */, initialStart)

return nil
}(); err != nil {
Expand Down Expand Up @@ -958,6 +882,124 @@ If problems persist, please see %s.`
return returnErr
}

// reportServerInfo prints out the server version and network details
// in a standardized format.
func reportServerInfo(
ctx context.Context,
startTime time.Time,
serverCfg *server.Config,
st *cluster.Settings,
isHostNode, initialStart bool,
) error {
srvS := redact.SafeString("SQL server")
if isHostNode {
srvS = "node"
}

var buf redact.StringBuilder
info := build.GetInfo()
buf.Printf("CockroachDB %s starting at %s (took %0.1fs)\n", srvS, timeutil.Now(), timeutil.Since(startTime).Seconds())
buf.Printf("build:\t%s %s @ %s (%s)\n",
redact.Safe(info.Distribution), redact.Safe(info.Tag), redact.Safe(info.Time), redact.Safe(info.GoVersion))
buf.Printf("webui:\t%s\n", serverCfg.AdminURL())

// (Re-)compute the client connection URL. We cannot do this
// earlier (e.g. above, in the runStart function) because
// at this time the address and port have not been resolved yet.
sCtx := rpc.MakeSecurityContext(serverCfg.Config, security.ClusterTLSSettings(serverCfg.Settings), roachpb.SystemTenantID)
pgURL, err := sCtx.PGURL(url.User(security.RootUser))
if err != nil {
log.Ops.Errorf(ctx, "failed computing the URL: %v", err)
return err
}
buf.Printf("sql:\t%s\n", pgURL.ToPQ())
buf.Printf("sql (JDBC):\t%s\n", pgURL.ToJDBC())

buf.Printf("RPC client flags:\t%s\n", clientFlagsRPC())
if len(serverCfg.SocketFile) != 0 {
buf.Printf("socket:\t%s\n", serverCfg.SocketFile)
}
logNum := 1
_ = cliCtx.logConfig.IterateDirectories(func(d string) error {
if logNum == 1 {
// Backward-compatibility.
buf.Printf("logs:\t%s\n", d)
} else {
buf.Printf("logs[%d]:\t%s\n", logNum, d)
}
logNum++
return nil
})
if serverCfg.Attrs != "" {
buf.Printf("attrs:\t%s\n", serverCfg.Attrs)
}
if len(serverCfg.Locality.Tiers) > 0 {
buf.Printf("locality:\t%s\n", serverCfg.Locality)
}
if tmpDir := serverCfg.SQLConfig.TempStorageConfig.Path; tmpDir != "" {
buf.Printf("temp dir:\t%s\n", tmpDir)
}
if ext := st.ExternalIODir; ext != "" {
buf.Printf("external I/O path: \t%s\n", ext)
} else {
buf.Printf("external I/O path: \t<disabled>\n")
}
for i, spec := range serverCfg.Stores.Specs {
buf.Printf("store[%d]:\t%s\n", i, spec)
}
buf.Printf("storage engine: \t%s\n", &serverCfg.StorageEngine)

// Print the commong server identifiers.
if baseCfg.ClusterName != "" {
buf.Printf("cluster name:\t%s\n", baseCfg.ClusterName)
}
clusterID := serverCfg.BaseConfig.ClusterIDContainer.Get().String()
buf.Printf("clusterID:\t%s\n", clusterID)

nodeID := serverCfg.BaseConfig.IDContainer.Get()
if isHostNode {
if initialStart {
if nodeID == kvserver.FirstNodeID {
buf.Printf("status:\tinitialized new cluster\n")
} else {
buf.Printf("status:\tinitialized new node, joined pre-existing cluster\n")
}
} else {
buf.Printf("status:\trestarted pre-existing node\n")
}
// Report the server identifiers.
buf.Printf("nodeID:\t%d\n", nodeID)
} else {
// Report the SQL server identifiers.
buf.Printf("tenantID:\t%s\n", serverCfg.SQLConfig.TenantID)
buf.Printf("instanceID:\t%d\n", nodeID)

if kvAddrs := serverCfg.SQLConfig.TenantKVAddrs; len(kvAddrs) > 0 {
// Report which KV servers are connected.
buf.Printf("KV addresses:\t")
comma := redact.SafeString("")
for _, addr := range serverCfg.SQLConfig.TenantKVAddrs {
buf.Printf("%s%s", comma, addr)
comma = ", "
}
buf.Printf("\n")
}
}

// Collect the formatted string and show it to the user.
msg, err := expandTabsInRedactableBytes(buf.RedactableBytes())
if err != nil {
return err
}
msgS := msg.ToString()
log.Ops.Infof(ctx, "%s startup completed:\n%s", srvS, msgS)
if !startCtx.inBackground && !log.LoggingToStderr(severity.INFO) {
fmt.Print(msgS.StripMarkers())
}

return nil
}

// expandTabsInRedactableBytes expands tabs in the redactable byte
// slice, so that columns are aligned. The correctness of this
// function depends on the assumption that the `tabwriter` does not
Expand Down
3 changes: 3 additions & 0 deletions pkg/roachpb/tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,9 @@ func (t TenantID) String() string {
}
}

// SafeValue implements the redact.SafeValue interface.
func (_ TenantID) SafeValue() {}

// Protects against zero value.
func checkValid(id uint64) {
if id == 0 {
Expand Down

0 comments on commit c631d86

Please sign in to comment.