Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
98899: feat: allow starting docker container via env variable r=rickystewart a=btkostner

Fixes #87043 by allowing you to specify args via the `COCKROACH_ARGS` env value instead of a command. This is required to be able to use the official cockroach image with GitHub actions via a service. More details in the issue.

Note, do to weirdness with merging commands and env values, I decided that setting the `COCKROACH_ARGS` would ignore any command given. This should reduce issues with people trying to use both ways of specifying args and instead force them to pick one.

Release note (general change): Allow setting docker command args via the `COCKROACH_ARGS` environment variable.

99607: sql: block DROP TENANT based on a session var r=stevendanna a=knz

Fixes #97972.
Epic: CRDB-23559

In clusters where we will promote tenant management operations, we would like to ensure there is one extra step needed for administrators to drop a tenant (and thus irremedially lose data). Given that `sql_safe_updates` is not set automatically when users open their SQL session using their own client, we need another mechanism.

This change introduces the new (hidden) session var, `disable_drop_tenant`. When set, tenant deletion fails with the following error message:

```
demo@127.0.0.1:26257/movr> drop tenant foo;
ERROR: rejected (via sql_safe_updates or disable_drop_tenant): DROP TENANT causes irreversible data loss
SQLSTATE: 01000
```

(The session var `sql_safe_updates` is _also_ included as a blocker in the mechanism so that folk using `cockroach sql` get double protection).

