Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
104350: sql/schemachanger: support for create sequence inside the declarative schema changer r=fqazi a=fqazi

This patch implements CREATE SEQUENCE in the declarative schema changer. The first three commits can be ignored and are included in (#104348, which should be reviewed and merged first). This patch will:

- Skip validation/backfill for descriptors in an added state
- Fixes prefix resolution creating new objects, where two-part names were not handled properly before
- Adds support for CREATE sequence in the declarative schema changer

Fixes: #104351

105454: roachtest: update `mixedversion` to always use secure clusters r=srosenberg a=renatolabs

Secure clusters are closer to production deployments and also allow
us to tests features that we couldn't before, like creating new users
with passwords during a test, and then performing SQL operations with
those users.

In the process of getting this to work, there were a few bugs that needed
to be fixed (first commit), and the `cluster` interface needed a small update
as well (second commit).

Epic: CRDB-19321

Release note: None

105765: ui: fix db page Node/Regions column rendering r=xinhaoz a=xinhaoz

Previously, the db page was not updating its columns if the
`showNodeRegionsColumn` prop changed. The db details page was
also not filtering out the regions column when desired.

Epic: none

Release note (bug fix): node/regions columns in db and db details
page should properly render. This column is hidden for tenants and
otherwise is shown for clusters with > 1 node.

105770: persistedsqlstats: skip TestSQLStatsPersistedLimitReached under stress r=zachlite a=zachlite

TestSQLStatsPersistedLimitReached succeeds under 'normal' testing conditions. We can and should get this test coverage when we can.

Informs #97488
Epic: none
Release note: None

105799: kv: expose env var to configure raft entry cache size r=erikgrinaker a=nvanbenschoten

Informs #98666.

This commit introduces a new `COCKROACH_RAFT_ENTRY_CACHE_SIZE` which can be used to configure the size of the raft entry cache. The default value is 16 MiB.

Release note: None

Co-authored-by: Faizan Qazi <[email protected]>
Co-authored-by: Renato Costa <[email protected]>
Co-authored-by: Xin Hao Zhang <[email protected]>
Co-authored-by: zachlite <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
6 people committed Jun 29, 2023
6 parents 0736712 + 74edfa9 + c2eaf1f + 039ac80 + bc70750 + c309edc commit 2b66f95
Show file tree
Hide file tree
Showing 124 changed files with 3,177 additions and 571 deletions.
5 changes: 5 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

5 changes: 5 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_mixed_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

31 changes: 17 additions & 14 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -1201,7 +1201,7 @@ func attachToExistingCluster(
}
if !opt.skipWipe {
if clusterWipe {
if err := c.WipeE(ctx, l, c.All()); err != nil {
if err := c.WipeE(ctx, l, false /* preserveCerts */, c.All()); err != nil {
return nil, err
}
} else {
Expand Down Expand Up @@ -1499,10 +1499,13 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger, dest
// Ignore the files in the the log directory; we pull the logs separately anyway
// so this would only cause duplication.
excludeFiles := "*.log,*.txt,*.pprof"
cmd := fmt.Sprintf(
"%s debug zip --include-range-info --exclude-files='%s' --url {pgurl:%d} %s",
defaultCockroachPath, excludeFiles, i, zipName,
)
cmd := roachtestutil.NewCommand("%s debug zip", defaultCockroachPath).
Option("include-range-info").
Flag("exclude-files", fmt.Sprintf("'%s'", excludeFiles)).
Flag("url", fmt.Sprintf("{pgurl:%d}", i)).
MaybeFlag(c.IsSecure(), "certs-dir", "certs").
Arg(zipName).
String()
if err := c.RunE(ctx, c.Node(i), cmd); err != nil {
l.Printf("%s debug zip failed on node %d: %v", defaultCockroachPath, i, err)
if i < c.spec.NodeCount {
Expand Down Expand Up @@ -2096,10 +2099,6 @@ func (c *clusterImpl) StartE(
if ctx.Err() != nil {
return errors.Wrap(ctx.Err(), "cluster.StartE")
}
// If the test failed (indicated by a canceled ctx), short-circuit.
if ctx.Err() != nil {
return ctx.Err()
}
c.setStatusForClusterOpt("starting", startOpts.RoachtestOpts.Worker, opts...)
defer c.clearStatusForClusterOpt(startOpts.RoachtestOpts.Worker)

Expand Down Expand Up @@ -2134,7 +2133,9 @@ func (c *clusterImpl) StartE(
return err
}

if settings.Secure {
// Do not refetch certs if that step already happened once (i.e., we
// are restarting a node).
if settings.Secure && c.localCertsDir == "" {
if err := c.RefetchCertsFromNode(ctx, 1); err != nil {
return err
}
Expand Down Expand Up @@ -2248,7 +2249,9 @@ func (c *clusterImpl) Signal(

// WipeE wipes a subset of the nodes in a cluster. See cluster.Start() for a
// description of the nodes parameter.
func (c *clusterImpl) WipeE(ctx context.Context, l *logger.Logger, nodes ...option.Option) error {
func (c *clusterImpl) WipeE(
ctx context.Context, l *logger.Logger, preserveCerts bool, nodes ...option.Option,
) error {
if ctx.Err() != nil {
return errors.Wrap(ctx.Err(), "cluster.WipeE")
}
Expand All @@ -2258,16 +2261,16 @@ func (c *clusterImpl) WipeE(ctx context.Context, l *logger.Logger, nodes ...opti
}
c.setStatusForClusterOpt("wiping", false, nodes...)
defer c.clearStatusForClusterOpt(false)
return roachprod.Wipe(ctx, l, c.MakeNodes(nodes...), false /* preserveCerts */)
return roachprod.Wipe(ctx, l, c.MakeNodes(nodes...), preserveCerts)
}

// Wipe is like WipeE, except instead of returning an error, it does
// c.t.Fatal(). c.t needs to be set.
func (c *clusterImpl) Wipe(ctx context.Context, nodes ...option.Option) {
func (c *clusterImpl) Wipe(ctx context.Context, preserveCerts bool, nodes ...option.Option) {
if ctx.Err() != nil {
return
}
if err := c.WipeE(ctx, c.l, nodes...); err != nil {
if err := c.WipeE(ctx, c.l, preserveCerts, nodes...); err != nil {
c.t.Fatal(err)
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/cluster/cluster_interface.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,8 +115,8 @@ type Cluster interface {

// Deleting CockroachDB data and logs on nodes.

WipeE(ctx context.Context, l *logger.Logger, opts ...option.Option) error
Wipe(ctx context.Context, opts ...option.Option)
WipeE(ctx context.Context, l *logger.Logger, preserveCerts bool, opts ...option.Option) error
Wipe(ctx context.Context, preserveCerts bool, opts ...option.Option)

// Internal niche tools.

Expand Down
15 changes: 9 additions & 6 deletions pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,17 +137,17 @@ func InstallFixtures(
return nil
}

// StartWithBinary starts a cockroach binary, assumed to already be
// present in the nodes in the path given.
func StartWithBinary(
// StartWithSettings starts cockroach and constructs settings according
// to the setting options passed.
func StartWithSettings(
ctx context.Context,
l *logger.Logger,
c cluster.Cluster,
nodes option.NodeListOption,
binaryPath string,
startOpts option.StartOpts,
opts ...install.ClusterSettingOption,
) error {
settings := install.MakeClusterSettings(install.BinaryOption(binaryPath))
settings := install.MakeClusterSettings(opts...)
return c.StartE(ctx, l, startOpts, settings, nodes)
}

Expand All @@ -171,6 +171,7 @@ func RestartNodesWithNewBinary(
nodes option.NodeListOption,
startOpts option.StartOpts,
newVersion string,
settings ...install.ClusterSettingOption,
) error {
// NB: We could technically stage the binary on all nodes before
// restarting each one, but on Unix it's invalid to write to an
Expand Down Expand Up @@ -200,7 +201,9 @@ func RestartNodesWithNewBinary(
if err != nil {
return err
}
if err := StartWithBinary(ctx, l, c, c.Node(node), binary, startOpts); err != nil {
if err := StartWithSettings(
ctx, l, c, c.Node(node), startOpts, append(settings, install.BinaryOption(binary))...,
); err != nil {
return err
}

Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_library(
"//pkg/cmd/roachtest/roachtestutil/clusterupgrade",
"//pkg/cmd/roachtest/test",
"//pkg/roachpb",
"//pkg/roachprod/install",
"//pkg/roachprod/logger",
"//pkg/util/ctxgroup",
"//pkg/util/randutil",
Expand Down
15 changes: 14 additions & 1 deletion pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/version"
Expand Down Expand Up @@ -114,6 +115,13 @@ var (
possibleDelaysMs = []int{
0, 50, 100, 200, 500,
}

// defaultClusterSettings is the set of cluster settings always
// passed to `clusterupgrade.StartWithSettings` when (re)starting
// nodes in a cluster.
defaultClusterSettings = []install.ClusterSettingOption{
install.SecureOption(true),
}
)

type (
Expand Down Expand Up @@ -483,7 +491,11 @@ func (s startFromCheckpointStep) Run(

startOpts := option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.Sequential = false
return clusterupgrade.StartWithBinary(ctx, l, c, s.crdbNodes, binaryPath, startOpts)
clusterSettings := append(
append([]install.ClusterSettingOption{}, defaultClusterSettings...),
install.BinaryOption(binaryPath),
)
return clusterupgrade.StartWithSettings(ctx, l, c, s.crdbNodes, startOpts, clusterSettings...)
}

// uploadCurrentVersionStep uploads the current cockroach binary to
Expand Down Expand Up @@ -606,6 +618,7 @@ func (s restartWithNewBinaryStep) Run(
// scheduled backup if necessary.
option.DefaultStartOptsNoBackups(),
s.version,
defaultClusterSettings...,
)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -587,7 +587,7 @@ func (r *testRunner) runWorker(
if c != nil && testToRun.canReuseCluster {
err = func() error {
l.PrintfCtx(ctx, "Using existing cluster: %s (arch=%q). Wiping", c.name, c.arch)
if err := c.WipeE(ctx, l); err != nil {
if err := c.WipeE(ctx, l, false /* preserveCerts */); err != nil {
return err
}
if err := c.RunE(ctx, c.All(), "rm -rf "+perfArtifactsDir); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/allocation_bench.go
Original file line number Diff line number Diff line change
Expand Up @@ -323,7 +323,7 @@ func setupStatCollector(
if err := c.StopGrafana(ctx, t.L(), t.ArtifactsDir()); err != nil {
t.L().ErrorfCtx(ctx, "Error(s) shutting down prom/grafana %s", err)
}
c.Wipe(ctx)
c.Wipe(ctx, false /* preserveCerts */)
}

promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), cfg)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/autoupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,7 @@ func registerAutoUpgrade(r registry.Registry) {

// Wipe n3 to exclude it from the dead node check the roachtest harness
// will perform after the test.
c.Wipe(ctx, c.Node(nodeDecommissioned))
c.Wipe(ctx, false /* preserveCerts */, c.Node(nodeDecommissioned))
}

r.Add(registry.TestSpec{
Expand Down
7 changes: 4 additions & 3 deletions pkg/cmd/roachtest/tests/backup_fixtures.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -135,10 +136,10 @@ func (bd *backupDriver) prepareCluster(ctx context.Context) {
bd.sp.hardware.getCRDBNodes(), version)
require.NoError(bd.t, err)

require.NoError(bd.t, clusterupgrade.StartWithBinary(ctx, bd.t.L(), bd.c,
require.NoError(bd.t, clusterupgrade.StartWithSettings(ctx, bd.t.L(), bd.c,
bd.sp.hardware.getCRDBNodes(),
binaryPath,
option.DefaultStartOptsNoBackups()))
option.DefaultStartOptsNoBackups(),
install.BinaryOption(binaryPath)))

bd.assertCorrectCockroachBinary(ctx)
if !bd.sp.backup.ignoreExistingBackups {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/clock_jump_crash.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ func runClockJump(ctx context.Context, t test.Test, c cluster.Cluster, tc clockJ
if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil {
c.Put(ctx, t.Cockroach(), "./cockroach", c.All())
}
c.Wipe(ctx)
c.Wipe(ctx, false /* preserveCerts */)
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())

db := c.Conn(ctx, t.L(), c.Spec().NodeCount)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/clock_monotonic.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func runClockMonotonicity(
if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil {
c.Put(ctx, t.Cockroach(), "./cockroach", c.All())
}
c.Wipe(ctx)
c.Wipe(ctx, false /* preserveCerts */)
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())

db := c.Conn(ctx, t.L(), c.Spec().NodeCount)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/cluster_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func runClusterInit(ctx context.Context, t test.Test, c cluster.Cluster) {
// default to just the first node) and we want to make sure that we're not
// relying on it.
for _, initNode := range []int{2, 1} {
c.Wipe(ctx)
c.Wipe(ctx, false /* preserveCerts */)
t.L().Printf("starting test with init node %d", initNode)
startOpts := option.DefaultStartOpts()

Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/decommission.go
Original file line number Diff line number Diff line change
Expand Up @@ -746,7 +746,7 @@ func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Clust
}
// Now stop+wipe them for good to keep the logs simple and the dead node detector happy.
c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.Nodes(targetNodeA, targetNodeB))
c.Wipe(ctx, c.Nodes(targetNodeA, targetNodeB))
c.Wipe(ctx, false /* preserveCerts */, c.Nodes(targetNodeA, targetNodeB))
}
}

Expand Down Expand Up @@ -871,7 +871,7 @@ func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Clust
t.L().Printf("wiping n%d and adding it back to the cluster as a new node\n", targetNode)

c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.Node(targetNode))
c.Wipe(ctx, c.Node(targetNode))
c.Wipe(ctx, false /*preserveCerts */, c.Node(targetNode))

joinNode := h.getRandNode()
internalAddrs, err := c.InternalAddr(ctx, t.L(), c.Node(joinNode))
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/decommission_self.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ func runDecommissionSelf(ctx context.Context, t test.Test, c cluster.Cluster) {
// Stop n2 and exclude it from post-test consistency checks,
// as this node can't contact cluster any more and operations
// on it will hang.
u.c.Wipe(ctx, c.Node(2))
u.c.Wipe(ctx, false /* preserveCerts */, c.Node(2))
},
checkOneMembership(1, "decommissioned"),
)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/inconsistency.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,5 +160,5 @@ func runInconsistency(ctx context.Context, t test.Test, c cluster.Cluster) {
// roachtest checks that no nodes are down when the test finishes, but in this
// case we have a down node that we can't restart. Remove the data dir, which
// tells roachtest to ignore this node.
c.Wipe(ctx, c.Node(1))
c.Wipe(ctx, false /* preserveCerts */, c.Node(1))
}
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -767,7 +767,7 @@ func registerKVScalability(r registry.Registry) {
const maxPerNodeConcurrency = 64
for i := nodes; i <= nodes*maxPerNodeConcurrency; i += nodes {
i := i // capture loop variable
c.Wipe(ctx, c.Range(1, nodes))
c.Wipe(ctx, false /* preserveCerts */, c.Range(1, nodes))
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, nodes))

t.Status("running workload")
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/kvbench.go
Original file line number Diff line number Diff line change
Expand Up @@ -229,7 +229,7 @@ func runKVBench(ctx context.Context, t test.Test, c cluster.Cluster, b kvBenchSp
// Wipe cluster before starting a new run because factors like load-based
// splitting can significantly change the underlying layout of the table and
// affect benchmark results.
c.Wipe(ctx, roachNodes)
c.Wipe(ctx, false /* preserveCerts */, roachNodes)
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), roachNodes)
time.Sleep(restartWait)

Expand Down
12 changes: 7 additions & 5 deletions pkg/cmd/roachtest/tests/mixed_version_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
Expand Down Expand Up @@ -811,7 +812,7 @@ func (sc *systemTableContents) loadShowResults(

query := fmt.Sprintf("SELECT * FROM [%s]%s", showStmt, aostFor(timestamp))
showCmd := roachtestutil.NewCommand("%s sql", mixedversion.CurrentCockroachPath).
Option("insecure").
Flag("certs-dir", "certs").
Flag("e", fmt.Sprintf("%q", query)).
String()

Expand Down Expand Up @@ -1963,14 +1964,15 @@ func (mvb *mixedVersionBackup) verifyBackupCollection(
func (mvb *mixedVersionBackup) resetCluster(
ctx context.Context, l *logger.Logger, version string,
) error {
l.Printf("resetting cluster using version %s", version)
if err := mvb.cluster.WipeE(ctx, l, mvb.roachNodes); err != nil {
l.Printf("resetting cluster using version %q", clusterupgrade.VersionMsg(version))
if err := mvb.cluster.WipeE(ctx, l, true /* preserveCerts */, mvb.roachNodes); err != nil {
return fmt.Errorf("failed to wipe cluster: %w", err)
}

cockroachPath := clusterupgrade.BinaryPathFromVersion(version)
return clusterupgrade.StartWithBinary(
ctx, l, mvb.cluster, mvb.roachNodes, cockroachPath, option.DefaultStartOptsNoBackups(),
return clusterupgrade.StartWithSettings(
ctx, l, mvb.cluster, mvb.roachNodes, option.DefaultStartOptsNoBackups(),
install.BinaryOption(cockroachPath), install.SecureOption(true),
)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/multitenant_tpch.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ func runMultiTenantTPCH(

// Restart and wipe the cluster to remove advantage of the second TPCH run.
c.Stop(ctx, t.L(), option.DefaultStopOpts())
c.Wipe(ctx)
c.Wipe(ctx, false /* preserveCerts */)
start()
singleTenantConn = c.Conn(ctx, t.L(), 1)
// Disable merge queue in the system tenant.
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/query_comparison_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ func runQueryComparison(
return
}
c.Stop(clusterCtx, t.L(), option.DefaultStopOpts())
c.Wipe(clusterCtx)
c.Wipe(clusterCtx, false /* preserveCerts */)
}
}

Expand Down
Loading

0 comments on commit 2b66f95

Please sign in to comment.