Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
113532: pkg/util/log: selectively apply file-defaults.format-options to stderr r=dhartunian a=abarganier

Fixes: #113321

The stderr sink in the log config is only allowed to use the
`crdb-v2-tty` format, and therefore, only `format-options` supported by
the `crdb-v2-tty` format should be applied to the stderr sink.

Unfortunately, a bug in the log config parse & validation code didn't
follow this rule. The configuration that's part of the `file-defaults`
is propagated to the stderr sink, and this included the
`format-options`, even if they were only relevant to another log format
type (e.g. `json`). This caused an error when trying to apply the
options to the stderr log sink on startup, e.g.:
```
ERROR: unknown format option: "datetime-format"
```

To solve this problem, we should only propagate the `format-options`
used in `file-defaults` to the stderr sink's config IFF the
`file-defaults` format is of a `crdb-v2` variety. Since the stderr sink
also uses the `crdb-v2-tty` format, we can only be sure that the
`format-options` used in `file-defaults` is supported by the stderr sink
if the `format` used in `file-defaults` is also part of `crdb-v2`.

However, if `format-options` is explicitly defined within the
`sinks.stderr` config, we need to be careful not to overwrite them with
those defined in `file-defaults`.

This patch accomplishes fixes for all these issues, and adds new tests
to cover all these scenarios.

Release note: none