The default value of this session var is `false` in single-tenant clusters, for compatibility with CC Serverless. It will be set to `true` via a config profile (#98466) when suitable.

Release note: None

99690: ui: drop index with space r=maryliag a=maryliag

Previously, if the index had a space on its name,
it would fail to drop.
This commit adds quotes so it can be executed.

Fixes #97988

Schema Insights
https://www.loom.com/share/04363b7f83484b5da19c760eb8d0de21

Table Details page
https://www.loom.com/share/1519b897a14440ddb066fb2ab03feb2d

Release note (bug fix): Index recommendation to DROP an index that have a space on its name can now be properly executed.

99750: sql: remove no longer used channel in createStatsNode r=yuzefovich a=yuzefovich

This hasn't been used as of fe6377c. Also mark `create_stats.go` as owned by SQL Queries.

Epic: None

Release note: None

99962: ui: add checks for values r=maryliag a=maryliag

Fixes #99655
Fixes #99538
Fixes #99539

Add checks to usages that could cause
`Cannot read properties of undefined`.

Release note: None

99963: roachtest: use local SSDs for disk-stall failover tests r=andrewbaptist a=nicktrav

The disk-stalled roachtests were updated in #99747 to use local SSDs. This change broke the `failover/*/disk-stall` tests, which look for `/dev/sdb` on GCE (the used for GCE Persistent Disks), but the tests still create clusters with local SSDs (the roachtest default).

Fix #99902.
Fix #99926.
Fix #99930.

Touches #97968.

Release note: None.

Co-authored-by: Blake Kostner <git@btkostner.io>
Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
Co-authored-by: maryliag <marylia@cockroachlabs.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: Nick Travers <travers@cockroachlabs.com>
  • Loading branch information
6 people committed Mar 30, 2023
7 parents d671fad + 1424d30 + bdc8d1a + e439695 + ab10d6c + 119c54d + 545d49a commit 331a672
Show file tree
Hide file tree
Showing 39 changed files with 179 additions and 73 deletions.
1 change: 1 addition & 0 deletions .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
/pkg/sql/stats/ @cockroachdb/sql-queries

/pkg/sql/col* @cockroachdb/sql-queries
/pkg/sql/create_stats* @cockroachdb/sql-queries
/pkg/sql/distsql*.go @cockroachdb/sql-queries
/pkg/sql/exec* @cockroachdb/sql-queries
#!/pkg/sql/exec_log*.go @cockroachdb/sql-queries-noreview
Expand Down
8 changes: 7 additions & 1 deletion build/deploy/cockroach.sh
Original file line number Diff line number Diff line change
Expand Up @@ -295,4 +295,10 @@ _main() {
esac
}

_main "$@"
set_env_var "COCKROACH_ARGS"

if [[ -n "$COCKROACH_ARGS" ]]; then
_main "$COCKROACH_ARGS"
else
_main "$@"
fi
2 changes: 2 additions & 0 deletions pkg/cmd/roachtest/tests/disk_stall.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,8 @@ func registerDiskStalledDetection(r registry.Registry) {
}
makeSpec := func() spec.ClusterSpec {
s := r.MakeClusterSpec(4, spec.ReuseNone())
// Use PDs in an attempt to work around flakes encountered when using SSDs.
// See #97968.
s.PreferLocalSSD = false
return s
}
Expand Down
15 changes: 12 additions & 3 deletions pkg/cmd/roachtest/tests/failover.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,11 +45,20 @@ func registerFailover(r registry.Registry) {
failureModeDiskStall,
} {
failureMode := failureMode // pin loop variable
makeSpec := func(nNodes, nCPU int) spec.ClusterSpec {
s := r.MakeClusterSpec(nNodes, spec.CPU(nCPU))
if failureMode == failureModeDiskStall {
// Use PDs in an attempt to work around flakes encountered when using
// SSDs. See #97968.
s.PreferLocalSSD = false
}
return s
}
r.Add(registry.TestSpec{
Name: fmt.Sprintf("failover/non-system/%s", failureMode),
Owner: registry.OwnerKV,
Timeout: 30 * time.Minute,
Cluster: r.MakeClusterSpec(7, spec.CPU(4)),
Cluster: makeSpec(7 /* nodes */, 4 /* cpus */),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runFailoverNonSystem(ctx, t, c, failureMode)
},
Expand All @@ -58,7 +67,7 @@ func registerFailover(r registry.Registry) {
Name: fmt.Sprintf("failover/liveness/%s", failureMode),
Owner: registry.OwnerKV,
Timeout: 30 * time.Minute,
Cluster: r.MakeClusterSpec(5, spec.CPU(4)),
Cluster: makeSpec(5 /* nodes */, 4 /* cpus */),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runFailoverLiveness(ctx, t, c, failureMode)
},
Expand All @@ -67,7 +76,7 @@ func registerFailover(r registry.Registry) {
Name: fmt.Sprintf("failover/system-non-liveness/%s", failureMode),
Owner: registry.OwnerKV,
Timeout: 30 * time.Minute,
Cluster: r.MakeClusterSpec(7, spec.CPU(4)),
Cluster: makeSpec(7 /* nodes */, 4 /* cpus */),
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runFailoverSystemNonLiveness(ctx, t, c, failureMode)
},
Expand Down
11 changes: 3 additions & 8 deletions pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,22 +104,19 @@ type createStatsNode struct {
// createStatsRun contains the run-time state of createStatsNode during local
// execution.
type createStatsRun struct {
resultsCh chan tree.Datums
errCh chan error
errCh chan error
}

func (n *createStatsNode) startExec(params runParams) error {
telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter("stats"))
n.run.resultsCh = make(chan tree.Datums)
n.run.errCh = make(chan error)
go func() {
err := n.startJob(params.ctx, n.run.resultsCh)
err := n.startJob(params.ctx)
select {
case <-params.ctx.Done():
case n.run.errCh <- err:
}
close(n.run.errCh)
close(n.run.resultsCh)
}()
return nil
}
Expand All @@ -130,16 +127,14 @@ func (n *createStatsNode) Next(params runParams) (bool, error) {
return false, params.ctx.Err()
case err := <-n.run.errCh:
return false, err
case <-n.run.resultsCh:
return true, nil
}
}

func (*createStatsNode) Close(context.Context) {}
func (*createStatsNode) Values() tree.Datums { return nil }

// startJob starts a CreateStats job to plan and execute statistics creation.
func (n *createStatsNode) startJob(ctx context.Context, resultsCh chan<- tree.Datums) error {
func (n *createStatsNode) startJob(ctx context.Context) error {
record, err := n.makeJobRecord(ctx)
if err != nil {
return err
Expand Down
23 changes: 23 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -747,6 +747,25 @@ var errTransactionInProgress = errors.New("there is already a transaction in pro
const sqlTxnName string = "sql txn"
const metricsSampleInterval = 10 * time.Second

// enableDropTenant (or rather, its inverted boolean value) defines
// the default value for the session var "disable_drop_tenant".
//
// Note:
// - We use a cluster setting here instead of a default role option
// because we need this to be settable also for the 'admin' role.
// - The cluster setting is named "enable" because boolean cluster
// settings are all ".enabled" -- we do not have ".disabled"
// settings anywhere.
// - The session var is named "disable_" because we want the Go
// default value (false) to mean that tenant deletion is enabled.
// This is needed for backward-compatibility with Cockroach Cloud.
var enableDropTenant = settings.RegisterBoolSetting(
settings.SystemOnly,
"sql.drop_tenant.enabled",
"default value (inverted) for the disable_drop_tenant session setting",
true,
)

// Fully-qualified names for metrics.
var (
MetaSQLExecLatency = metric.Metadata{
Expand Down Expand Up @@ -3156,6 +3175,10 @@ func (m *sessionDataMutator) SetSafeUpdates(val bool) {
m.data.SafeUpdates = val
}

func (m *sessionDataMutator) SetDisableDropTenant(val bool) {
m.data.DisableDropTenant = val
}

func (m *sessionDataMutator) SetCheckFunctionBodies(val bool) {
m.data.CheckFunctionBodies = val
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -5204,6 +5204,7 @@ default_transaction_read_only off
default_transaction_use_follower_reads off
default_with_oids off
descriptor_validation on
disable_drop_tenant off
disable_hoist_projection_in_join_limitation off
disable_partially_distributed_plans off
disable_plan_gists off
Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/tenant
Original file line number Diff line number Diff line change
Expand Up @@ -474,3 +474,21 @@ DROP TENANT tmpl

statement ok
RESET CLUSTER SETTING sql.create_tenant.default_template

subtest block_drop_tenant

statement ok
SET disable_drop_tenant = 'true'

statement ok
CREATE TENANT nodelete

statement error rejected.*irreversible data loss
DROP TENANT nodelete

statement ok
RESET disable_drop_tenant

statement ok
DROP TENANT nodelete

4 changes: 3 additions & 1 deletion pkg/sql/sessiondatapb/local_only_session_data.proto
Original file line number Diff line number Diff line change
Expand Up @@ -318,7 +318,6 @@ message LocalOnlySessionData {
// optimizer should use improved statistics calculations for disjunctive
// filters.
bool optimizer_use_improved_disjunction_stats = 86;

// OptimizerUseLimitOrderingForStreamingGroupBy enables the exploration rule
// which optimizes 'SELECT ... GROUP BY ... ORDER BY ... LIMIT n' queries.
// The rule uses the required ordering in the limit expression to inform an
Expand Down Expand Up @@ -367,6 +366,9 @@ message LocalOnlySessionData {
int64 prepared_statements_cache_size = 97;
// StreamerEnabled controls whether the Streamer API can be used.
bool streamer_enabled = 98;
// DisableDropTenant causes errors when the client
// attempts to drop tenants or tenant records.
bool disable_drop_tenant = 99;

///////////////////////////////////////////////////////////////////////////
// WARNING: consider whether a session parameter you're adding needs to //
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/tenant_deletion.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,13 @@ import (
func (p *planner) DropTenantByID(
ctx context.Context, tenID uint64, synchronousImmediateDrop, ignoreServiceMode bool,
) error {
if p.SessionData().DisableDropTenant || p.SessionData().SafeUpdates {
err := errors.Newf("DROP TENANT causes irreversible data loss")
err = errors.WithMessage(err, "rejected (via sql_safe_updates or disable_drop_tenant)")
err = pgerror.WithCandidateCode(err, pgcode.Warning)
return err
}

if p.EvalContext().TxnReadOnly {
return readOnlyError("DROP TENANT")
}
Expand Down
28 changes: 28 additions & 0 deletions pkg/sql/vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -1520,6 +1520,34 @@ var varGen = map[string]sessionVar{
// Setting is done by the SetTracing statement.
},

// CockroachDB extension.
`disable_drop_tenant`: {
Hidden: true,
Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) {
return formatBoolAsPostgresSetting(evalCtx.SessionData().DisableDropTenant), nil
},
Set: func(_ context.Context, m sessionDataMutator, s string) error {
b, err := paramparse.ParseBoolVar("disable_drop_tenant", s)
if err != nil {
return err
}
m.SetDisableDropTenant(b)
return nil
},
GlobalDefault: func(sv *settings.Values) string {
// Note:
// - We use a cluster setting here instead of a default role option
// because we need this to be settable also for the 'admin' role.
// - The cluster setting is named "enable" because boolean cluster
// settings are all ".enabled" -- we do not have ".disabled"
// settings anywhere.
// - The session var is named "disable_" because we want the Go
// default value (false) to mean that tenant deletion is enabled.
// This is needed for backward-compatibility with Cockroach Cloud.
return formatBoolAsPostgresSetting(!enableDropTenant.Get(sv))
},
},

// CockroachDB extension.
`allow_prepare_as_opt_plan`: {
Hidden: true,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ui/workspaces/cluster-ui/src/api/safesql.ts
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ export class SQL implements SQLStringer {
// case sensitive when used in a query. If the input string contains a zero
// byte, the result will be truncated immediately before it.
// Cribbed from https://github.com/lib/pq and Typescript-ified.
function QuoteIdentifier(name: string): string {
export function QuoteIdentifier(name: string): string {
// Use a search regex to replace all occurrences instead of just the first occurrence.
const search = /"/g;
return `"` + name.replace(search, `""`) + `"`;
Expand Down
10 changes: 9 additions & 1 deletion pkg/ui/workspaces/cluster-ui/src/api/schemaInsightsApi.ts
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import {
recommendDropUnusedIndex,
} from "../insights";
import { HexStringToInt64String } from "../util";
import { QuoteIdentifier } from "./safesql";

// Export for db-console import from clusterUiApi.
export type { InsightRecommendation } from "../insights";
Expand Down Expand Up @@ -72,7 +73,11 @@ function clusterIndexUsageStatsToSchemaInsight(
results[key] = {
type: "DropIndex",
database: row.database_name,
query: `DROP INDEX ${row.schema_name}.${row.table_name}@${row.index_name};`,
query: `DROP INDEX ${QuoteIdentifier(
row.schema_name,
)}.${QuoteIdentifier(row.table_name)}@${QuoteIdentifier(
row.index_name,
)};`,
indexDetails: {
table: row.table_name,
indexID: row.index_id,
Expand All @@ -95,6 +100,9 @@ function createIndexRecommendationsToSchemaInsight(

txn_result.rows.forEach(row => {
row.index_recommendations.forEach(rec => {
if (!rec.includes(" : ")) {
return;
}
const recSplit = rec.split(" : ");
const recType = recSplit[0];
const recQuery = recSplit[1];
Expand Down
2 changes: 1 addition & 1 deletion pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts
Original file line number Diff line number Diff line change
Expand Up @@ -384,7 +384,7 @@ function formatTxnInsightsRow(row: TxnInsightsResponseRow): TxnInsightEvent {
transactionExecutionID: row.txn_id,
transactionFingerprintID: row.txn_fingerprint_id,
implicitTxn: row.implicit_txn,
query: row.query.split(" ; ").join("\n"),
query: row.query?.split(" ; ").join("\n") || "",
startTime,
endTime,
elapsedTimeMillis: endTime.diff(startTime, "milliseconds"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -378,9 +378,9 @@ export class DatabaseDetailsPage extends React.Component<
const { search, tables, filters, nodeRegions } = this.props;

const regionsSelected =
filters.regions.length > 0 ? filters.regions.split(",") : [];
filters.regions?.length > 0 ? filters.regions.split(",") : [];
const nodesSelected =
filters.nodes.length > 0 ? filters.nodes.split(",") : [];
filters.nodes?.length > 0 ? filters.nodes.split(",") : [];

return tables
.filter(table => (search ? filterBySearchQuery(table, search) : true))
Expand All @@ -392,13 +392,11 @@ export class DatabaseDetailsPage extends React.Component<
let foundNode = nodesSelected.length == 0;

table.details.nodes?.forEach(node => {
if (
foundRegion ||
regionsSelected.includes(nodeRegions[node.toString()])
) {
const n = node?.toString() || "";
if (foundRegion || regionsSelected.includes(nodeRegions[n])) {
foundRegion = true;
}
if (foundNode || nodesSelected.includes("n" + node.toString())) {
if (foundNode || nodesSelected.includes("n" + n)) {
foundNode = true;
}
if (foundNode && foundRegion) return true;
Expand Down Expand Up @@ -738,7 +736,7 @@ export class DatabaseDetailsPage extends React.Component<
hideAppNames={true}
regions={regions}
hideTimeLabel={true}
nodes={nodes.map(n => "n" + n.toString())}
nodes={nodes.map(n => "n" + n?.toString())}
activeFilters={activeFilters}
filters={defaultFilters}
onSubmitFilters={this.onSubmitFilters}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ import { RecommendationType } from "../indexDetailsPage";
import LoadingError from "../sqlActivity/errorComponent";
import { Loading } from "../loading";
import { UIConfigState } from "../store";
import { QuoteIdentifier } from "../api/safesql";

const cx = classNames.bind(styles);
const booleanSettingCx = classnames.bind(booleanSettingStyles);
Expand Down Expand Up @@ -375,7 +376,9 @@ export class DatabaseTablePage extends React.Component<
const query = indexStat.indexRecommendations.map(recommendation => {
switch (recommendation.type) {
case "DROP_UNUSED":
return `DROP INDEX ${this.props.name}@${indexStat.indexName};`;
return `DROP INDEX ${QuoteIdentifier(
this.props.name,
)}@${QuoteIdentifier(indexStat.indexName)};`;
}
});
if (query.length === 0) {
Expand Down
14 changes: 6 additions & 8 deletions pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -424,9 +424,9 @@ export class DatabasesPage extends React.Component<

// The regions and nodes selected from the filter dropdown.
const regionsSelected =
filters.regions.length > 0 ? filters.regions.split(",") : [];
filters.regions?.length > 0 ? filters.regions.split(",") : [];
const nodesSelected =
filters.nodes.length > 0 ? filters.nodes.split(",") : [];
filters.nodes?.length > 0 ? filters.nodes.split(",") : [];

return databases
.filter(db => (search ? filterBySearchQuery(db, search) : true))
Expand All @@ -438,13 +438,11 @@ export class DatabasesPage extends React.Component<
let foundNode = nodesSelected.length == 0;

db.nodes?.forEach(node => {
if (
foundRegion ||
regionsSelected.includes(nodeRegions[node.toString()])
) {
const n = node?.toString() || "";
if (foundRegion || regionsSelected.includes(nodeRegions[n])) {
foundRegion = true;
}
if (foundNode || nodesSelected.includes("n" + node.toString())) {
if (foundNode || nodesSelected.includes("n" + n)) {
foundNode = true;
}
if (foundNode && foundRegion) return true;
Expand Down Expand Up @@ -617,7 +615,7 @@ export class DatabasesPage extends React.Component<
hideAppNames={true}
regions={regions}
hideTimeLabel={true}
nodes={nodes.map(n => "n" + n.toString())}
nodes={nodes.map(n => "n" + n?.toString())}
activeFilters={activeFilters}
filters={defaultFilters}
onSubmitFilters={this.onSubmitFilters}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ export function getHighlightedText(
})
.join("|");
const parts = isOriginalText
? text.split(new RegExp(`(${search})`, "gi"))
? text?.split(new RegExp(`(${search})`, "gi"))
: rebaseText(text, highlight).split(new RegExp(`(${search})`, "gi"));
const highlightClass = hasDarkBkg ? "_text-bold-light" : "_text-bold";
return parts.map((part, i) => {
Expand Down
Loading

0 comments on commit 331a672

Please sign in to comment.