Skip to content

Commit

Permalink
slstorage: support regional by row system.sqlliveness
Browse files Browse the repository at this point in the history
The COCKROACH_MR_SYSTEM_DATABASE environment variable was introduced to
control features added as part of #85736. The variable will be removed
once migrations and version gates are implemented for the multi-region
system database optimizations.

The sqlliveness table now supports an index format compatible with
regional by row tables. Multi region serverless is the motivation for
this change. When a sql server starts up, it must write its session to
the sqlliveness table. The remote session write can add ~400ms to a
servers startup time.

```
$ COCKROACH_MR_SYSTEM_DATABASE=1 ./cockroach-short demo
[email protected]:26257/movr> SELECT * FROM system.sqlliveness;
  crdb_region |            session_uuid            |           expiration
--------------+------------------------------------+---------------------------------
  \x80        | \x5895d5d121984d86bcb81d6a89d5635d | 1667511355275484444.0000000000

$ COCKROACH_MR_SYSTEM_DATABASE=0 ./cockroach-short demo
[email protected]:26257/movr> select * from system.sqlliveness;
                 session_id                |           expiration
-------------------------------------------+---------------------------------
  \x0101805fc92bdb0e8d4cb4aec3e1527a3052b8 | 1667511392212781946.0000000000
```

Part of #85736

Release note: None
  • Loading branch information