113534: roachtest: test AOST restore in backup-restore/* roachtests r=renatolabs a=msbutler

This patch allows the backup-restore driver to run and validate AOST restores from revision history backups. If the driver created a revision history backup, there's a 50% chance it will restore from an AOST between the full backup end time and the last incremental start time. A future patch will allow for AOST restores from non-revision history backups.

Epic: none

Release note: none

Co-authored-by: Alex Barganier <[email protected]>
Co-authored-by: Michael Butler <[email protected]>
  • Loading branch information
3 people committed Nov 2, 2023
3 parents 8e059c2 + c0addf4 + acae46b commit 4dad7f2
Show file tree
Hide file tree
Showing 5 changed files with 195 additions and 13 deletions.
1 change: 0 additions & 1 deletion pkg/cli/interactive_tests/test_log_flags.tcl
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,6 @@ end_test

start_test "Check that the log flag is properly recognized for non-server commands"
send "$argv debug reset-quorum 123 --log='sinks: {stderr: {format: json }}'\r"
eexpect "\"severity\":\"ERROR\""
eexpect "connection to server failed"
eexpect ":/# "
end_test
Expand Down
93 changes: 87 additions & 6 deletions pkg/cmd/roachtest/tests/mixed_version_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"sync/atomic"
"time"

"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
Expand All @@ -41,6 +42,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
Expand All @@ -65,6 +67,9 @@ const (
// mixed-version state.
mixedVersionRestoreProbability = 0.5

// probability that we will attempt to run an AOST restore.
restoreFromAOSTProbability = 0.5

// string label added to the names of backups taken while the cluster is
// upgrading.
finalizingLabel = "_finalizing"
Expand Down Expand Up @@ -377,6 +382,10 @@ type (
// in the `contents` field.
tables []string
contents []tableContents

// restoreAOST contains the AOST used in restore, if non-empty. It also
// determines the system time used to grab fingerprints.
restoreAOST string
}

fullBackup struct {
Expand Down Expand Up @@ -984,6 +993,68 @@ func (bc *backupCollection) encryptionOption() *encryptionPassphrase {
return nil
}

func (bc *backupCollection) withRevisionHistory() bool {
for _, option := range bc.options {
if _, ok := option.(revisionHistory); ok {
return true
}
}
return false
}

// maybeUseRestoreAOST potentially picks a restore AOST between the
// full backup end time and the last incremental backup end time.
//
// We don't bother choosing an AOST before the full backup endtime since the
// restore may fail. We lack good observability for choosing a valid AOST within
// the revision history full backup.
func (bc *backupCollection) maybeUseRestoreAOST(
l *logger.Logger, rng *rand.Rand, fullBackupEndTime, lastBackupEndTime string,
) error {
// TODO(msbutler): pick AOST restore for non revision history backups by
// randomly choosing a backup end time.
if !bc.withRevisionHistory() || rng.Float64() > restoreFromAOSTProbability {
return nil
}

parseAOST := func(aost string) (hlc.Timestamp, error) {
d, _, err := apd.NewFromString(aost)
if err != nil {
return hlc.Timestamp{}, err
}
ts, err := hlc.DecimalToHLC(d)
if err != nil {
return hlc.Timestamp{}, err
}
return ts, nil
}

min, err := parseAOST(fullBackupEndTime)
if err != nil {
return err
}
max, err := parseAOST(lastBackupEndTime)
if err != nil {
return err
}

// Choose a random AOST between min and max with the following approach:
// divide the interval between min and max over 100 bins and randomly choose a
// bin. Randomly choosing a bin is more reproducible than randomly picking a
// time between min and max.
interval := max.WallTime - min.WallTime
binCount := int64(100)
bin := rng.Int63n(binCount)

restoreAOST := hlc.Timestamp{
WallTime: (bin*interval)/binCount + min.WallTime,
}

l.Printf("preparing for an AOST restore at %s, between full backup end time %s and last incremental backup end time %s", restoreAOST.GoTime(), min.GoTime(), max.GoTime())
bc.restoreAOST = restoreAOST.AsOfSystemTime()
return nil
}

// backupCollectionDesc builds a string that describes how a backup
// collection comprised of a full backup and a follow-up incremental
// backup was generated (in terms of which versions planned vs
Expand Down Expand Up @@ -1846,12 +1917,13 @@ func (d *BackupRestoreTestDriver) createBackupCollection(
internalSystemJobs bool,
) (*backupCollection, error) {
var collection backupCollection
var timestamp string
var latestIncBackupEndTime string
var fullBackupEndTime string

// Create full backup.
if err := d.testUtils.runJobOnOneOf(ctx, l, fullBackupSpec.Execute.Nodes, func() error {
var err error
collection, _, err = d.runBackup(
collection, fullBackupEndTime, err = d.runBackup(
ctx, l, rng, fullBackupSpec.Plan.Nodes, fullBackupSpec.PauseProbability, fullBackup{backupNamePrefix}, internalSystemJobs,
)
return err
Expand All @@ -1866,7 +1938,7 @@ func (d *BackupRestoreTestDriver) createBackupCollection(
d.randomWait(l, rng)
if err := d.testUtils.runJobOnOneOf(ctx, l, incBackupSpec.Execute.Nodes, func() error {
var err error
collection, timestamp, err = d.runBackup(
collection, latestIncBackupEndTime, err = d.runBackup(
ctx, l, rng, incBackupSpec.Plan.Nodes, incBackupSpec.PauseProbability, incrementalBackup{collection: collection, incNum: i + 1}, internalSystemJobs,
)
return err
Expand All @@ -1875,7 +1947,15 @@ func (d *BackupRestoreTestDriver) createBackupCollection(
}
}

return d.saveContents(ctx, l, rng, &collection, timestamp)
if err := collection.maybeUseRestoreAOST(l, rng, fullBackupEndTime, latestIncBackupEndTime); err != nil {
return nil, err
}

fingerprintAOST := latestIncBackupEndTime
if collection.restoreAOST != "" {
fingerprintAOST = collection.restoreAOST
}
return d.saveContents(ctx, l, rng, &collection, fingerprintAOST)
}

// sentinelFilePath returns the path to the file that prevents job
Expand Down Expand Up @@ -2173,9 +2253,10 @@ func (bc *backupCollection) verifyBackupCollection(
optionsStr = fmt.Sprintf(" WITH %s", strings.Join(restoreOptions, ", "))
}
restoreStmt := fmt.Sprintf(
"%s FROM LATEST IN '%s'%s",
restoreCmd, bc.uri(), optionsStr,
"%s FROM LATEST IN '%s'%s %s",
restoreCmd, bc.uri(), aostFor(bc.restoreAOST), optionsStr,
)
l.Printf("Running restore: %s", restoreStmt)
var jobID int
if err := d.testUtils.QueryRow(ctx, rng, restoreStmt).Scan(&jobID); err != nil {
return fmt.Errorf("backup %s: error in restore statement: %w", bc.name, err)
Expand Down
6 changes: 4 additions & 2 deletions pkg/util/log/logconfig/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,10 @@ import (
// specified in a configuration.
const DefaultFileFormat = `crdb-v2`

// DefaultStderrFormat is the entry format for stderr sinks
// when not specified in a configuration.
// DefaultStderrFormat is the entry format for stderr sinks.
// NB: The format for stderr is always set to `crdb-v2-tty`,
// and cannot be changed. We enforce this in the validation step.
// See: https://www.cockroachlabs.com/docs/stable/configure-logs#output-to-stderr
const DefaultStderrFormat = `crdb-v2-tty`

// DefaultFluentFormat is the entry format for fluent sinks
Expand Down
87 changes: 87 additions & 0 deletions pkg/util/log/logconfig/testdata/validate
Original file line number Diff line number Diff line change
Expand Up @@ -254,6 +254,93 @@ capture-stray-errors:
dir: /default-dir
max-group-size: 100MiB

# Check that file-defaults format options are transferred to stderr if using a crdb-v2 format.
yaml
file-defaults:
format: crdb-v2
format-options: {timezone: america/new_york}
----
sinks:
file-groups:
default:
channels: {INFO: all}
filter: INFO
format-options:
timezone: america/new_york
stderr:
filter: NONE
format-options:
timezone: america/new_york
capture-stray-errors:
enable: true
dir: /default-dir
max-group-size: 100MiB

# Check that file-defaults format options are NOT transferred to stderr if NOT using a crdb-v2 format.
yaml
file-defaults:
format: json
format-options: {datetime-format: rfc3339, datetime-timezone: America/New_York}
----
sinks:
file-groups:
default:
channels: {INFO: all}
filter: INFO
format: json
format-options:
datetime-format: rfc3339
datetime-timezone: America/New_York
stderr:
filter: NONE
capture-stray-errors:
enable: true
dir: /default-dir
max-group-size: 100MiB

# Check that file-defaults format options do NOT overwrite format-options if explicitly defined in stderr.
yaml
file-defaults:
format: crdb-v2
format-options: {timezone: america/new_york}
sinks:
stderr:
format-options: {timezone: america/chicago}
----
sinks:
file-groups:
default:
channels: {INFO: all}
filter: INFO
format-options:
timezone: america/new_york
stderr:
filter: NONE
format-options:
timezone: america/chicago
capture-stray-errors:
enable: true
dir: /default-dir
max-group-size: 100MiB

# Check that stderr always uses crdb-v2-tty format, even if we try to set it to an invalid format
yaml
sinks:
stderr:
format: crdb-v1-tty
----
sinks:
file-groups:
default:
channels: {INFO: all}
filter: INFO
stderr:
filter: NONE
capture-stray-errors:
enable: true
dir: /default-dir
max-group-size: 100MiB

# Check that NONE filter elides files.
yaml
file-defaults: {filter: NONE}
Expand Down
21 changes: 17 additions & 4 deletions pkg/util/log/logconfig/validate.go
Original file line number Diff line number Diff line change
Expand Up @@ -170,15 +170,28 @@ func (c *Config) Validate(defaultLogDir *string) (resErr error) {
if c.Sinks.Stderr.Filter == logpb.Severity_UNKNOWN {
c.Sinks.Stderr.Filter = logpb.Severity_NONE
}
// We need to know if format-options were specifically defined on the stderr sink later on,
// since this information is lost once propagateCommonDefaults is called.
stdErrFormatOptionsOriginallySet := len(c.Sinks.Stderr.FormatOptions) > 0
propagateCommonDefaults(&c.Sinks.Stderr.CommonSinkConfig, c.FileDefaults.CommonSinkConfig)
if c.Sinks.Stderr.Auditable != nil && *c.Sinks.Stderr.Auditable {
if *c.Sinks.Stderr.Format == "crdb-v1-tty" {
f := "crdb-v1-tty-count"
c.Sinks.Stderr.Format = &f
}
c.Sinks.Stderr.Criticality = &bt
}
c.Sinks.Stderr.Auditable = nil
// The format parameter for stderr is set to `crdb-v2-tty` and cannot be changed.
// See docs: https://www.cockroachlabs.com/docs/stable/configure-logs#output-to-stderr
if *c.Sinks.Stderr.Format != DefaultStderrFormat {
f := DefaultStderrFormat
c.Sinks.Stderr.Format = &f
}
// FormatOptions are format-specific. We should only copy them over to StdErr from
// FileDefaults if FileDefaults is also making use of a crdb-v2 format. Otherwise,
// we are likely to error when trying to apply an unsupported format option.
if c.FileDefaults.CommonSinkConfig.Format != nil &&
!strings.Contains(*c.FileDefaults.CommonSinkConfig.Format, "v2") &&
!stdErrFormatOptionsOriginallySet {
c.Sinks.Stderr.CommonSinkConfig.FormatOptions = map[string]string{}
}
if err := c.ValidateCommonSinkConfig(c.Sinks.Stderr.CommonSinkConfig); err != nil {
fmt.Fprintf(&errBuf, "stderr sink: %v\n", err)
}
Expand Down

0 comments on commit 4dad7f2

Please sign in to comment.