Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
72992: util/log: fix redactability of logging tags r=abarganier a=knz

Fixes #72905.

Some time in the v21.2 cycle, the log entry preparation logic was
refactored and a mistake was introduced: the logging tags were not any
more subject to the redaction logic. The result was that redaction
markers were missing in log tag values, and if a value had contained
unbalanced redaction markers in a value string (say, as part of a SQL
table key), it would have caused log file corruption and possibly a
confidential data leak.

This patch fixes that, by preparing the logging tags in the same way
as the main message for each entry.

Release note (cli change): A bug affecting the redactability of
logging tags in output log entries has been fixed. This bug had
been introduced in the v21.2 release.

73937: setting: introduce setting classes r=RaduBerinde a=RaduBerinde

This commit introduces the three setting classes in the RFC (#73349):
`SystemOnly`, `TenantReadOnly`, and `TenantWritable`. The `SystemOnly`
class replaces the existing `WithSystemOnly()`.

In this change we don't yet implement the advertised semantics. We
mechanically use `TenantWritable` for all settings except those that
were using `WithSystemOnly()` which use `SystemOnly`; this should not
change any existing behavior. The classes will be revisited in a
separate change, after we implement the semantics.

Release note: None

73978: opt: fix like escape processing for span constraints r=cucaroach a=cucaroach

Fixes: #44123

Previously no attempt was made to properly handle escape ('\\') sequence
in like patterns being turned into constraints. Refactor code used to
process like at runtime to generate a regexp and use that to properly
handle index constraint generation.

Release note (sql change): Escape character processing was missing from
constraint span generation which resulted in incorrect results when
doing escaped like lookups.

74102: sql: do not fetch virtual columns during backfill r=mgartner a=mgartner

Fixes #73372

Release note (bug fix): A bug has been fixed that caused internal errors
when altering the primary key of a table. The bug was only present if
the table had a partial index with a predicate that referenced a virtual
computed column. This bug was present since virtual computed columns
were added in version 21.1.0.

74110: bazel: require setting `cockroach_cross=y` to opt into cross toolchains r=irfansharif a=rickystewart

With `--incompatible_enable_cc_toolchain_resolution` set in #73819, now
Bazel selects the appropriate toolchain for you. Bazel was selecting the
`cross_linux_toolchain` when building for the host platform on Linux,
resulting in link errors when trying to compile `stress` under `race`.
We update the toolchains to instead require opting into the cross
toolchains by defining `cockroach_cross=y`.

Closes #73997.

Release note: None

74111: bench/rttanalysis: allow roundtrips to be off by 1 r=ajwerner a=ajwerner

If we don't have a range, let the currently estimate be wrong by 1.
We mostly care about the ballpark and the growth rate. I'm sick of
these flakes.

Fixes #73884.

Release note: None

74152: ci: don't delete `test.json.txt` after processing r=tbg a=rickystewart

We've seen some issues where Bazel jobs are failing in `github-post`
(#73841, #74013) with the following output:

    found outstanding output. Considering last test failed:

It's hard to say what the problem is because these scripts haven't kept
the `test.json.txt` in `artifacts`. Here I remove the logic to clean up
the file so we can RC further instances of the problem.

Release note: None

Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Tommy Reilly <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Andrew Werner <[email protected]>
  • Loading branch information
7 people committed Dec 21, 2021
8 parents 54debf3 + e57ace1 + 52c2df4 + d25385b + 0d4e0d9 + 59f8eaa + 0750472 + 4cca226 commit 779f08b
Show file tree
Hide file tree
Showing 184 changed files with 1,384 additions and 368 deletions.
10 changes: 1 addition & 9 deletions build/teamcity-bazel-support.sh
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ _tc_release_branch() {
# `$BAZEL_BIN/pkg/cmd/github-post/github-post_/github-post`
# artifacts_dir: usually `/artifacts`
# test_json: path to test's JSON output, usually generated by `rules_go`'s and
# `GO_TEST_JSON_OUTPUT_FILE`. The file is removed after processing.
# `GO_TEST_JSON_OUTPUT_FILE`.
# create_tarball: whether to create a tarball with full logs. If the test's
# exit code is passed, the tarball is generated on failures.
process_test_json() {
Expand All @@ -58,12 +58,6 @@ process_test_json() {
local test_json=$4
local create_tarball=$5

# move test.json.txt to the artifacts directory in order to simplify tarball creation
if [ ! -e $artifacts_dir/test.json.txt ]; then
mv -f $test_json $artifacts_dir/test.json.txt
test_json=$artifacts_dir/test.json.txt
fi

$testfilter -mode=strip < "$test_json" | $testfilter -mode=omit | $testfilter -mode=convert > "$artifacts_dir"/failures.txt
failures_size=$(stat --format=%s "$artifacts_dir"/failures.txt)
if [ $failures_size = 0 ]; then
Expand Down Expand Up @@ -103,8 +97,6 @@ process_test_json() {
rm -rf "$artifacts_dir"/full_output.txt
fi

rm -f "$test_json"

# Some unit tests test automatic ballast creation. These ballasts can be
# larger than the maximum artifact size. Remove any artifacts with the
# EMERGENCY_BALLAST filename.
Expand Down
2 changes: 1 addition & 1 deletion build/teamcity/cockroach/nightlies/stress_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ fi
bazel build //pkg/cmd/bazci //pkg/cmd/github-post //pkg/cmd/testfilter --config=ci
BAZEL_BIN=$(bazel info bazel-bin --config=ci)
ARTIFACTS_DIR=/artifacts
GO_TEST_JSON_OUTPUT_FILE=$ARTIFACTS_DIR/test.json.txt

# Query to list all affected tests.
PKG=${PKG#"./"}
Expand All @@ -34,6 +33,7 @@ do
continue
fi
exit_status=0
GO_TEST_JSON_OUTPUT_FILE=$ARTIFACTS_DIR/$(echo "$test" | cut -d: -f2).test.json.txt
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci --config=ci test "$test" -- \
--test_env=COCKROACH_NIGHTLY_STRESS=true \
--test_env=GO_TEST_JSON_OUTPUT_FILE=$GO_TEST_JSON_OUTPUT_FILE \
Expand Down
19 changes: 19 additions & 0 deletions build/toolchains/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,9 @@ toolchain(
"@platforms//os:linux",
"@platforms//cpu:x86_64",
],
target_settings = [
":cross",
],
toolchain = "@toolchain_cross_x86_64-unknown-linux-gnu//:toolchain",
toolchain_type = "@bazel_tools//tools/cpp:toolchain_type",
)
Expand All @@ -30,6 +33,9 @@ toolchain(
"@platforms//os:windows",
"@platforms//cpu:x86_64",
],
target_settings = [
":cross",
],
toolchain = "@toolchain_cross_x86_64-w64-mingw32//:toolchain",
toolchain_type = "@bazel_tools//tools/cpp:toolchain_type",
)
Expand All @@ -52,6 +58,9 @@ toolchain(
"@platforms//os:macos",
"@platforms//cpu:x86_64",
],
target_settings = [
":cross",
],
toolchain = "@toolchain_cross_x86_64-apple-darwin19//:toolchain",
toolchain_type = "@bazel_tools//tools/cpp:toolchain_type",
)
Expand All @@ -74,6 +83,9 @@ toolchain(
"@platforms//os:linux",
"@platforms//cpu:arm64",
],
target_settings = [
":cross",
],
toolchain = "@toolchain_cross_aarch64-unknown-linux-gnu//:toolchain",
toolchain_type = "@bazel_tools//tools/cpp:toolchain_type",
)
Expand Down Expand Up @@ -138,6 +150,13 @@ config_setting(
},
)

config_setting(
name = "cross",
define_values = {
"cockroach_cross": "y",
},
)

config_setting(
name = "is_cross_macos",
constraint_values = [
Expand Down
6 changes: 3 additions & 3 deletions docs/RFCS/20211106_multitenant_cluster_settings.md
Original file line number Diff line number Diff line change
Expand Up @@ -215,11 +215,11 @@ following guidelines:

- control settings relevant to tenant-specific internal implementation (like
tenant throttling) that we want to be able to control per-tenant should be
`system`.
`tenant-ro`.

- when in doubt the first choice to consider should be `tenant-ro`.
- when in doubt the first choice to consider should be `tenant-rw`.

- `System` should be used with caution - we have to be sure that there is no
- `system` should be used with caution - we have to be sure that there is no
internal code running on the tenant that needs to consult them.

We fully hide `system` settings from non-system tenants. The cluster settings
Expand Down
5 changes: 4 additions & 1 deletion pkg/bench/rttanalysis/validate_benchmark_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -319,7 +319,10 @@ func (b benchmarkExpectations) find(name string) (benchmarkExpectation, bool) {
}

func (e benchmarkExpectation) matches(roundTrips int) bool {
return e.min <= roundTrips && roundTrips <= e.max
// Either the value falls within the expected range, or
return (e.min <= roundTrips && roundTrips <= e.max) ||
// the expectation isn't a range, so give it a leeway of one.
e.min == e.max && (roundTrips == e.min-1 || roundTrips == e.min+1)
}

func (e benchmarkExpectation) String() string {
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ var _ cloud.KMSEnv = &backupKMSEnv{}

// featureBackupEnabled is used to enable and disable the BACKUP feature.
var featureBackupEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"feature.backup.enabled",
"set to true to enable backups, false to disable; default is true",
featureflag.FeatureFlagEnabledDefault,
Expand Down
7 changes: 7 additions & 0 deletions pkg/ccl/backupccl/backup_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,42 +50,49 @@ var backupOutputTypes = []*types.T{}

var (
useTBI = settings.RegisterBoolSetting(
settings.TenantWritable,
"kv.bulk_io_write.experimental_incremental_export_enabled",
"use experimental time-bound file filter when exporting in BACKUP",
true,
)
priorityAfter = settings.RegisterDurationSetting(
settings.TenantWritable,
"bulkio.backup.read_with_priority_after",
"amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads",
time.Minute,
settings.NonNegativeDuration,
).WithPublic()
delayPerAttmpt = settings.RegisterDurationSetting(
settings.TenantWritable,
"bulkio.backup.read_retry_delay",
"amount of time since the read-as-of time, per-prior attempt, to wait before making another attempt",
time.Second*5,
settings.NonNegativeDuration,
)
timeoutPerAttempt = settings.RegisterDurationSetting(
settings.TenantWritable,
"bulkio.backup.read_timeout",
"amount of time after which a read attempt is considered timed out, which causes the backup to fail",
time.Minute*5,
settings.NonNegativeDuration,
).WithPublic()
targetFileSize = settings.RegisterByteSizeSetting(
settings.TenantWritable,
"bulkio.backup.file_size",
"target size for individual data files produced during BACKUP",
128<<20,
).WithPublic()

smallFileBuffer = settings.RegisterByteSizeSetting(
settings.TenantWritable,
"bulkio.backup.merge_file_buffer_size",
"size limit used when buffering backup files before merging them",
16<<20,
settings.NonNegativeInt,
)

splitKeysOnTimestamps = settings.RegisterBoolSetting(
settings.TenantWritable,
"bulkio.backup.split_keys_on_timestamps",
"split backup data on timestamps when writing revision history",
false,
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ var scheduledBackupOptionExpectValues = map[string]sql.KVStringOptValidate{
// scheduledBackupGCProtectionEnabled is used to enable and disable the chaining
// of protected timestamps amongst scheduled backups.
var scheduledBackupGCProtectionEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"schedules.backup.gc_protection.enabled",
"enable chaining of GC protection across backups run as part of a schedule; default is false",
false, /* defaultValue */
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/restore_data_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,7 @@ var defaultNumWorkers = util.ConstantWithMetamorphicTestRange(
// The maximum is not enforced since if the maximum is reduced in the future that
// may cause the cluster setting to fail.
var numRestoreWorkers = settings.RegisterIntSetting(
settings.TenantWritable,
"kv.bulk_io_write.restore_node_concurrency",
fmt.Sprintf("the number of workers processing a restore per job per node; maximum %d",
maxConcurrentRestoreWorkers),
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@ var allowedDebugPauseOnValues = map[string]struct{}{

// featureRestoreEnabled is used to enable and disable the RESTORE feature.
var featureRestoreEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"feature.restore.enabled",
"set to true to enable restore, false to disable; default is true",
featureflag.FeatureFlagEnabledDefault,
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/changefeed_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ import (

// featureChangefeedEnabled is used to enable and disable the CHANGEFEED feature.
var featureChangefeedEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"feature.changefeed.enabled",
"set to true to enable changefeeds, false to disable; default is true",
featureflag.FeatureFlagEnabledDefault,
Expand Down
9 changes: 9 additions & 0 deletions pkg/ccl/changefeedccl/changefeedbase/settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
// NB: The more generic name of this setting precedes its current
// interpretation. It used to control additional polling rates.
var TableDescriptorPollInterval = settings.RegisterDurationSetting(
settings.TenantWritable,
"changefeed.experimental_poll_interval",
"polling interval for the table descriptors",
1*time.Second,
Expand All @@ -43,13 +44,15 @@ func TestingSetDefaultMinCheckpointFrequency(f time.Duration) func() {
// PerChangefeedMemLimit controls how much data can be buffered by
// a single changefeed.
var PerChangefeedMemLimit = settings.RegisterByteSizeSetting(
settings.TenantWritable,
"changefeed.memory.per_changefeed_limit",
"controls amount of data that can be buffered per changefeed",
1<<30,
)

// SlowSpanLogThreshold controls when we will log slow spans.
var SlowSpanLogThreshold = settings.RegisterDurationSetting(
settings.TenantWritable,
"changefeed.slow_span_log_threshold",
"a changefeed will log spans with resolved timestamps this far behind the current wall-clock time; if 0, a default value is calculated based on other cluster settings",
0,
Expand All @@ -58,6 +61,7 @@ var SlowSpanLogThreshold = settings.RegisterDurationSetting(

// FrontierCheckpointFrequency controls the frequency of frontier checkpoints.
var FrontierCheckpointFrequency = settings.RegisterDurationSetting(
settings.TenantWritable,
"changefeed.frontier_checkpoint_frequency",
"controls the frequency with which span level checkpoints will be written; if 0, disabled.",
10*time.Minute,
Expand All @@ -78,6 +82,7 @@ var FrontierCheckpointFrequency = settings.RegisterDurationSetting(
// Therefore, we should write at most 6 MB of checkpoint/hour; OR, based on the default
// FrontierCheckpointFrequency setting, 1 MB per checkpoint.
var FrontierCheckpointMaxBytes = settings.RegisterByteSizeSetting(
settings.TenantWritable,
"changefeed.frontier_checkpoint_max_bytes",
"controls the maximum size of the checkpoint as a total size of key bytes",
1<<20,
Expand All @@ -87,6 +92,7 @@ var FrontierCheckpointMaxBytes = settings.RegisterByteSizeSetting(
// Scan requests are issued when changefeed performs the backfill.
// If set to 0, a reasonable default will be chosen.
var ScanRequestLimit = settings.RegisterIntSetting(
settings.TenantWritable,
"changefeed.backfill.concurrent_scan_requests",
"number of concurrent scan requests per node issued during a backfill",
0,
Expand All @@ -112,6 +118,7 @@ type SinkThrottleConfig struct {
// NodeSinkThrottleConfig is the node wide throttling configuration for changefeeds.
var NodeSinkThrottleConfig = func() *settings.StringSetting {
s := settings.RegisterValidatedStringSetting(
settings.TenantWritable,
"changefeed.node_throttle_config",
"specifies node level throttling configuration for all changefeeeds",
"",
Expand All @@ -133,6 +140,7 @@ func validateSinkThrottleConfig(values *settings.Values, configStr string) error
// MinHighWaterMarkCheckpointAdvance specifies the minimum amount of time the
// changefeed high water mark must advance for it to be eligible for checkpointing.
var MinHighWaterMarkCheckpointAdvance = settings.RegisterDurationSetting(
settings.TenantWritable,
"changefeed.min_highwater_advance",
"minimum amount of time the changefeed high water mark must advance "+
"for it to be eligible for checkpointing; Default of 0 will checkpoint every time frontier "+
Expand All @@ -148,6 +156,7 @@ var MinHighWaterMarkCheckpointAdvance = settings.RegisterDurationSetting(
// with complex schemes to accurately measure and adjust current memory usage,
// we'll request the amount of memory multiplied by this fudge factor.
var EventMemoryMultiplier = settings.RegisterFloatSetting(
settings.TenantWritable,
"changefeed.event_memory_multiplier",
"the amount of memory required to process an event is multiplied by this factor",
3,
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/importccl/import_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,7 @@ var allowedIntoFormats = map[string]struct{}{

// featureImportEnabled is used to enable and disable the IMPORT feature.
var featureImportEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"feature.import.enabled",
"set to true to enable imports, false to disable; default is true",
featureflag.FeatureFlagEnabledDefault,
Expand Down
5 changes: 5 additions & 0 deletions pkg/ccl/importccl/import_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ const readImportDataProcessorName = "readImportDataProcessor"

var importPKAdderBufferSize = func() *settings.ByteSizeSetting {
s := settings.RegisterByteSizeSetting(
settings.TenantWritable,
"kv.bulk_ingest.pk_buffer_size",
"the initial size of the BulkAdder buffer handling primary index imports",
32<<20,
Expand All @@ -56,6 +57,7 @@ var importPKAdderBufferSize = func() *settings.ByteSizeSetting {

var importPKAdderMaxBufferSize = func() *settings.ByteSizeSetting {
s := settings.RegisterByteSizeSetting(
settings.TenantWritable,
"kv.bulk_ingest.max_pk_buffer_size",
"the maximum size of the BulkAdder buffer handling primary index imports",
128<<20,
Expand All @@ -65,6 +67,7 @@ var importPKAdderMaxBufferSize = func() *settings.ByteSizeSetting {

var importIndexAdderBufferSize = func() *settings.ByteSizeSetting {
s := settings.RegisterByteSizeSetting(
settings.TenantWritable,
"kv.bulk_ingest.index_buffer_size",
"the initial size of the BulkAdder buffer handling secondary index imports",
32<<20,
Expand All @@ -74,6 +77,7 @@ var importIndexAdderBufferSize = func() *settings.ByteSizeSetting {

var importIndexAdderMaxBufferSize = func() *settings.ByteSizeSetting {
s := settings.RegisterByteSizeSetting(
settings.TenantWritable,
"kv.bulk_ingest.max_index_buffer_size",
"the maximum size of the BulkAdder buffer handling secondary index imports",
512<<20,
Expand All @@ -83,6 +87,7 @@ var importIndexAdderMaxBufferSize = func() *settings.ByteSizeSetting {

var importBufferIncrementSize = func() *settings.ByteSizeSetting {
s := settings.RegisterByteSizeSetting(
settings.TenantWritable,
"kv.bulk_ingest.buffer_increment",
"the size by which the BulkAdder attempts to grow its buffer before flushing",
32<<20,
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
// measure of how long closed timestamp updates are supposed to take from the
// leaseholder to the followers.
var ClosedTimestampPropagationSlack = settings.RegisterDurationSetting(
settings.TenantWritable,
"kv.closed_timestamp.propagation_slack",
"a conservative estimate of the amount of time expect for closed timestamps to "+
"propagate from a leaseholder to followers. This is taken into account by "+
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (

// TargetPeriodSetting is exported for testing purposes.
var TargetPeriodSetting = settings.RegisterDurationSetting(
settings.TenantWritable,
"tenant_cost_control_period",
"target duration between token bucket requests from tenants (requires restart)",
10*time.Second,
Expand All @@ -39,6 +40,7 @@ var TargetPeriodSetting = settings.RegisterDurationSetting(

// CPUUsageAllowance is exported for testing purposes.
var CPUUsageAllowance = settings.RegisterDurationSetting(
settings.TenantWritable,
"tenant_cpu_usage_allowance",
"this much CPU usage per second is considered background usage and "+
"doesn't contribute to consumption; for example, if it is set to 10ms, "+
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/multitenantccl/tenantcostserver/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ type instance struct {
// Note: the "four" in the description comes from
// tenantcostclient.extendedReportingPeriodFactor.
var instanceInactivity = settings.RegisterDurationSetting(
settings.TenantWritable,
"tenant_usage_instance_inactivity",
"instances that have not reported consumption for longer than this value are cleaned up; "+
"should be at least four times higher than the tenant_cost_control_period of any tenant",
Expand Down
Loading

0 comments on commit 779f08b

Please sign in to comment.