jeffswenson committed Nov 4, 2022
1 parent e01f162 commit 15f6438
Show file tree
Hide file tree
Showing 10 changed files with 535 additions and 111 deletions.
8 changes: 8 additions & 0 deletions pkg/ccl/multiregionccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ go_test(
srcs = [
"datadriven_test.go",
"main_test.go",
"multiregion_system_table_test.go",
"multiregion_test.go",
"region_test.go",
"regional_by_row_test.go",
Expand Down Expand Up @@ -62,10 +63,12 @@ go_test(
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/desctestutils",
"//pkg/sql/enum",
"//pkg/sql/execinfra",
"//pkg/sql/parser",
"//pkg/sql/rowenc",
"//pkg/sql/sem/tree",
"//pkg/sql/sqlliveness/slstorage",
"//pkg/sql/sqltestutils",
"//pkg/sql/tests",
"//pkg/testutils",
Expand All @@ -74,12 +77,17 @@ go_test(
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/envutil",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/tracing",
"//pkg/util/tracing/tracingpb",
"//pkg/util/uuid",
"@com_github_cockroachdb_apd_v3//:apd",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
Expand Down
121 changes: 121 additions & 0 deletions pkg/ccl/multiregionccl/multiregion_system_table_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
// Copyright 2022 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package multiregionccl

import (
"context"
"fmt"
"testing"
"time"

"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl/multiregionccltestutils"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/enum"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slstorage"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/stretchr/testify/require"
)

func createSqllivenessTable(
t *testing.T, db *sqlutils.SQLRunner, dbName string,
) (tableID descpb.ID) {
t.Helper()
db.Exec(t, fmt.Sprintf(`
CREATE DATABASE IF NOT EXISTS "%s"
WITH PRIMARY REGION "us-east1"
REGIONS "us-east1", "us-east2", "us-east3"
`, dbName))

// expiration needs to be column 2. slstorage.Table assumes the column id.
// session_uuid and crdb_region are identified by their location in the
// primary key.
db.Exec(t, fmt.Sprintf(`
CREATE TABLE "%s".sqlliveness (
session_uuid BYTES NOT NULL,
expiration DECIMAL NOT NULL,
crdb_region "%s".public.crdb_internal_region,
PRIMARY KEY(crdb_region, session_uuid)
) LOCALITY REGIONAL BY ROW;
`, dbName, dbName))
db.QueryRow(t, `
select u.id
from system.namespace t
join system.namespace u
on t.id = u."parentID"
where t.name = $1 and u.name = $2`,
dbName, "sqlliveness").Scan(&tableID)
return tableID
}

func TestRbrSqllivenessTable(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
defer envutil.TestSetEnv(t, "COCKROACH_MR_SYSTEM_DATABASE", "1")()

ctx := context.Background()

cluster, sqlDB, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster(t, 3, base.TestingKnobs{})
defer cleanup()
kvDB := cluster.Servers[0].DB()
settings := cluster.Servers[0].Cfg.Settings
stopper := cluster.Servers[0].Stopper()

tDB := sqlutils.MakeSQLRunner(sqlDB)

t0 := time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC)
timeSource := timeutil.NewManualTime(t0)
clock := hlc.NewClock(timeSource, base.DefaultMaxClockOffset)

setup := func(t *testing.T) *slstorage.Storage {
dbName := t.Name()
tableID := createSqllivenessTable(t, tDB, dbName)
var ambientCtx log.AmbientContext
// rbrIndexID is the index id used to access the regional by row index in
// tests. In production it will be index 2, but the freshly created test table
// will have index 1.
const rbrIndexID = 1
return slstorage.NewTestingStorage(ambientCtx, stopper, clock, kvDB, keys.SystemSQLCodec, settings,
tableID, rbrIndexID, timeSource.NewTimer)
}

t.Run("SqlRead", func(t *testing.T) {
storage := setup(t)

initialUUID := uuid.MakeV4()
session, err := slstorage.MakeSessionID(enum.One, initialUUID)
require.NoError(t, err)

writeExpiration := clock.Now().Add(10, 00)
require.NoError(t, storage.Insert(ctx, session, writeExpiration))

var sessionUUID string
var crdbRegion string
var rawExpiration apd.Decimal

row := tDB.QueryRow(t, fmt.Sprintf(`SELECT crdb_region, session_uuid, expiration FROM "%s".sqlliveness`, t.Name()))
row.Scan(&crdbRegion, &sessionUUID, &rawExpiration)

require.Contains(t, []string{"us-east1", "us-east2", "us-east3"}, crdbRegion)
require.Equal(t, sessionUUID, string(initialUUID.GetBytes()))

readExpiration, err := hlc.DecimalToHLC(&rawExpiration)
require.NoError(t, err)

require.Equal(t, writeExpiration, readExpiration)
})
}
1 change: 1 addition & 0 deletions pkg/sql/catalog/systemschema/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ go_library(
"//pkg/sql/sem/catconstants",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util/envutil",
"//pkg/util/log",
],
)
Expand Down
95 changes: 75 additions & 20 deletions pkg/sql/catalog/systemschema/system.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

Expand Down Expand Up @@ -460,7 +461,16 @@ CREATE TABLE system.sqlliveness (
session_id BYTES NOT NULL,
expiration DECIMAL NOT NULL,
CONSTRAINT "primary" PRIMARY KEY (session_id),
FAMILY fam0_session_id_expiration (session_id, expiration)
FAMILY fam0_session_id_expiration (session_id, expiration)
)`

MrSqllivenessTableSchema = `
CREATE TABLE system.sqlliveness (
session_uuid BYTES NOT NULL,
expiration DECIMAL NOT NULL,
crdb_region BYTES NOT NULL,
CONSTRAINT "primary" PRIMARY KEY (crdb_region, session_uuid),
FAMILY "primary" (crdb_region, session_uuid, expiration)
)`

MigrationsTableSchema = `
Expand Down Expand Up @@ -1996,26 +2006,58 @@ var (
))

// SqllivenessTable is the descriptor for the sqlliveness table.
SqllivenessTable = registerSystemTable(
SqllivenessTableSchema,
systemTable(
catconstants.SqllivenessTableName,
keys.SqllivenessID,
[]descpb.ColumnDescriptor{
{Name: "session_id", ID: 1, Type: types.Bytes, Nullable: false},
{Name: "expiration", ID: 2, Type: types.Decimal, Nullable: false},
},
[]descpb.ColumnFamilyDescriptor{
{
Name: "fam0_session_id_expiration",
ID: 0,
ColumnNames: []string{"session_id", "expiration"},
ColumnIDs: []descpb.ColumnID{1, 2},
DefaultColumnID: 2,
SqllivenessTable = func() catalog.TableDescriptor {
if SupportMultiRegion() {
return registerSystemTable(
MrSqllivenessTableSchema,
systemTable(
catconstants.SqllivenessTableName,
keys.SqllivenessID,
[]descpb.ColumnDescriptor{
{Name: "crdb_region", ID: 4, Type: types.Bytes, Nullable: false},
{Name: "session_uuid", ID: 3, Type: types.Bytes, Nullable: false},
{Name: "expiration", ID: 2, Type: types.Decimal, Nullable: false},
},
[]descpb.ColumnFamilyDescriptor{
{
Name: "primary",
ID: 0,
ColumnNames: []string{"crdb_region", "session_uuid", "expiration"},
ColumnIDs: []descpb.ColumnID{4, 3, 2},
DefaultColumnID: 2,
},
},
descpb.IndexDescriptor{
Name: "primary",
ID: 2,
Unique: true,
KeyColumnNames: []string{"crdb_region", "session_uuid"},
KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC, catpb.IndexColumn_ASC},
KeyColumnIDs: []descpb.ColumnID{4, 3},
},
))
}
return registerSystemTable(
SqllivenessTableSchema,
systemTable(
catconstants.SqllivenessTableName,
keys.SqllivenessID,
[]descpb.ColumnDescriptor{
{Name: "session_id", ID: 1, Type: types.Bytes, Nullable: false},
{Name: "expiration", ID: 2, Type: types.Decimal, Nullable: false},
},
},
pk("session_id"),
))
[]descpb.ColumnFamilyDescriptor{
{
Name: "fam0_session_id_expiration",
ID: 0,
ColumnNames: []string{"session_id", "expiration"},
ColumnIDs: []descpb.ColumnID{1, 2},
DefaultColumnID: 2,
},
},
pk("session_id"),
))
}()

// MigrationsTable is the descriptor for the migrations table. It stores facts
// about the completion state of long-running migrations. It is used to
Expand Down Expand Up @@ -2638,3 +2680,16 @@ func IsUnleasableSystemDescriptorByName(

// SpanConfigurationsTableName represents system.span_configurations.
var SpanConfigurationsTableName = tree.NewTableNameWithSchema("system", tree.PublicSchemaName, tree.Name(catconstants.SpanConfigurationsTableName))

// SupportMultiRegion returns true if the cluster should support multi-region
// optimized system databases.
//
// TODO(jeffswenson): remove SupportMultiRegion after implementing migrations
// and version gates to migrate to the new regional by row compatible schemas.
func SupportMultiRegion() bool {
val, ok := envutil.EnvString("COCKROACH_MR_SYSTEM_DATABASE", 0)
if !ok {
return false
}
return val == "1"
}
8 changes: 6 additions & 2 deletions pkg/sql/sqlliveness/slstorage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "slstorage",
srcs = [
"key_encoder.go",
"metrics.go",
"sessionid.go",
"slstorage.go",
Expand All @@ -18,7 +19,8 @@ go_library(
"//pkg/roachpb",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/sem/catid",
"//pkg/sql/sem/eval",
"//pkg/sql/sqlliveness",
"//pkg/util/cache",
Expand All @@ -42,13 +44,14 @@ go_test(
name = "slstorage_test",
size = "small",
srcs = [
"key_encoder_test.go",
"main_test.go",
"sessionid_test.go",
"slstorage_test.go",
],
args = ["-test.timeout=55s"],
embed = [":slstorage"],
deps = [
":slstorage",
"//pkg/base",
"//pkg/keys",
"//pkg/kv/kvserver",
Expand All @@ -65,6 +68,7 @@ go_test(
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/envutil",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
Expand Down
Loading

0 comments on commit 15f6438

Please sign in to comment.