From 2e91ca3dd913cd60feb71f84694b82c620020b63 Mon Sep 17 00:00:00 2001
From: Bilal Akhtar
Date: Wed, 10 Mar 2021 16:41:27 -0500
Subject: [PATCH 1/6] server: use certificate manager in generateJoinToken
Previously, the generation of join tokens bypassed the
certificate manager when loading CA certificates. This change
updates that code path and associated tests to use the certificate
manager, to make testing easier.
Release note: None.
---
pkg/server/join_token.go | 10 ++--------
pkg/server/join_token_test.go | 14 ++++----------
pkg/server/status.go | 6 +++++-
3 files changed, 11 insertions(+), 19 deletions(-)
diff --git a/pkg/server/join_token.go b/pkg/server/join_token.go
index 36b639874aa7..e7fcfae271f7 100644
--- a/pkg/server/join_token.go
+++ b/pkg/server/join_token.go
@@ -41,19 +41,13 @@ type joinToken struct {
}
// Generates a new join token, and signs it with the CA cert in sslCertsDir.
-func generateJoinToken(sslCertsDir string) (joinToken, error) {
+func generateJoinToken(cm *security.CertificateManager) (joinToken, error) {
var jt joinToken
jt.tokenID = uuid.MakeV4()
r := rand.New(rand.NewSource(timeutil.Now().UnixNano()))
jt.sharedSecret = randutil.RandBytes(r, joinTokenSecretLen)
-
- certLocator := security.MakeCertsLocator(sslCertsDir)
- caCert, err := loadCertificateFile(certLocator.CACertPath())
- if err != nil {
- return joinToken{}, errors.Wrap(err, "could not open CA cert")
- }
- jt.sign(caCert)
+ jt.sign(cm.CACert().FileContents)
return jt, nil
}
diff --git a/pkg/server/join_token_test.go b/pkg/server/join_token_test.go
index fbfe9269f436..d00ab2b23faf 100644
--- a/pkg/server/join_token_test.go
+++ b/pkg/server/join_token_test.go
@@ -11,13 +11,10 @@
package server
import (
- "io/ioutil"
"math/rand"
- "path"
"testing"
"github.com/cockroachdb/cockroach/pkg/security"
- "github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
@@ -54,14 +51,11 @@ func TestGenerateJoinToken(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
- tempDir, cleanup := testutils.TempDir(t)
- defer cleanup()
- caCertFile := path.Join(tempDir, security.CACertFilename())
- caCert := []byte("foobar")
- require.NoError(t, ioutil.WriteFile(caCertFile, caCert, 0600))
+ cm, err := security.NewCertificateManager(security.EmbeddedCertsDir, security.CommandTLSSettings{})
+ require.NoError(t, err)
- token, err := generateJoinToken(tempDir)
+ token, err := generateJoinToken(cm)
require.NoError(t, err)
require.NotEmpty(t, token)
- require.True(t, token.verifySignature(caCert))
+ require.True(t, token.verifySignature(cm.CACert().FileContents))
}
diff --git a/pkg/server/status.go b/pkg/server/status.go
index e02aa5022794..d548bda9fad5 100644
--- a/pkg/server/status.go
+++ b/pkg/server/status.go
@@ -2532,7 +2532,11 @@ func (s *statusServer) GenerateJoinToken(ctx context.Context) (string, error) {
return "", errors.New("join token generation disabled")
}
- jt, err := generateJoinToken(s.cfg.SSLCertsDir)
+ cm, err := s.rpcCtx.GetCertificateManager()
+ if err != nil {
+ return "", errors.Wrap(err, "error when getting certificate manager")
+ }
+ jt, err := generateJoinToken(cm)
if err != nil {
return "", errors.Wrap(err, "error when generating join token")
}
From 1edb0884ff9904b799d7b73c64b2252d7d3a4691 Mon Sep 17 00:00:00 2001
From: Bilal Akhtar
Date: Fri, 5 Mar 2021 17:03:49 -0500
Subject: [PATCH 2/6] sql: Add crdb_internal.create_join_token sql builtin
function
Adds a create_join_token builtin function for use in TLS auto-joins.
This function, when run on a self-hosted single-tenant
Cockroach node, creates and returns a new join token. This
join token can then be copy-pasted to new nodes and used
to give them the set of certificates for secure auto TLS
initialization.
See RFC #51991. Part of #60632.
Release note: None.
---
docs/generated/sql/functions.md | 2 ++
pkg/sql/join_token.go | 7 +++----
pkg/sql/sem/builtins/builtins.go | 14 ++++++++++++++
3 files changed, 19 insertions(+), 4 deletions(-)
diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md
index 551675221772..1ee43c589b4f 100644
--- a/docs/generated/sql/functions.md
+++ b/docs/generated/sql/functions.md
@@ -2659,6 +2659,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)
crdb_internal.force_assertion_error(msg: string) → int | This function is used only by CockroachDB’s developers for testing purposes.
diff --git a/pkg/sql/join_token.go b/pkg/sql/join_token.go
index 8bb4d8b8c8c5..5d9a46962348 100644
--- a/pkg/sql/join_token.go
+++ b/pkg/sql/join_token.go
@@ -10,7 +10,9 @@
package sql
-import "github.com/cockroachdb/cockroach/pkg/settings"
+import (
+ "github.com/cockroachdb/cockroach/pkg/settings"
+)
// FeatureTLSAutoJoinEnabled is used to enable and disable the TLS auto-join
// feature.
@@ -19,6 +21,3 @@ var FeatureTLSAutoJoinEnabled = settings.RegisterBoolSetting(
"set to true to enable tls auto join through join tokens, false to disable; default is false",
false,
)
-
-// TODO(bilal): Implement a CREATE JOIN TOKEN statement, gated by
-// FeatureTLSAutoJoinEnabled.
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index bba388d06dc0..110829120063 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -3825,6 +3825,20 @@ may increase either contention or retry errors, or both.`,
},
),
+ "crdb_internal.create_join_token": makeBuiltin(
+ tree.FunctionProperties{Category: categorySystemInfo},
+ tree.Overload{
+ Types: tree.ArgTypes{},
+ ReturnType: tree.FixedReturnType(types.String),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ // TODO(bilal): Implement this.
+ return nil, nil
+ },
+ Info: "Creates a join token for use when adding a new node to a secure cluster.",
+ Volatility: tree.VolatilityVolatile,
+ },
+ ),
+
"crdb_internal.destroy_tenant": makeBuiltin(
tree.FunctionProperties{
Category: categoryMultiTenancy,
From 40d640b2a6741689a179e92e69828436e46bfd28 Mon Sep 17 00:00:00 2001
From: Bilal Akhtar
Date: Mon, 15 Mar 2021 17:14:44 -0400
Subject: [PATCH 3/6] {security,server}: Move JoinToken struct to security
package
As we're going with a system table based appraoch for saving join
tokens, it makes more sense to move the underlying struct to the
security package so that it can be used in the sql subsystem.
The reason why it was in server was so we could access
Gossip in the same code as the JoinToken struct itself.
Also remove the status server method added to generate/gossip
join token.
Release note: None.
---
pkg/{server => security}/join_token.go | 50 ++++++++++-----------
pkg/{server => security}/join_token_test.go | 21 +++++----
pkg/server/serverpb/status.go | 1 -
pkg/server/status.go | 23 ----------
4 files changed, 35 insertions(+), 60 deletions(-)
rename pkg/{server => security}/join_token.go (70%)
rename pkg/{server => security}/join_token_test.go (72%)
diff --git a/pkg/server/join_token.go b/pkg/security/join_token.go
similarity index 70%
rename from pkg/server/join_token.go
rename to pkg/security/join_token.go
index e7fcfae271f7..574ebb0f5fad 100644
--- a/pkg/server/join_token.go
+++ b/pkg/security/join_token.go
@@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
-package server
+package security
import (
"bytes"
@@ -19,7 +19,6 @@ import (
"io/ioutil"
"math/rand"
- "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
@@ -32,38 +31,39 @@ const (
joinTokenSecretLen = 16
)
-// joinToken is a container for a tokenID and associated sharedSecret for use
+// JoinToken is a container for a TokenID and associated SharedSecret for use
// in certificate-free add/join operations.
-type joinToken struct {
- tokenID uuid.UUID
- sharedSecret []byte
+type JoinToken struct {
+ TokenID uuid.UUID
+ SharedSecret []byte
fingerprint []byte
}
-// Generates a new join token, and signs it with the CA cert in sslCertsDir.
-func generateJoinToken(cm *security.CertificateManager) (joinToken, error) {
- var jt joinToken
+// GenerateJoinToken generates a new join token, and signs it with the CA cert
+// in the certificate manager.
+func GenerateJoinToken(cm *CertificateManager) (JoinToken, error) {
+ var jt JoinToken
- jt.tokenID = uuid.MakeV4()
+ jt.TokenID = uuid.MakeV4()
r := rand.New(rand.NewSource(timeutil.Now().UnixNano()))
- jt.sharedSecret = randutil.RandBytes(r, joinTokenSecretLen)
+ jt.SharedSecret = randutil.RandBytes(r, joinTokenSecretLen)
jt.sign(cm.CACert().FileContents)
return jt, nil
}
// sign signs the provided CA cert using the shared secret, and sets the
// fingerprint field on the join token to the HMAC signature.
-func (j *joinToken) sign(caCert []byte) {
- signer := hmac.New(sha256.New, j.sharedSecret)
+func (j *JoinToken) sign(caCert []byte) {
+ signer := hmac.New(sha256.New, j.SharedSecret)
_, _ = signer.Write(caCert)
j.fingerprint = signer.Sum(nil)
}
-// verifySignature verifies that the fingerprint provided in the join token
+// VerifySignature verifies that the fingerprint provided in the join token
// matches the signature of the provided CA cert with the join token's shared
// secret.
-func (j *joinToken) verifySignature(caCert []byte) bool {
- signer := hmac.New(sha256.New, j.sharedSecret)
+func (j *JoinToken) VerifySignature(caCert []byte) bool {
+ signer := hmac.New(sha256.New, j.SharedSecret)
_, _ = signer.Write(caCert)
// TODO(aaron-crl): Avoid timing attacks here.
return bytes.Equal(signer.Sum(nil), j.fingerprint)
@@ -72,8 +72,8 @@ func (j *joinToken) verifySignature(caCert []byte) bool {
// UnmarshalText implements the encoding.TextUnmarshaler interface.
//
// The format of the text (after base64-decoding) is:
-//
-func (j *joinToken) UnmarshalText(text []byte) error {
+//
+func (j *JoinToken) UnmarshalText(text []byte) error {
decoder := base64.NewDecoder(base64.URLEncoding, bytes.NewReader(text))
decoded, err := ioutil.ReadAll(decoder)
if err != nil {
@@ -90,28 +90,28 @@ func (j *joinToken) UnmarshalText(text []byte) error {
if cSum != expectedCSum {
return errors.New("invalid join token")
}
- if err := j.tokenID.UnmarshalBinary(decoded[:uuid.Size]); err != nil {
+ if err := j.TokenID.UnmarshalBinary(decoded[:uuid.Size]); err != nil {
return err
}
decoded = decoded[uuid.Size:]
- j.sharedSecret = decoded[:joinTokenSecretLen]
+ j.SharedSecret = decoded[:joinTokenSecretLen]
j.fingerprint = decoded[joinTokenSecretLen : len(decoded)-4]
return nil
}
// MarshalText implements the encoding.TextMarshaler interface.
-func (j *joinToken) MarshalText() ([]byte, error) {
- tokenID, err := j.tokenID.MarshalBinary()
+func (j *JoinToken) MarshalText() ([]byte, error) {
+ tokenID, err := j.TokenID.MarshalBinary()
if err != nil {
return nil, err
}
- if len(j.sharedSecret) != joinTokenSecretLen {
+ if len(j.SharedSecret) != joinTokenSecretLen {
return nil, errors.New("join token shared secret not of the right size")
}
- token := make([]byte, 0, len(tokenID)+len(j.sharedSecret)+len(j.fingerprint)+4)
+ token := make([]byte, 0, len(tokenID)+len(j.SharedSecret)+len(j.fingerprint)+4)
token = append(token, tokenID...)
- token = append(token, j.sharedSecret...)
+ token = append(token, j.SharedSecret...)
token = append(token, j.fingerprint...)
// Checksum.
cSum := crc32.ChecksumIEEE(token)
diff --git a/pkg/server/join_token_test.go b/pkg/security/join_token_test.go
similarity index 72%
rename from pkg/server/join_token_test.go
rename to pkg/security/join_token_test.go
index d00ab2b23faf..6d0c967c91a4 100644
--- a/pkg/server/join_token_test.go
+++ b/pkg/security/join_token_test.go
@@ -8,13 +8,12 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
-package server
+package security
import (
"math/rand"
"testing"
- "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
@@ -28,20 +27,20 @@ func TestJoinToken(t *testing.T) {
defer log.Scope(t).Close(t)
rng := rand.New(rand.NewSource(timeutil.Now().UnixNano()))
- j := &joinToken{
- tokenID: uuid.MakeV4(),
- sharedSecret: randutil.RandBytes(rng, joinTokenSecretLen),
+ j := &JoinToken{
+ TokenID: uuid.MakeV4(),
+ SharedSecret: randutil.RandBytes(rng, joinTokenSecretLen),
fingerprint: nil,
}
testCACert := []byte("foobar")
j.sign(testCACert)
- require.True(t, j.verifySignature(testCACert))
- require.False(t, j.verifySignature([]byte("test")))
+ require.True(t, j.VerifySignature(testCACert))
+ require.False(t, j.VerifySignature([]byte("test")))
require.NotNil(t, j.fingerprint)
marshaled, err := j.MarshalText()
require.NoError(t, err)
- j2 := &joinToken{}
+ j2 := &JoinToken{}
require.NoError(t, j2.UnmarshalText(marshaled))
require.Equal(t, j, j2)
@@ -51,11 +50,11 @@ func TestGenerateJoinToken(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
- cm, err := security.NewCertificateManager(security.EmbeddedCertsDir, security.CommandTLSSettings{})
+ cm, err := NewCertificateManager(EmbeddedCertsDir, CommandTLSSettings{})
require.NoError(t, err)
- token, err := generateJoinToken(cm)
+ token, err := GenerateJoinToken(cm)
require.NoError(t, err)
require.NotEmpty(t, token)
- require.True(t, token.verifySignature(cm.CACert().FileContents))
+ require.True(t, token.VerifySignature(cm.CACert().FileContents))
}
diff --git a/pkg/server/serverpb/status.go b/pkg/server/serverpb/status.go
index 09c53396327e..6451c38a564b 100644
--- a/pkg/server/serverpb/status.go
+++ b/pkg/server/serverpb/status.go
@@ -50,7 +50,6 @@ func MakeOptionalNodesStatusServer(s NodesStatusServer) OptionalNodesStatusServe
// by the SQL subsystem but is unavailable to tenants.
type NodesStatusServer interface {
Nodes(context.Context, *NodesRequest) (*NodesResponse, error)
- GenerateJoinToken(context.Context) (string, error)
}
// OptionalNodesStatusServer returns the wrapped NodesStatusServer, if it is
diff --git a/pkg/server/status.go b/pkg/server/status.go
index d548bda9fad5..e751c3d452de 100644
--- a/pkg/server/status.go
+++ b/pkg/server/status.go
@@ -2523,26 +2523,3 @@ func (s *statusServer) JobStatus(
return &serverpb.JobStatusResponse{Job: res}, nil
}
-
-// GenerateJoinToken generates a new ephemeral join token. For use by the sql
-// subsystem directly. The response is a base64 marshaled form of the join token
-// that can be shared to new nodes that want to join this cluster.
-func (s *statusServer) GenerateJoinToken(ctx context.Context) (string, error) {
- if !sql.FeatureTLSAutoJoinEnabled.Get(&s.st.SV) {
- return "", errors.New("join token generation disabled")
- }
-
- cm, err := s.rpcCtx.GetCertificateManager()
- if err != nil {
- return "", errors.Wrap(err, "error when getting certificate manager")
- }
- jt, err := generateJoinToken(cm)
- if err != nil {
- return "", errors.Wrap(err, "error when generating join token")
- }
- token, err := jt.MarshalText()
- if err != nil {
- return "", errors.Wrap(err, "error when marshaling join token")
- }
- return string(token), nil
-}
From ce1c68397db8ebc222ed201fef1f9ca92485ddcd Mon Sep 17 00:00:00 2001
From: Bilal Akhtar
Date: Mon, 15 Mar 2021 19:58:04 -0400
Subject: [PATCH 4/6] sql: Add system table join_tokens, use it in
create_join_token()
This change adds a new system table, `join_tokens`, for the
exclusive use of storing join tokens. This is necessary as
we need guaranteed at-most-once semantics with these, which
transactions give us pretty easily.
This change also updates the `create_join_token()` function to
create and store a join token in that table.
Currently, there's no mechanism to remove expired join tokens.
Release note (general change): Add `crdb_internal.create_join_token()`
sql builtin function to create join tokens for use when joining
new nodes to a secure cluster. This functionality is hidden behind
a feature flag.
---
pkg/ccl/backupccl/system_schema.go | 3 ++
pkg/keys/constants.go | 1 +
pkg/migration/migrations/BUILD.bazel | 1 +
pkg/security/BUILD.bazel | 9 +++-
pkg/security/join_token.go | 5 ++
pkg/server/BUILD.bazel | 4 --
pkg/sql/BUILD.bazel | 1 +
pkg/sql/catalog/bootstrap/metadata.go | 4 ++
pkg/sql/catalog/descpb/privilege.go | 1 +
pkg/sql/catalog/systemschema/system.go | 51 +++++++++++++++++++++
pkg/sql/conn_executor.go | 1 +
pkg/sql/join_token.go | 53 ++++++++++++++++++++++
pkg/sql/join_token_test.go | 63 ++++++++++++++++++++++++++
pkg/sql/planner.go | 1 +
pkg/sql/sem/builtins/builtins.go | 7 ++-
pkg/sql/sem/tree/eval.go | 12 +++++
pkg/sql/tests/system_table_test.go | 1 +
17 files changed, 211 insertions(+), 7 deletions(-)
create mode 100644 pkg/sql/join_token_test.go
diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go
index f4342a1417b7..1153b3b6cb75 100644
--- a/pkg/ccl/backupccl/system_schema.go
+++ b/pkg/ccl/backupccl/system_schema.go
@@ -244,6 +244,9 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{
systemschema.MigrationsTable.GetName(): {
shouldIncludeInClusterBackup: optOutOfClusterBackup,
},
+ systemschema.JoinTokensTable.GetName(): {
+ shouldIncludeInClusterBackup: optOutOfClusterBackup,
+ },
}
// GetSystemTablesToIncludeInClusterBackup returns a set of system table names that
diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go
index 203bc3621a6e..24248f496408 100644
--- a/pkg/keys/constants.go
+++ b/pkg/keys/constants.go
@@ -403,6 +403,7 @@ const (
TenantsRangesID = 38 // pseudo
SqllivenessID = 39
MigrationsID = 40
+ JoinTokensTableID = 41
// CommentType is type for system.comments
DatabaseCommentType = 0
diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel
index dc2d58e07a5a..b822b306f842 100644
--- a/pkg/migration/migrations/BUILD.bazel
+++ b/pkg/migration/migrations/BUILD.bazel
@@ -4,6 +4,7 @@ go_library(
name = "migrations",
srcs = [
"foreign_key_representation_upgrade.go",
+ "join_tokens.go",
"migrations.go",
"migrations_table.go",
"namespace_migration.go",
diff --git a/pkg/security/BUILD.bazel b/pkg/security/BUILD.bazel
index e65540200470..6da72ea71f7e 100644
--- a/pkg/security/BUILD.bazel
+++ b/pkg/security/BUILD.bazel
@@ -8,6 +8,7 @@ go_library(
"certificate_loader.go",
"certificate_manager.go",
"certs.go",
+ "join_token.go",
"ocsp.go",
"password.go",
"pem.go",
@@ -25,14 +26,17 @@ go_library(
"//pkg/settings/cluster",
"//pkg/sql/lexbase",
"//pkg/util/contextutil",
+ "//pkg/util/encoding",
"//pkg/util/envutil",
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/metric",
+ "//pkg/util/randutil",
"//pkg/util/stop",
"//pkg/util/syncutil",
"//pkg/util/sysutil",
"//pkg/util/timeutil",
+ "//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_redact//:redact",
@@ -54,13 +58,14 @@ go_test(
"certs_rotation_test.go",
"certs_tenant_test.go",
"certs_test.go",
+ "join_token_test.go",
"main_test.go",
"tls_test.go",
"username_test.go",
"x509_test.go",
],
+ embed = [":security"],
deps = [
- ":security",
"//pkg/base",
"//pkg/roachpb",
"//pkg/rpc",
@@ -71,7 +76,9 @@ go_test(
"//pkg/util/envutil",
"//pkg/util/leaktest",
"//pkg/util/log",
+ "//pkg/util/randutil",
"//pkg/util/timeutil",
+ "//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
"@org_golang_x_exp//rand",
diff --git a/pkg/security/join_token.go b/pkg/security/join_token.go
index 574ebb0f5fad..9e5ab1a8613b 100644
--- a/pkg/security/join_token.go
+++ b/pkg/security/join_token.go
@@ -18,6 +18,7 @@ import (
"hash/crc32"
"io/ioutil"
"math/rand"
+ "time"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
@@ -29,6 +30,10 @@ import (
const (
// Length of the join token shared secret.
joinTokenSecretLen = 16
+
+ // JoinTokenExpiration is the default expiration time of newly created join
+ // tokens.
+ JoinTokenExpiration = 30 * time.Minute
)
// JoinToken is a container for a TokenID and associated SharedSecret for use
diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel
index 390ed668e12f..a08593bc78f5 100644
--- a/pkg/server/BUILD.bazel
+++ b/pkg/server/BUILD.bazel
@@ -20,7 +20,6 @@ go_library(
"idle_monitor.go",
"init.go",
"init_handshake.go",
- "join_token.go",
"loopback.go",
"migration.go",
"node.go",
@@ -147,7 +146,6 @@ go_library(
"//pkg/ui",
"//pkg/util",
"//pkg/util/contextutil",
- "//pkg/util/encoding",
"//pkg/util/envutil",
"//pkg/util/grpcutil",
"//pkg/util/hlc",
@@ -163,7 +161,6 @@ go_library(
"//pkg/util/netutil",
"//pkg/util/protoutil",
"//pkg/util/quotapool",
- "//pkg/util/randutil",
"//pkg/util/retry",
"//pkg/util/stop",
"//pkg/util/syncutil",
@@ -256,7 +253,6 @@ go_test(
"idle_monitor_test.go",
"init_handshake_test.go",
"intent_test.go",
- "join_token_test.go",
"main_test.go",
"migration_test.go",
"multi_store_test.go",
diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel
index 30023cf96d4b..711976ae0f1c 100644
--- a/pkg/sql/BUILD.bazel
+++ b/pkg/sql/BUILD.bazel
@@ -438,6 +438,7 @@ go_test(
"indexbackfiller_test.go",
"instrumentation_test.go",
"internal_test.go",
+ "join_token_test.go",
"main_test.go",
"materialized_view_test.go",
"metric_test.go",
diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go
index bdbb1439f62a..1837107832ba 100644
--- a/pkg/sql/catalog/bootstrap/metadata.go
+++ b/pkg/sql/catalog/bootstrap/metadata.go
@@ -343,6 +343,10 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) {
target.AddDescriptor(keys.SystemDatabaseID, systemschema.ScheduledJobsTable)
target.AddDescriptor(keys.SystemDatabaseID, systemschema.SqllivenessTable)
target.AddDescriptor(keys.SystemDatabaseID, systemschema.MigrationsTable)
+
+ // Tables introduced in 21.1.
+
+ target.AddDescriptor(keys.SystemDatabaseID, systemschema.JoinTokensTable)
}
// addSplitIDs adds a split point for each of the PseudoTableIDs to the supplied
diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go
index 5daad847d396..ae835d49c71c 100644
--- a/pkg/sql/catalog/descpb/privilege.go
+++ b/pkg/sql/catalog/descpb/privilege.go
@@ -448,6 +448,7 @@ var SystemAllowedPrivileges = map[ID]privilege.List{
keys.ScheduledJobsTableID: privilege.ReadWriteData,
keys.SqllivenessID: privilege.ReadWriteData,
keys.MigrationsID: privilege.ReadWriteData,
+ keys.JoinTokensTableID: privilege.ReadWriteData,
}
// SetOwner sets the owner of the privilege descriptor to the provided string.
diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go
index 1fc3f0620dcc..9de2714dd8c7 100644
--- a/pkg/sql/catalog/systemschema/system.go
+++ b/pkg/sql/catalog/systemschema/system.go
@@ -357,6 +357,14 @@ CREATE TABLE system.migrations (
FAMILY "primary" (major, minor, patch, internal, completed_at),
PRIMARY KEY (major, minor, patch, internal)
)`
+
+ JoinTokensTableSchema = `
+CREATE TABLE system.join_tokens (
+ id UUID NOT NULL PRIMARY KEY,
+ secret BYTES NOT NULL,
+ expiration TIMESTAMPTZ NOT NULL,
+ FAMILY "primary" (id, secret, expiration),
+)`
)
func pk(name string) descpb.IndexDescriptor {
@@ -1756,6 +1764,49 @@ var (
FormatVersion: descpb.InterleavedFormatVersion,
NextMutationID: 1,
})
+
+ // MigrationsTable is the descriptor for the migrations table. It stores facts
+ // about the completion state of long-running migrations. It is used to
+ // prevent migrations from running again after they have been completed.
+ JoinTokensTable = makeTable(descpb.TableDescriptor{
+ Name: "join_tokens",
+ ID: keys.JoinTokensTableID,
+ ParentID: keys.SystemDatabaseID,
+ UnexposedParentSchemaID: keys.PublicSchemaID,
+ Version: 1,
+ Columns: []descpb.ColumnDescriptor{
+ {Name: "id", ID: 1, Type: types.Uuid, Nullable: false},
+ {Name: "secret", ID: 2, Type: types.String, Nullable: false},
+ {Name: "expiration", ID: 3, Type: types.TimestampTZ, Nullable: false},
+ },
+ NextColumnID: 4,
+ Families: []descpb.ColumnFamilyDescriptor{
+ {
+ Name: "primary",
+ ID: 0,
+ ColumnNames: []string{"id", "secret", "expiration"},
+ ColumnIDs: []descpb.ColumnID{1, 2, 3},
+ DefaultColumnID: 2,
+ },
+ },
+ NextFamilyID: 1,
+ PrimaryIndex: descpb.IndexDescriptor{
+ Name: tabledesc.PrimaryKeyIndexName,
+ ID: 1,
+ Unique: true,
+ ColumnNames: []string{"id"},
+ ColumnDirections: []descpb.IndexDescriptor_Direction{
+ descpb.IndexDescriptor_ASC,
+ },
+ ColumnIDs: []descpb.ColumnID{1},
+ Version: descpb.EmptyArraysInInvertedIndexesVersion,
+ },
+ NextIndexID: 2,
+ Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor(
+ descpb.SystemAllowedPrivileges[keys.JoinTokensTableID], security.NodeUserName()),
+ FormatVersion: descpb.InterleavedFormatVersion,
+ NextMutationID: 1,
+ })
)
// newCommentPrivilegeDescriptor returns a privilege descriptor for comment table
diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go
index d480ea2c099e..b531f57af07f 100644
--- a/pkg/sql/conn_executor.go
+++ b/pkg/sql/conn_executor.go
@@ -2192,6 +2192,7 @@ func (ex *connExecutor) initEvalCtx(ctx context.Context, evalCtx *extendedEvalCo
ClientNoticeSender: p,
Sequence: p,
Tenant: p,
+ JoinTokenCreator: p,
SessionData: ex.sessionData,
Settings: ex.server.cfg.Settings,
TestingKnobs: ex.server.cfg.EvalContextTestingKnobs,
diff --git a/pkg/sql/join_token.go b/pkg/sql/join_token.go
index 5d9a46962348..722af85696d3 100644
--- a/pkg/sql/join_token.go
+++ b/pkg/sql/join_token.go
@@ -11,7 +11,17 @@
package sql
import (
+ "context"
+ "time"
+
+ "github.com/cockroachdb/cockroach/pkg/featureflag"
+ "github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings"
+ "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
+ "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/errors"
)
// FeatureTLSAutoJoinEnabled is used to enable and disable the TLS auto-join
@@ -21,3 +31,46 @@ var FeatureTLSAutoJoinEnabled = settings.RegisterBoolSetting(
"set to true to enable tls auto join through join tokens, false to disable; default is false",
false,
)
+
+// CreateJoinToken implements the tree.JoinTokenCreator interface.
+func (p *planner) CreateJoinToken(ctx context.Context) (string, error) {
+ hasAdmin, err := p.HasAdminRole(ctx)
+ if err != nil {
+ return "", err
+ }
+ if !hasAdmin {
+ return "", pgerror.New(pgcode.InsufficientPrivilege, "must be admin to create join token")
+ }
+ if err = featureflag.CheckEnabled(
+ ctx, p.ExecCfg(), FeatureTLSAutoJoinEnabled, "create join tokens"); err != nil {
+ return "", err
+ }
+
+ cm, err := p.ExecCfg().RPCContext.SecurityContext.GetCertificateManager()
+ if err != nil {
+ return "", errors.Wrap(err, "error when getting certificate manager")
+ }
+
+ jt, err := security.GenerateJoinToken(cm)
+ if err != nil {
+ return "", errors.Wrap(err, "error when generating join token")
+ }
+ token, err := jt.MarshalText()
+ if err != nil {
+ return "", errors.Wrap(err, "error when marshaling join token")
+ }
+ expiration := timeutil.Now().Add(security.JoinTokenExpiration)
+ err = p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
+ _, err = p.ExecCfg().InternalExecutor.Exec(
+ ctx, "insert-join-token", txn,
+ "insert into system.join_tokens(id, secret, expiration) "+
+ "values($1, $2, $3)",
+ jt.TokenID.String(), jt.SharedSecret, expiration.Format(time.RFC3339),
+ )
+ return err
+ })
+ if err != nil {
+ return "", errors.Wrap(err, "could not persist join token in system table")
+ }
+ return string(token), nil
+}
diff --git a/pkg/sql/join_token_test.go b/pkg/sql/join_token_test.go
new file mode 100644
index 000000000000..7c88bfafaf7e
--- /dev/null
+++ b/pkg/sql/join_token_test.go
@@ -0,0 +1,63 @@
+// Copyright 2021 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 sql
+
+import (
+ "context"
+ "testing"
+
+ "github.com/cockroachdb/cockroach/pkg/base"
+ "github.com/cockroachdb/cockroach/pkg/security"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
+ "github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
+ "github.com/cockroachdb/cockroach/pkg/util/leaktest"
+ "github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/stretchr/testify/require"
+)
+
+func TestCreateJoinToken(t *testing.T) {
+ defer leaktest.AfterTest(t)()
+ defer log.Scope(t).Close(t)
+
+ settings := cluster.MakeTestingClusterSettings()
+ FeatureTLSAutoJoinEnabled.Override(&settings.SV, true)
+ s, sqldb, _ := serverutils.StartServer(t, base.TestServerArgs{
+ Settings: settings,
+ })
+ defer s.Stopper().Stop(context.Background())
+
+ rows, err := sqldb.Query("SELECT crdb_internal.create_join_token();")
+ require.NoError(t, err)
+ count := 0
+ var token string
+ for rows.Next() {
+ count++
+ require.NoError(t, rows.Scan(&token))
+ require.NotEmpty(t, token)
+ }
+ var jt security.JoinToken
+ require.NoError(t, jt.UnmarshalText([]byte(token)))
+ require.Equal(t, 1, count)
+ require.NoError(t, rows.Close())
+
+ rows, err = sqldb.Query("SELECT id FROM system.join_tokens;")
+ require.NoError(t, err)
+ count = 0
+ for rows.Next() {
+ count++
+ var tokenID string
+ require.NoError(t, rows.Scan(&tokenID))
+ require.NotEmpty(t, jt)
+ require.Equal(t, jt.TokenID.String(), tokenID)
+ }
+ require.Equal(t, 1, count)
+ require.NoError(t, rows.Close())
+}
diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go
index 8d5d9ce35fba..c5e6529b9791 100644
--- a/pkg/sql/planner.go
+++ b/pkg/sql/planner.go
@@ -362,6 +362,7 @@ func newInternalPlanner(
p.extendedEvalCtx.ClientNoticeSender = p
p.extendedEvalCtx.Sequence = p
p.extendedEvalCtx.Tenant = p
+ p.extendedEvalCtx.JoinTokenCreator = p
p.extendedEvalCtx.ClusterID = execCfg.ClusterID()
p.extendedEvalCtx.ClusterName = execCfg.RPCContext.ClusterName()
p.extendedEvalCtx.NodeID = execCfg.NodeID
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index 110829120063..49b36c33b63d 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -3831,8 +3831,11 @@ may increase either contention or retry errors, or both.`,
Types: tree.ArgTypes{},
ReturnType: tree.FixedReturnType(types.String),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
- // TODO(bilal): Implement this.
- return nil, nil
+ token, err := ctx.JoinTokenCreator.CreateJoinToken(ctx.Context)
+ if err != nil {
+ return nil, err
+ }
+ return tree.NewDString(token), nil
},
Info: "Creates a join token for use when adding a new node to a secure cluster.",
Volatility: tree.VolatilityVolatile,
diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go
index 18fd0522c113..b624043f6db0 100644
--- a/pkg/sql/sem/tree/eval.go
+++ b/pkg/sql/sem/tree/eval.go
@@ -3236,6 +3236,16 @@ type TenantOperator interface {
GCTenant(ctx context.Context, tenantID uint64) error
}
+// JoinTokenCreator is capable of creating and persisting join tokens, allowing
+// SQL builtin functions to create join tokens. The methods will return errors
+// when run on multi-tenant clusters or with this functionality unavailable.
+type JoinTokenCreator interface {
+ // CreateJoinToken creates a new ephemeral join token and persists it
+ // across the cluster. This join token can then be used to have new nodes
+ // join the cluster and exchange certificates securely.
+ CreateJoinToken(ctx context.Context) (string, error)
+}
+
// EvalContextTestingKnobs contains test knobs.
type EvalContextTestingKnobs struct {
// AssertFuncExprReturnTypes indicates whether FuncExpr evaluations
@@ -3364,6 +3374,8 @@ type EvalContext struct {
Tenant TenantOperator
+ JoinTokenCreator JoinTokenCreator
+
// The transaction in which the statement is executing.
Txn *kv.Txn
// A handle to the database.
diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go
index 25ee0b7fb2db..715d7b53d422 100644
--- a/pkg/sql/tests/system_table_test.go
+++ b/pkg/sql/tests/system_table_test.go
@@ -191,6 +191,7 @@ func TestSystemTableLiterals(t *testing.T) {
{keys.ScheduledJobsTableID, systemschema.ScheduledJobsTableSchema, systemschema.ScheduledJobsTable},
{keys.SqllivenessID, systemschema.SqllivenessTableSchema, systemschema.SqllivenessTable},
{keys.MigrationsID, systemschema.MigrationsTableSchema, systemschema.MigrationsTable},
+ {keys.JoinTokensTableID, systemschema.JoinTokensTableSchema, systemschema.JoinTokensTable},
} {
privs := *test.pkg.GetPrivileges()
gen, err := sql.CreateTestTableDescriptor(
From 093ff6666f163a3d2b438a4bb3a6ebf286c85283 Mon Sep 17 00:00:00 2001
From: Bilal Akhtar
Date: Mon, 22 Mar 2021 16:51:45 -0400
Subject: [PATCH 5/6] migration: Add SQL migration for new systems.join_tokens
table
This change adds a SQL migration (gated on cluster versions)
for the join_tokens table added in a previous commit.
Release note: None.
---
.../settings/settings-for-tenants.txt | 2 +-
docs/generated/settings/settings.html | 2 +-
pkg/clusterversion/cockroach_versions.go | 7 +++++
pkg/migration/migrations/join_tokens.go | 28 +++++++++++++++++++
pkg/migration/migrations/migrations.go | 5 ++++
5 files changed, 42 insertions(+), 2 deletions(-)
create mode 100644 pkg/migration/migrations/join_tokens.go
diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt
index f8ab5fb3ddce..b2280dc49b8c 100644
--- a/docs/generated/settings/settings-for-tenants.txt
+++ b/docs/generated/settings/settings-for-tenants.txt
@@ -105,4 +105,4 @@ trace.datadog.project string CockroachDB the project under which traces will be
trace.debug.enable boolean false if set, traces for recent requests can be seen at https:///debug/requests
trace.lightstep.token string if set, traces go to Lightstep using this token
trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time.
-version version 20.2-48 set the active cluster version in the format '.'
+version version 20.2-50 set the active cluster version in the format '.'
diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index 9e4fd280bf16..51697cadebc0 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -107,6 +107,6 @@
trace.debug.enable | boolean | false | if set, traces for recent requests can be seen at https:///debug/requests |
trace.lightstep.token | string |
| if set, traces go to Lightstep using this token |
trace.zipkin.collector | string |
| if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time. |
-version | version | 20.2-48 | set the active cluster version in the format '.' |
+version | version | 20.2-50 | set the active cluster version in the format '.' |
diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go
index 651af26ac892..a760518b9c44 100644
--- a/pkg/clusterversion/cockroach_versions.go
+++ b/pkg/clusterversion/cockroach_versions.go
@@ -275,6 +275,9 @@ const (
// ProtectedTsMetaPrivilegesMigration is for the migration which fixes the
// privileges of the protected_ts_meta system table.
ProtectedTsMetaPrivilegesMigration
+ // JoinTokensTable adds the system table for storing ephemeral generated
+ // join tokens.
+ JoinTokensTable
// Step (1): Add new versions here.
)
@@ -477,6 +480,10 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: ProtectedTsMetaPrivilegesMigration,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 48},
},
+ {
+ Key: JoinTokensTable,
+ Version: roachpb.Version{Major: 20, Minor: 2, Internal: 50},
+ },
// Step (2): Add new versions here.
})
diff --git a/pkg/migration/migrations/join_tokens.go b/pkg/migration/migrations/join_tokens.go
new file mode 100644
index 000000000000..dcc4335e229d
--- /dev/null
+++ b/pkg/migration/migrations/join_tokens.go
@@ -0,0 +1,28 @@
+// Copyright 2021 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 migrations
+
+import (
+ "context"
+
+ "github.com/cockroachdb/cockroach/pkg/clusterversion"
+ "github.com/cockroachdb/cockroach/pkg/migration"
+ "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
+ "github.com/cockroachdb/cockroach/pkg/sqlmigrations"
+)
+
+func joinTokensTableMigration(
+ ctx context.Context, _ clusterversion.ClusterVersion, d migration.SQLDeps,
+) error {
+ return sqlmigrations.CreateSystemTable(
+ ctx, d.DB, d.Codec, d.Settings, systemschema.JoinTokensTable,
+ )
+}
diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go
index 66655dfc5495..258b90b4d3a7 100644
--- a/pkg/migration/migrations/migrations.go
+++ b/pkg/migration/migrations/migrations.go
@@ -62,6 +62,11 @@ var migrations = []migration.Migration{
toCV(clusterversion.ProtectedTsMetaPrivilegesMigration),
protectedTsMetaPrivilegesMigration,
),
+ migration.NewSQLMigration(
+ "add the systems.join_tokens table",
+ toCV(clusterversion.JoinTokensTable),
+ joinTokensTableMigration,
+ ),
}
func init() {
From 3a44b45f074d5a9fadac7d4e9db86e988e3ea8e2 Mon Sep 17 00:00:00 2001
From: Bilal Akhtar
Date: Mon, 29 Mar 2021 11:30:52 -0400
Subject: [PATCH 6/6] cli,sql: Update tests for new system table
A lot of tests in the cli and sql packages assert on
system table IDs, range numbers, etc., that were failing
due to the addition of the `join_tokens` table. This change
updates all those tests to account for that table.
Release note: None.
---
pkg/cli/testdata/doctor/testcluster | 2 +-
pkg/cli/testdata/zip/partial1 | 9 +-
pkg/cli/testdata/zip/partial1_excluded | 9 +-
pkg/cli/testdata/zip/partial2 | 9 +-
pkg/cli/testdata/zip/specialnames | 3 +-
pkg/cli/testdata/zip/testzip | 6 +-
pkg/clusterversion/key_string.go | 5 +-
pkg/sql/catalog/systemschema/system.go | 6 +-
.../testdata/logic_test/distsql_crdb_internal | 1 -
.../logictest/testdata/logic_test/grant_table | 15 ++
.../testdata/logic_test/information_schema | 32 ++++
.../logictest/testdata/logic_test/pg_catalog | 2 +
pkg/sql/logictest/testdata/logic_test/ranges | 6 +-
.../logictest/testdata/logic_test/show_source | 2 +
pkg/sql/logictest/testdata/logic_test/system | 12 ++
.../testdata/logic_test/system_namespace | 1 +
.../testdata/autocommit_nonmetamorphic | 174 +++++++++---------
pkg/sql/opt/exec/execbuilder/testdata/delete | 6 +-
.../testdata/show_trace_nonmetamorphic | 2 +-
pkg/sql/pgwire/pgwire_test.go | 2 +-
pkg/sql/tests/testdata/initial_keys | 15 +-
21 files changed, 202 insertions(+), 117 deletions(-)
diff --git a/pkg/cli/testdata/doctor/testcluster b/pkg/cli/testdata/doctor/testcluster
index acea014ed492..5f8c7b598c80 100644
--- a/pkg/cli/testdata/doctor/testcluster
+++ b/pkg/cli/testdata/doctor/testcluster
@@ -1,7 +1,7 @@
doctor cluster
----
debug doctor cluster
-Examining 35 descriptors and 36 namespace entries...
+Examining 36 descriptors and 37 namespace entries...
ParentID 50, ParentSchemaID 29: relation "foo" (53): expected matching namespace entry, found none
Examining 1 running jobs...
ERROR: validation failed
diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1
index 7dc9cedf58ae..86d4c9d101a6 100644
--- a/pkg/cli/testdata/zip/partial1
+++ b/pkg/cli/testdata/zip/partial1
@@ -61,7 +61,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t
^- resulted in ...
requesting log file ...
requesting log file ...
-requesting ranges... 36 found
+requesting ranges... 37 found
writing: debug/nodes/1/ranges/1.json
writing: debug/nodes/1/ranges/2.json
writing: debug/nodes/1/ranges/3.json
@@ -98,6 +98,7 @@ writing: debug/nodes/1/ranges/33.json
writing: debug/nodes/1/ranges/34.json
writing: debug/nodes/1/ranges/35.json
writing: debug/nodes/1/ranges/36.json
+writing: debug/nodes/1/ranges/37.json
writing: debug/nodes/2/status.json
using SQL connection URL for node 2: postgresql://...
retrieving SQL data for crdb_internal.feature_usage... writing: debug/nodes/2/crdb_internal.feature_usage.txt
@@ -202,7 +203,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t
^- resulted in ...
requesting log file ...
requesting log file ...
-requesting ranges... 36 found
+requesting ranges... 37 found
writing: debug/nodes/3/ranges/1.json
writing: debug/nodes/3/ranges/2.json
writing: debug/nodes/3/ranges/3.json
@@ -239,6 +240,7 @@ writing: debug/nodes/3/ranges/33.json
writing: debug/nodes/3/ranges/34.json
writing: debug/nodes/3/ranges/35.json
writing: debug/nodes/3/ranges/36.json
+writing: debug/nodes/3/ranges/37.json
doctor examining cluster...writing: debug/reports/doctor.txt
requesting list of SQL databases... 3 found
requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json
@@ -246,7 +248,7 @@ requesting database details for defaultdb... writing: debug/schema/defaultdb@det
requesting database details for postgres... writing: debug/schema/postgres@details.json
0 tables found
requesting database details for system... writing: debug/schema/system@details.json
-30 tables found
+31 tables found
requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json
requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json
requesting table details for system.public.users... writing: debug/schema/system/public_users.json
@@ -277,5 +279,6 @@ requesting table details for system.public.statement_diagnostics... writing: deb
requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json
requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json
requesting table details for system.public.migrations... writing: debug/schema/system/public_migrations.json
+requesting table details for system.public.join_tokens... writing: debug/schema/system/public_join_tokens.json
writing: debug/pprof-summary.sh
writing: debug/hot-ranges.sh
diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded
index ae3bf0a31890..25ac73bed2a9 100644
--- a/pkg/cli/testdata/zip/partial1_excluded
+++ b/pkg/cli/testdata/zip/partial1_excluded
@@ -61,7 +61,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t
^- resulted in ...
requesting log file ...
requesting log file ...
-requesting ranges... 36 found
+requesting ranges... 37 found
writing: debug/nodes/1/ranges/1.json
writing: debug/nodes/1/ranges/2.json
writing: debug/nodes/1/ranges/3.json
@@ -98,6 +98,7 @@ writing: debug/nodes/1/ranges/33.json
writing: debug/nodes/1/ranges/34.json
writing: debug/nodes/1/ranges/35.json
writing: debug/nodes/1/ranges/36.json
+writing: debug/nodes/1/ranges/37.json
writing: debug/nodes/2.skipped
writing: debug/nodes/3/status.json
using SQL connection URL for node 3: postgresql://...
@@ -130,7 +131,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t
^- resulted in ...
requesting log file ...
requesting log file ...
-requesting ranges... 36 found
+requesting ranges... 37 found
writing: debug/nodes/3/ranges/1.json
writing: debug/nodes/3/ranges/2.json
writing: debug/nodes/3/ranges/3.json
@@ -167,6 +168,7 @@ writing: debug/nodes/3/ranges/33.json
writing: debug/nodes/3/ranges/34.json
writing: debug/nodes/3/ranges/35.json
writing: debug/nodes/3/ranges/36.json
+writing: debug/nodes/3/ranges/37.json
doctor examining cluster...writing: debug/reports/doctor.txt
requesting list of SQL databases... 3 found
requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json
@@ -174,7 +176,7 @@ requesting database details for defaultdb... writing: debug/schema/defaultdb@det
requesting database details for postgres... writing: debug/schema/postgres@details.json
0 tables found
requesting database details for system... writing: debug/schema/system@details.json
-30 tables found
+31 tables found
requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json
requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json
requesting table details for system.public.users... writing: debug/schema/system/public_users.json
@@ -205,5 +207,6 @@ requesting table details for system.public.statement_diagnostics... writing: deb
requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json
requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json
requesting table details for system.public.migrations... writing: debug/schema/system/public_migrations.json
+requesting table details for system.public.join_tokens... writing: debug/schema/system/public_join_tokens.json
writing: debug/pprof-summary.sh
writing: debug/hot-ranges.sh
diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2
index 2a8927e160be..79ab6d505265 100644
--- a/pkg/cli/testdata/zip/partial2
+++ b/pkg/cli/testdata/zip/partial2
@@ -61,7 +61,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t
^- resulted in ...
requesting log file ...
requesting log file ...
-requesting ranges... 36 found
+requesting ranges... 37 found
writing: debug/nodes/1/ranges/1.json
writing: debug/nodes/1/ranges/2.json
writing: debug/nodes/1/ranges/3.json
@@ -98,6 +98,7 @@ writing: debug/nodes/1/ranges/33.json
writing: debug/nodes/1/ranges/34.json
writing: debug/nodes/1/ranges/35.json
writing: debug/nodes/1/ranges/36.json
+writing: debug/nodes/1/ranges/37.json
writing: debug/nodes/3/status.json
using SQL connection URL for node 3: postgresql://...
retrieving SQL data for crdb_internal.feature_usage... writing: debug/nodes/3/crdb_internal.feature_usage.txt
@@ -129,7 +130,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t
^- resulted in ...
requesting log file ...
requesting log file ...
-requesting ranges... 36 found
+requesting ranges... 37 found
writing: debug/nodes/3/ranges/1.json
writing: debug/nodes/3/ranges/2.json
writing: debug/nodes/3/ranges/3.json
@@ -166,6 +167,7 @@ writing: debug/nodes/3/ranges/33.json
writing: debug/nodes/3/ranges/34.json
writing: debug/nodes/3/ranges/35.json
writing: debug/nodes/3/ranges/36.json
+writing: debug/nodes/3/ranges/37.json
doctor examining cluster...writing: debug/reports/doctor.txt
requesting list of SQL databases... 3 found
requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json
@@ -173,7 +175,7 @@ requesting database details for defaultdb... writing: debug/schema/defaultdb@det
requesting database details for postgres... writing: debug/schema/postgres@details.json
0 tables found
requesting database details for system... writing: debug/schema/system@details.json
-30 tables found
+31 tables found
requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json
requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json
requesting table details for system.public.users... writing: debug/schema/system/public_users.json
@@ -204,5 +206,6 @@ requesting table details for system.public.statement_diagnostics... writing: deb
requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json
requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json
requesting table details for system.public.migrations... writing: debug/schema/system/public_migrations.json
+requesting table details for system.public.join_tokens... writing: debug/schema/system/public_join_tokens.json
writing: debug/pprof-summary.sh
writing: debug/hot-ranges.sh
diff --git a/pkg/cli/testdata/zip/specialnames b/pkg/cli/testdata/zip/specialnames
index 3035b5113d8b..21c2bdc8d828 100644
--- a/pkg/cli/testdata/zip/specialnames
+++ b/pkg/cli/testdata/zip/specialnames
@@ -22,7 +22,7 @@ requesting table details for defaultdb.public."../system"... writing: debug/sche
requesting database details for postgres... writing: debug/schema/postgres@details.json
0 tables found
requesting database details for system... writing: debug/schema/system-1@details.json
-30 tables found
+31 tables found
requesting table details for system.public.namespace... writing: debug/schema/system-1/public_namespace.json
requesting table details for system.public.descriptor... writing: debug/schema/system-1/public_descriptor.json
requesting table details for system.public.users... writing: debug/schema/system-1/public_users.json
@@ -53,3 +53,4 @@ requesting table details for system.public.statement_diagnostics... writing: deb
requesting table details for system.public.scheduled_jobs... writing: debug/schema/system-1/public_scheduled_jobs.json
requesting table details for system.public.sqlliveness... writing: debug/schema/system-1/public_sqlliveness.json
requesting table details for system.public.migrations... writing: debug/schema/system-1/public_migrations.json
+requesting table details for system.public.join_tokens... writing: debug/schema/system-1/public_join_tokens.json
diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip
index b2c9aa9cf283..6184c46f3312 100644
--- a/pkg/cli/testdata/zip/testzip
+++ b/pkg/cli/testdata/zip/testzip
@@ -60,7 +60,7 @@ requesting heap profile for node 1... writing: debug/nodes/1/heap.pprof
requesting heap files for node 1... ? found
requesting goroutine files for node 1... 0 found
requesting log file ...
-requesting ranges... 36 found
+requesting ranges... 37 found
writing: debug/nodes/1/ranges/1.json
writing: debug/nodes/1/ranges/2.json
writing: debug/nodes/1/ranges/3.json
@@ -97,6 +97,7 @@ writing: debug/nodes/1/ranges/33.json
writing: debug/nodes/1/ranges/34.json
writing: debug/nodes/1/ranges/35.json
writing: debug/nodes/1/ranges/36.json
+writing: debug/nodes/1/ranges/37.json
doctor examining cluster...writing: debug/reports/doctor.txt
requesting list of SQL databases... 3 found
requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json
@@ -104,7 +105,7 @@ requesting database details for defaultdb... writing: debug/schema/defaultdb@det
requesting database details for postgres... writing: debug/schema/postgres@details.json
0 tables found
requesting database details for system... writing: debug/schema/system@details.json
-30 tables found
+31 tables found
requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json
requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json
requesting table details for system.public.users... writing: debug/schema/system/public_users.json
@@ -135,5 +136,6 @@ requesting table details for system.public.statement_diagnostics... writing: deb
requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json
requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json
requesting table details for system.public.migrations... writing: debug/schema/system/public_migrations.json
+requesting table details for system.public.join_tokens... writing: debug/schema/system/public_join_tokens.json
writing: debug/pprof-summary.sh
writing: debug/hot-ranges.sh
diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go
index 3b78d363f6aa..21bb9a3cb0e9 100644
--- a/pkg/clusterversion/key_string.go
+++ b/pkg/clusterversion/key_string.go
@@ -52,11 +52,12 @@ func _() {
_ = x[PriorReadSummaries-41]
_ = x[NonVotingReplicas-42]
_ = x[ProtectedTsMetaPrivilegesMigration-43]
+ _ = x[JoinTokensTable-44]
}
-const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsSequencesRegclassImplicitColumnPartitioningMultiRegionFeaturesClosedTimestampsRaftTransportChangefeedsSupportPrimaryIndexChangesNamespaceTableWithSchemasMigrationForeignKeyRepresentationMigrationPriorReadSummariesNonVotingReplicasProtectedTsMetaPrivilegesMigration"
+const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsSequencesRegclassImplicitColumnPartitioningMultiRegionFeaturesClosedTimestampsRaftTransportChangefeedsSupportPrimaryIndexChangesNamespaceTableWithSchemasMigrationForeignKeyRepresentationMigrationPriorReadSummariesNonVotingReplicasProtectedTsMetaPrivilegesMigrationJoinTokensTable"
-var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 542, 592, 608, 629, 667, 709, 725, 761, 778, 804, 823, 852, 889, 923, 956, 974, 991, 1025}
+var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 542, 592, 608, 629, 667, 709, 725, 761, 778, 804, 823, 852, 889, 923, 956, 974, 991, 1025, 1040}
func (i Key) String() string {
if i < 0 || i >= Key(len(_Key_index)-1) {
diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go
index 9de2714dd8c7..464627a5c8b2 100644
--- a/pkg/sql/catalog/systemschema/system.go
+++ b/pkg/sql/catalog/systemschema/system.go
@@ -363,7 +363,7 @@ CREATE TABLE system.join_tokens (
id UUID NOT NULL PRIMARY KEY,
secret BYTES NOT NULL,
expiration TIMESTAMPTZ NOT NULL,
- FAMILY "primary" (id, secret, expiration),
+ FAMILY "primary" (id, secret, expiration)
)`
)
@@ -1776,7 +1776,7 @@ var (
Version: 1,
Columns: []descpb.ColumnDescriptor{
{Name: "id", ID: 1, Type: types.Uuid, Nullable: false},
- {Name: "secret", ID: 2, Type: types.String, Nullable: false},
+ {Name: "secret", ID: 2, Type: types.Bytes, Nullable: false},
{Name: "expiration", ID: 3, Type: types.TimestampTZ, Nullable: false},
},
NextColumnID: 4,
@@ -1786,7 +1786,7 @@ var (
ID: 0,
ColumnNames: []string{"id", "secret", "expiration"},
ColumnIDs: []descpb.ColumnID{1, 2, 3},
- DefaultColumnID: 2,
+ DefaultColumnID: 0,
},
},
NextFamilyID: 1,
diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_crdb_internal b/pkg/sql/logictest/testdata/logic_test/distsql_crdb_internal
index 48c688916a66..5391a02792d6 100644
--- a/pkg/sql/logictest/testdata/logic_test/distsql_crdb_internal
+++ b/pkg/sql/logictest/testdata/logic_test/distsql_crdb_internal
@@ -45,4 +45,3 @@ SELECT a, count(*) AS cnt FROM data GROUP BY a HAVING crdb_internal.reset_sql_st
3 1000
4 1000
5 1000
-
diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table
index 6132d9f06c4a..12f5be64f03c 100644
--- a/pkg/sql/logictest/testdata/logic_test/grant_table
+++ b/pkg/sql/logictest/testdata/logic_test/grant_table
@@ -795,6 +795,16 @@ system public migrations root GRANT
system public migrations root INSERT
system public migrations root SELECT
system public migrations root UPDATE
+system public join_tokens admin DELETE
+system public join_tokens admin GRANT
+system public join_tokens admin INSERT
+system public join_tokens admin SELECT
+system public join_tokens admin UPDATE
+system public join_tokens root DELETE
+system public join_tokens root GRANT
+system public join_tokens root INSERT
+system public join_tokens root SELECT
+system public join_tokens root UPDATE
a pg_extension NULL admin ALL
a pg_extension NULL readwrite ALL
a pg_extension NULL root ALL
@@ -1173,6 +1183,11 @@ system public jobs root GRA
system public jobs root INSERT
system public jobs root SELECT
system public jobs root UPDATE
+system public join_tokens root DELETE
+system public join_tokens root GRANT
+system public join_tokens root INSERT
+system public join_tokens root SELECT
+system public join_tokens root UPDATE
system public lease root DELETE
system public lease root GRANT
system public lease root INSERT
diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema
index b38146e6eb4c..9044460eeb35 100755
--- a/pkg/sql/logictest/testdata/logic_test/information_schema
+++ b/pkg/sql/logictest/testdata/logic_test/information_schema
@@ -877,6 +877,7 @@ system public statement_diagnostics BASE T
system public scheduled_jobs BASE TABLE YES 1
system public sqlliveness BASE TABLE YES 1
system public migrations BASE TABLE YES 1
+system public join_tokens BASE TABLE YES 1
statement ok
ALTER TABLE other_db.xyz ADD COLUMN j INT
@@ -952,6 +953,10 @@ system public 630200280_15_2_not_null system
system public 630200280_15_3_not_null system public jobs CHECK NO NO
system public 630200280_15_4_not_null system public jobs CHECK NO NO
system public primary system public jobs PRIMARY KEY NO NO
+system public 630200280_41_1_not_null system public join_tokens CHECK NO NO
+system public 630200280_41_2_not_null system public join_tokens CHECK NO NO
+system public 630200280_41_3_not_null system public join_tokens CHECK NO NO
+system public primary system public join_tokens PRIMARY KEY NO NO
system public 630200280_11_1_not_null system public lease CHECK NO NO
system public 630200280_11_2_not_null system public lease CHECK NO NO
system public 630200280_11_3_not_null system public lease CHECK NO NO
@@ -1203,6 +1208,9 @@ system public 630200280_40_2_not_null minor IS NOT NU
system public 630200280_40_3_not_null patch IS NOT NULL
system public 630200280_40_4_not_null internal IS NOT NULL
system public 630200280_40_5_not_null completed_at IS NOT NULL
+system public 630200280_41_1_not_null id IS NOT NULL
+system public 630200280_41_2_not_null secret IS NOT NULL
+system public 630200280_41_3_not_null expiration IS NOT NULL
system public 630200280_4_1_not_null username IS NOT NULL
system public 630200280_4_3_not_null isRole IS NOT NULL
system public 630200280_5_1_not_null id IS NOT NULL
@@ -1226,6 +1234,7 @@ system public descriptor id sy
system public eventlog timestamp system public primary
system public eventlog uniqueID system public primary
system public jobs id system public primary
+system public join_tokens id system public primary
system public lease descID system public primary
system public lease expiration system public primary
system public lease nodeID system public primary
@@ -1392,6 +1401,9 @@ system public jobs id
system public jobs payload 4
system public jobs progress 5
system public jobs status 2
+system public join_tokens expiration 3
+system public join_tokens id 1
+system public join_tokens secret 2
system public lease descID 1
system public lease expiration 4
system public lease nodeID 3
@@ -2178,6 +2190,16 @@ NULL root system public jobs
NULL root system public jobs INSERT NULL NO
NULL root system public jobs SELECT NULL YES
NULL root system public jobs UPDATE NULL NO
+NULL admin system public join_tokens DELETE NULL NO
+NULL admin system public join_tokens GRANT NULL NO
+NULL admin system public join_tokens INSERT NULL NO
+NULL admin system public join_tokens SELECT NULL YES
+NULL admin system public join_tokens UPDATE NULL NO
+NULL root system public join_tokens DELETE NULL NO
+NULL root system public join_tokens GRANT NULL NO
+NULL root system public join_tokens INSERT NULL NO
+NULL root system public join_tokens SELECT NULL YES
+NULL root system public join_tokens UPDATE NULL NO
NULL admin system public lease DELETE NULL NO
NULL admin system public lease GRANT NULL NO
NULL admin system public lease INSERT NULL NO
@@ -2838,6 +2860,16 @@ NULL root system public migrations
NULL root system public migrations INSERT NULL NO
NULL root system public migrations SELECT NULL YES
NULL root system public migrations UPDATE NULL NO
+NULL admin system public join_tokens DELETE NULL NO
+NULL admin system public join_tokens GRANT NULL NO
+NULL admin system public join_tokens INSERT NULL NO
+NULL admin system public join_tokens SELECT NULL YES
+NULL admin system public join_tokens UPDATE NULL NO
+NULL root system public join_tokens DELETE NULL NO
+NULL root system public join_tokens GRANT NULL NO
+NULL root system public join_tokens INSERT NULL NO
+NULL root system public join_tokens SELECT NULL YES
+NULL root system public join_tokens UPDATE NULL NO
statement ok
CREATE TABLE other_db.xyz (i INT)
diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog
index cf5b47676d29..24636272b5ad 100644
--- a/pkg/sql/logictest/testdata/logic_test/pg_catalog
+++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog
@@ -847,6 +847,7 @@ indexrelid indrelid indnatts indisunique indisprimary indisexclusion indim
543291289 23 1 false false false false false true false false true false 2 3403232968 0 2 NULL NULL 1
543291291 23 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2
803027558 26 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3
+923576837 41 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1
1062763829 25 4 true true false true false true false false true false 1 2 3 4 0 0 3403232968 3403232968 0 0 0 0 2 2 2 2 NULL NULL 4
1276104432 12 2 true true false true false true false false true false 1 6 0 0 0 0 2 2 NULL NULL 2
1322500096 28 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1
@@ -898,6 +899,7 @@ indexrelid operator_argument_type_oid operator_argument_position
803027558 0 1
803027558 0 2
803027558 0 3
+923576837 0 1
1062763829 0 1
1062763829 0 2
1062763829 0 3
diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges
index c0151119790a..f4491f49887f 100644
--- a/pkg/sql/logictest/testdata/logic_test/ranges
+++ b/pkg/sql/logictest/testdata/logic_test/ranges
@@ -310,7 +310,8 @@ start_key start_pretty end_key
[173] /Table/37 [174] /Table/38 system scheduled_jobs · {1} 1
[174] /Table/38 [175] /Table/39 · · · {1} 1
[175] /Table/39 [176] /Table/40 system sqlliveness · {1} 1
-[176] /Table/40 [189 137] /Table/53/1 system migrations · {1} 1
+[176] /Table/40 [177] /Table/41 system migrations · {1} 1
+[177] /Table/41 [189 137] /Table/53/1 system join_tokens · {1} 1
[189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1
[189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3
[189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1
@@ -369,7 +370,8 @@ start_key start_pretty end_key
[173] /Table/37 [174] /Table/38 system scheduled_jobs · {1} 1
[174] /Table/38 [175] /Table/39 · · · {1} 1
[175] /Table/39 [176] /Table/40 system sqlliveness · {1} 1
-[176] /Table/40 [189 137] /Table/53/1 system migrations · {1} 1
+[176] /Table/40 [177] /Table/41 system migrations · {1} 1
+[177] /Table/41 [189 137] /Table/53/1 system join_tokens · {1} 1
[189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1
[189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3
[189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1
diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source
index 2d2842731c8f..2b24e7ec98c3 100644
--- a/pkg/sql/logictest/testdata/logic_test/show_source
+++ b/pkg/sql/logictest/testdata/logic_test/show_source
@@ -199,6 +199,7 @@ SELECT * FROM [SHOW TABLES FROM system]
----
schema_name table_name type owner estimated_row_count locality
public namespace table NULL 0 NULL
+public join_tokens table NULL 0 NULL
public migrations table NULL 0 NULL
public sqlliveness table NULL 0 NULL
public scheduled_jobs table NULL 0 NULL
@@ -234,6 +235,7 @@ SELECT * FROM [SHOW TABLES FROM system WITH COMMENT]
----
schema_name table_name type owner estimated_row_count locality comment
public namespace table NULL 0 NULL ·
+public join_tokens table NULL 0 NULL ·
public migrations table NULL 0 NULL ·
public sqlliveness table NULL 0 NULL ·
public scheduled_jobs table NULL 0 NULL ·
diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system
index fbfce8af9d83..623b4ff2cacc 100644
--- a/pkg/sql/logictest/testdata/logic_test/system
+++ b/pkg/sql/logictest/testdata/logic_test/system
@@ -14,6 +14,7 @@ public comments table NULL 0 NULL
public descriptor table NULL 0 NULL
public eventlog table NULL 0 NULL
public jobs table NULL 0 NULL
+public join_tokens table NULL 0 NULL
public lease table NULL 0 NULL
public locations table NULL 0 NULL
public migrations table NULL 0 NULL
@@ -75,6 +76,7 @@ SELECT id FROM system.descriptor
37
39
40
+41
50
51
52
@@ -228,6 +230,16 @@ system public jobs root GRANT
system public jobs root INSERT
system public jobs root SELECT
system public jobs root UPDATE
+system public join_tokens admin DELETE
+system public join_tokens admin GRANT
+system public join_tokens admin INSERT
+system public join_tokens admin SELECT
+system public join_tokens admin UPDATE
+system public join_tokens root DELETE
+system public join_tokens root GRANT
+system public join_tokens root INSERT
+system public join_tokens root SELECT
+system public join_tokens root UPDATE
system public lease admin DELETE
system public lease admin GRANT
system public lease admin INSERT
diff --git a/pkg/sql/logictest/testdata/logic_test/system_namespace b/pkg/sql/logictest/testdata/logic_test/system_namespace
index 6291b9c053f3..9a2b2c16a3f9 100644
--- a/pkg/sql/logictest/testdata/logic_test/system_namespace
+++ b/pkg/sql/logictest/testdata/logic_test/system_namespace
@@ -16,6 +16,7 @@ SELECT * FROM system.namespace
1 29 descriptor 3
1 29 eventlog 12
1 29 jobs 15
+1 29 join_tokens 41
1 29 lease 11
1 29 locations 21
1 29 migrations 40
diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic
index 0a016b360620..099b5aa839fa 100644
--- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic
+++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic
@@ -41,7 +41,7 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 CPut, 1 EndTxn to (n1,s1):1
# Multi-row insert should auto-commit.
query B
@@ -62,7 +62,7 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 2 CPut, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 CPut, 1 EndTxn to (n1,s1):1
# No auto-commit inside a transaction.
statement ok
@@ -86,7 +86,7 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
statement ok
ROLLBACK
@@ -110,8 +110,8 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 2 CPut, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 2 CPut, 1 EndTxn to (n1,s1):1
# TODO(radu): allow non-side-effecting projections.
query B
@@ -133,9 +133,9 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
# Insert with RETURNING statement with side-effects should not auto-commit.
# In this case division can (in principle) error out.
@@ -158,9 +158,9 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
# Another way to test the scenario above: generate an error and ensure that the
# mutation was not committed.
@@ -195,8 +195,8 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 1 Put, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 1 Put, 1 EndTxn to (n1,s1):1
# Multi-row upsert should auto-commit.
query B
@@ -217,8 +217,8 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 2 Put, 1 EndTxn to (n1,s1):1
# No auto-commit inside a transaction.
statement ok
@@ -242,7 +242,7 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
statement ok
ROLLBACK
@@ -266,8 +266,8 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 2 Put, 1 EndTxn to (n1,s1):1
# TODO(radu): allow non-side-effecting projections.
query B
@@ -289,9 +289,9 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
# Upsert with RETURNING statement with side-effects should not auto-commit.
# In this case division can (in principle) error out.
@@ -314,9 +314,9 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
# Another way to test the scenario above: generate an error and ensure that the
# mutation was not committed.
@@ -351,9 +351,9 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put, 1 EndTxn to (n1,s1):1
# No auto-commit inside a transaction.
statement ok
@@ -377,8 +377,8 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
statement ok
ROLLBACK
@@ -402,10 +402,10 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put, 1 EndTxn to (n1,s1):1
# TODO(radu): allow non-side-effecting projections.
query B
@@ -427,11 +427,11 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
# Update with RETURNING statement with side-effects should not auto-commit.
# In this case division can (in principle) error out.
@@ -454,11 +454,11 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
# Another way to test the scenario above: generate an error and ensure that the
# mutation was not committed.
@@ -493,9 +493,9 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1
# Multi-row delete should auto-commit.
query B
@@ -516,9 +516,9 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Put to (n1,s1):1
-dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Put to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1
# No auto-commit inside a transaction.
statement ok
@@ -542,7 +542,7 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
statement ok
ROLLBACK
@@ -566,9 +566,9 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%PushTxn%'
AND message NOT LIKE '%QueryTxn%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Del, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Del, 1 EndTxn to (n1,s1):1
# TODO(radu): allow non-side-effecting projections.
query B
@@ -590,10 +590,10 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Del to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Del to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
# Insert with RETURNING statement with side-effects should not auto-commit.
# In this case division can (in principle) error out.
@@ -616,10 +616,10 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 2 Del to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 2 Del to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
statement ok
INSERT INTO ab VALUES (12, 0);
@@ -666,10 +666,10 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 2 Scan to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 2 Scan to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
query B
SELECT count(*) > 0 FROM [
@@ -690,11 +690,11 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 1 Put to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 1 Put to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
query B
SELECT count(*) > 0 FROM [
@@ -716,11 +716,11 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 1 Del to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 1 Del to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
# Test with a single cascade, which should use autocommit.
statement ok
@@ -744,10 +744,10 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 1 Scan to (n1,s1):1
-dist sender send r36: sending batch 1 Del, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 Scan to (n1,s1):1
+dist sender send r37: sending batch 1 Del, 1 EndTxn to (n1,s1):1
# -----------------------
# Multiple mutation tests
@@ -775,10 +775,10 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
query B
SELECT count(*) > 0 FROM [
@@ -801,7 +801,7 @@ WHERE message LIKE '%r$rangeid: sending batch%'
AND message NOT LIKE '%QueryTxn%'
AND operation NOT LIKE '%async%'
----
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 2 CPut to (n1,s1):1
-dist sender send r36: sending batch 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 2 CPut to (n1,s1):1
+dist sender send r37: sending batch 1 EndTxn to (n1,s1):1
diff --git a/pkg/sql/opt/exec/execbuilder/testdata/delete b/pkg/sql/opt/exec/execbuilder/testdata/delete
index 752dd4fb9228..74ad2218c2b0 100644
--- a/pkg/sql/opt/exec/execbuilder/testdata/delete
+++ b/pkg/sql/opt/exec/execbuilder/testdata/delete
@@ -231,9 +231,9 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION]
WHERE message LIKE '%DelRange%' OR message LIKE '%DelRng%'
----
flow DelRange /Table/57/1 - /Table/57/2
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
flow DelRange /Table/57/1/601/0 - /Table/57/2
-dist sender send r36: sending batch 1 DelRng to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng to (n1,s1):1
# Ensure that DelRange requests are autocommitted when DELETE FROM happens on a
# chunk of fewer than 600 keys.
@@ -249,7 +249,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION]
WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%'
----
flow DelRange /Table/57/1/5 - /Table/57/1/5/#
-dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1
+dist sender send r37: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1
# Test use of fast path when there are interleaved tables.
diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic
index 9a620c6cce5b..91ca597752cc 100644
--- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic
+++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic
@@ -238,7 +238,7 @@ SET tracing = on; INSERT INTO t.kv3 (k, v) VALUES (1,1); SET tracing = off
query T
SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE e'%1 CPut, 1 EndTxn%' AND message NOT LIKE e'%proposing command%'
----
-r37: sending batch 1 CPut, 1 EndTxn to (n1,s1):1
+r38: sending batch 1 CPut, 1 EndTxn to (n1,s1):1
node received request: 1 CPut, 1 EndTxn
# Temporarily disabled flaky test (#58202).
diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go
index 0ee9586276ea..c19553e3de38 100644
--- a/pkg/sql/pgwire/pgwire_test.go
+++ b/pkg/sql/pgwire/pgwire_test.go
@@ -556,7 +556,7 @@ func TestPGPreparedQuery(t *testing.T) {
baseTest.Results("users", "primary", false, 1, "username", "ASC", false, false),
}},
{"SHOW TABLES FROM system", []preparedQueryTest{
- baseTest.Results("public", "comments", "table", gosql.NullString{}, 0, gosql.NullString{}).Others(29),
+ baseTest.Results("public", "comments", "table", gosql.NullString{}, 0, gosql.NullString{}).Others(30),
}},
{"SHOW SCHEMAS FROM system", []preparedQueryTest{
baseTest.Results("crdb_internal", gosql.NullString{}).Others(4),
diff --git a/pkg/sql/tests/testdata/initial_keys b/pkg/sql/tests/testdata/initial_keys
index fede84a0c496..017850ad88e1 100644
--- a/pkg/sql/tests/testdata/initial_keys
+++ b/pkg/sql/tests/testdata/initial_keys
@@ -1,6 +1,6 @@
initial-keys tenant=system
----
-71 keys:
+73 keys:
/System/"desc-idgen"
/Table/3/1/1/2/1
/Table/3/1/2/2/1
@@ -33,6 +33,7 @@ initial-keys tenant=system
/Table/3/1/37/2/1
/Table/3/1/39/2/1
/Table/3/1/40/2/1
+ /Table/3/1/41/2/1
/Table/5/1/0/2/1
/Table/5/1/1/2/1
/Table/5/1/16/2/1
@@ -46,6 +47,7 @@ initial-keys tenant=system
/NamespaceTable/30/1/1/29/"descriptor"/4/1
/NamespaceTable/30/1/1/29/"eventlog"/4/1
/NamespaceTable/30/1/1/29/"jobs"/4/1
+ /NamespaceTable/30/1/1/29/"join_tokens"/4/1
/NamespaceTable/30/1/1/29/"lease"/4/1
/NamespaceTable/30/1/1/29/"locations"/4/1
/NamespaceTable/30/1/1/29/"migrations"/4/1
@@ -72,7 +74,7 @@ initial-keys tenant=system
/NamespaceTable/30/1/1/29/"users"/4/1
/NamespaceTable/30/1/1/29/"web_sessions"/4/1
/NamespaceTable/30/1/1/29/"zones"/4/1
-30 splits:
+31 splits:
/Table/11
/Table/12
/Table/13
@@ -103,10 +105,11 @@ initial-keys tenant=system
/Table/38
/Table/39
/Table/40
+ /Table/41
initial-keys tenant=5
----
-62 keys:
+64 keys:
/Tenant/5/Table/3/1/1/2/1
/Tenant/5/Table/3/1/2/2/1
/Tenant/5/Table/3/1/3/2/1
@@ -137,6 +140,7 @@ initial-keys tenant=5
/Tenant/5/Table/3/1/37/2/1
/Tenant/5/Table/3/1/39/2/1
/Tenant/5/Table/3/1/40/2/1
+ /Tenant/5/Table/3/1/41/2/1
/Tenant/5/Table/7/1/0/0
/Tenant/5/NamespaceTable/30/1/0/0/"system"/4/1
/Tenant/5/NamespaceTable/30/1/1/0/"public"/4/1
@@ -145,6 +149,7 @@ initial-keys tenant=5
/Tenant/5/NamespaceTable/30/1/1/29/"descriptor_id_seq"/4/1
/Tenant/5/NamespaceTable/30/1/1/29/"eventlog"/4/1
/Tenant/5/NamespaceTable/30/1/1/29/"jobs"/4/1
+ /Tenant/5/NamespaceTable/30/1/1/29/"join_tokens"/4/1
/Tenant/5/NamespaceTable/30/1/1/29/"lease"/4/1
/Tenant/5/NamespaceTable/30/1/1/29/"locations"/4/1
/Tenant/5/NamespaceTable/30/1/1/29/"migrations"/4/1
@@ -174,7 +179,7 @@ initial-keys tenant=5
initial-keys tenant=999
----
-62 keys:
+64 keys:
/Tenant/999/Table/3/1/1/2/1
/Tenant/999/Table/3/1/2/2/1
/Tenant/999/Table/3/1/3/2/1
@@ -205,6 +210,7 @@ initial-keys tenant=999
/Tenant/999/Table/3/1/37/2/1
/Tenant/999/Table/3/1/39/2/1
/Tenant/999/Table/3/1/40/2/1
+ /Tenant/999/Table/3/1/41/2/1
/Tenant/999/Table/7/1/0/0
/Tenant/999/NamespaceTable/30/1/0/0/"system"/4/1
/Tenant/999/NamespaceTable/30/1/1/0/"public"/4/1
@@ -213,6 +219,7 @@ initial-keys tenant=999
/Tenant/999/NamespaceTable/30/1/1/29/"descriptor_id_seq"/4/1
/Tenant/999/NamespaceTable/30/1/1/29/"eventlog"/4/1
/Tenant/999/NamespaceTable/30/1/1/29/"jobs"/4/1
+ /Tenant/999/NamespaceTable/30/1/1/29/"join_tokens"/4/1
/Tenant/999/NamespaceTable/30/1/1/29/"lease"/4/1
/Tenant/999/NamespaceTable/30/1/1/29/"locations"/4/1
/Tenant/999/NamespaceTable/30/1/1/29/"migrations"/4/1
|