From baf49d13a22c6820728a7da396588ea2cca71d99 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Thu, 14 Jan 2021 14:10:17 +0100 Subject: [PATCH] util/log: stop syncing writes excessively Release note (cli change): Previously, for certain log files CockroachDB would both flush individual writes (i.e. propagate them from within the `cockroach` process to the OS) and also synchronize writes (i.e. ask the OS to confirm the log data was written to disk). The per-write synchronization part was unnecessary and, in fact, found to be possibly detrimental to performance and operating cost, so it was removed. Meanwhile, the log data continues to be flushed as previously, and CockroachDB also periodically (every 30s) request synchronization, also as previously. Release note (cli change): The parameter `sync-writes` for file sink configurations has been removed. (This is not a backward-incompatible change because the configuration feature is new in v21.1.) Release note (cli change): The parameter `buffered-writes` for file sink configurations has been added. It is set to `true` (writes are buffered) by default; and set to `false` (i.e. avoid buffering and flush every log entry) when the `auditable` flag is requested. --- pkg/cli/log_flags_test.go | 11 +- pkg/cli/start.go | 14 +- pkg/cli/testdata/logflags | 170 +++++++++++------------ pkg/util/log/clog.go | 6 +- pkg/util/log/file.go | 28 ++-- pkg/util/log/file_log_gc_test.go | 4 +- pkg/util/log/flags.go | 20 +-- pkg/util/log/log_flush.go | 10 +- pkg/util/log/logconfig/config.go | 12 +- pkg/util/log/logconfig/export.go | 6 +- pkg/util/log/logconfig/testdata/export | 14 +- pkg/util/log/logconfig/testdata/validate | 22 ++- pkg/util/log/logconfig/validate.go | 12 +- pkg/util/log/testdata/config | 6 +- 14 files changed, 176 insertions(+), 159 deletions(-) diff --git a/pkg/cli/log_flags_test.go b/pkg/cli/log_flags_test.go index 95d492fa2506..fa99b57613b7 100644 --- a/pkg/cli/log_flags_test.go +++ b/pkg/cli/log_flags_test.go @@ -35,21 +35,22 @@ func TestSetupLogging(t *testing.T) { reSimplify := regexp.MustCompile(`(?ms:^\s*(auditable: false|redact: false|exit-on-error: true|max-group-size: 100MiB)\n)`) const defaultFluentConfig = `fluent-defaults: {` + + `buffered-writes: true, ` + `filter: INFO, ` + `format: json-fluent-compact, ` + `redactable: true, ` + `exit-on-error: false` + `}, ` stdFileDefaultsRe := regexp.MustCompile( - `file-defaults: \{dir: (?P[^,]+), max-file-size: 10MiB, filter: INFO, format: crdb-v1, redactable: true\}`) + `file-defaults: \{dir: (?P[^,]+), max-file-size: 10MiB, buffered-writes: true, filter: INFO, format: crdb-v1, redactable: true\}`) fileDefaultsNoMaxSizeRe := regexp.MustCompile( - `file-defaults: \{dir: (?P[^,]+), filter: INFO, format: crdb-v1, redactable: true\}`) - const fileDefaultsNoDir = `file-defaults: {filter: INFO, format: crdb-v1, redactable: true}` + `file-defaults: \{dir: (?P[^,]+), buffered-writes: true, filter: INFO, format: crdb-v1, redactable: true\}`) + const fileDefaultsNoDir = `file-defaults: {buffered-writes: true, filter: INFO, format: crdb-v1, redactable: true}` const defaultLogDir = `PWD/cockroach-data/logs` stdCaptureFd2Re := regexp.MustCompile( `capture-stray-errors: \{enable: true, dir: (?P[^}]+)\}`) fileCfgRe := regexp.MustCompile( - `\{channels: (?P[^ ]+), dir: (?P[^,]+), max-file-size: 10MiB, sync-writes: (?P[^,]+), filter: INFO, format: (?P[^,]+), redactable: true\}`) + `\{channels: (?P[^ ]+), dir: (?P[^,]+), max-file-size: 10MiB, buffered-writes: (?P[^,]+), filter: INFO, format: (?P[^,]+), redactable: true\}`) stderrCfgRe := regexp.MustCompile( `stderr: {channels: all, filter: (?P[^,]+), format: crdb-v1-tty, redactable: (?P[^}]+)}`) @@ -110,7 +111,7 @@ func TestSetupLogging(t *testing.T) { actual = fileDefaultsNoMaxSizeRe.ReplaceAllString(actual, "") actual = strings.ReplaceAll(actual, fileDefaultsNoDir, "") actual = stdCaptureFd2Re.ReplaceAllString(actual, "") - actual = fileCfgRe.ReplaceAllString(actual, "") + actual = fileCfgRe.ReplaceAllString(actual, "") actual = stderrCfgRe.ReplaceAllString(actual, "") actual = strings.ReplaceAll(actual, ``, ``) actual = strings.ReplaceAll(actual, ``, ``) diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 5cc8686a2454..880537db2efc 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -717,24 +717,26 @@ If problems persist, please see %s.` // is stopped externally (for example, via the quit endpoint). select { case err := <-errChan: - // SetSync both flushes and ensures that subsequent log writes are flushed too. - log.StartSync() + // StartAlwaysFlush both flushes and ensures that subsequent log + // writes are flushed too. + log.StartAlwaysFlush() return err case <-stopper.ShouldStop(): // Server is being stopped externally and our job is finished // here since we don't know if it's a graceful shutdown or not. <-stopper.IsStopped() - // StartSync both flushes and ensures that subsequent log writes are flushed too. - log.StartSync() + // StartAlwaysFlush both flushes and ensures that subsequent log + // writes are flushed too. + log.StartAlwaysFlush() return nil case sig := <-signalCh: - // We start synchronizing log writes from here, because if a + // We start flushing log writes from here, because if a // signal was received there is a non-zero chance the sender of // this signal will follow up with SIGKILL if the shutdown is not // timely, and we don't want logs to be lost. - log.StartSync() + log.StartAlwaysFlush() log.Ops.Infof(shutdownCtx, "received signal '%s'", sig) switch sig { diff --git a/pkg/cli/testdata/logflags b/pkg/cli/testdata/logflags index f3622b7c1168..82abdf18554b 100644 --- a/pkg/cli/testdata/logflags +++ b/pkg/cli/testdata/logflags @@ -13,13 +13,13 @@ run start ---- config: {)>, -sinks: {file-groups: {default: ,false,crdb-v1)>, -pebble: ,false,crdb-v1)>, -sql-audit: ,true,crdb-v1-count)>, -sql-auth: ,true,crdb-v1-count)>, -sql-exec: ,false,crdb-v1)>, -sql-slow: ,false,crdb-v1)>, -sql-slow-internal-only: ,false,crdb-v1)>}, +sinks: {file-groups: {default: ,true,crdb-v1)>, +pebble: ,true,crdb-v1)>, +sql-audit: ,false,crdb-v1-count)>, +sql-auth: ,false,crdb-v1-count)>, +sql-exec: ,true,crdb-v1)>, +sql-slow: ,true,crdb-v1)>, +sql-slow-internal-only: ,true,crdb-v1)>}, }, )>} @@ -28,13 +28,13 @@ run start-single-node ---- config: {)>, -sinks: {file-groups: {default: ,false,crdb-v1)>, -pebble: ,false,crdb-v1)>, -sql-audit: ,true,crdb-v1-count)>, -sql-auth: ,true,crdb-v1-count)>, -sql-exec: ,false,crdb-v1)>, -sql-slow: ,false,crdb-v1)>, -sql-slow-internal-only: ,false,crdb-v1)>}, +sinks: {file-groups: {default: ,true,crdb-v1)>, +pebble: ,true,crdb-v1)>, +sql-audit: ,false,crdb-v1-count)>, +sql-auth: ,false,crdb-v1-count)>, +sql-exec: ,true,crdb-v1)>, +sql-slow: ,true,crdb-v1)>, +sql-slow-internal-only: ,true,crdb-v1)>}, }, )>} @@ -98,13 +98,13 @@ start --store=path=/pathB ---- config: {, -sinks: {file-groups: {default: , -pebble: , -sql-audit: , -sql-auth: , -sql-exec: , -sql-slow: , -sql-slow-internal-only: }, +sinks: {file-groups: {default: , +pebble: , +sql-audit: , +sql-auth: , +sql-exec: , +sql-slow: , +sql-slow-internal-only: }, }, } @@ -115,13 +115,13 @@ start --log=file-defaults: {dir: /mypath} ---- config: {, -sinks: {file-groups: {default: , -pebble: , -sql-audit: , -sql-auth: , -sql-exec: , -sql-slow: , -sql-slow-internal-only: }, +sinks: {file-groups: {default: , +pebble: , +sql-audit: , +sql-auth: , +sql-exec: , +sql-slow: , +sql-slow-internal-only: }, }, } @@ -133,13 +133,13 @@ start --log=file-defaults: {dir: /pathA/logs} ---- config: {, -sinks: {file-groups: {default: , -pebble: , -sql-audit: , -sql-auth: , -sql-exec: , -sql-slow: , -sql-slow-internal-only: }, +sinks: {file-groups: {default: , +pebble: , +sql-audit: , +sql-auth: , +sql-exec: , +sql-slow: , +sql-slow-internal-only: }, }, } @@ -156,13 +156,13 @@ start --log=file-defaults: {dir: /mypath} ---- config: {, -sinks: {file-groups: {default: , -pebble: , -sql-audit: , -sql-auth: , -sql-exec: , -sql-slow: , -sql-slow-internal-only: }, +sinks: {file-groups: {default: , +pebble: , +sql-audit: , +sql-auth: , +sql-exec: , +sql-slow: , +sql-slow-internal-only: }, }, } @@ -172,13 +172,13 @@ start --log=sinks: {stderr: {filter: ERROR}} ---- config: {)>, -sinks: {file-groups: {default: ,false,crdb-v1)>, -pebble: ,false,crdb-v1)>, -sql-audit: ,true,crdb-v1-count)>, -sql-auth: ,true,crdb-v1-count)>, -sql-exec: ,false,crdb-v1)>, -sql-slow: ,false,crdb-v1)>, -sql-slow-internal-only: ,false,crdb-v1)>}, +sinks: {file-groups: {default: ,true,crdb-v1)>, +pebble: ,true,crdb-v1)>, +sql-audit: ,false,crdb-v1-count)>, +sql-auth: ,false,crdb-v1-count)>, +sql-exec: ,true,crdb-v1)>, +sql-slow: ,true,crdb-v1)>, +sql-slow-internal-only: ,true,crdb-v1)>}, }, )>} @@ -189,13 +189,13 @@ start --log=capture-stray-errors: {enable: false} ---- config: {)>, -sinks: {file-groups: {default: ,false,crdb-v1)>, -pebble: ,false,crdb-v1)>, -sql-audit: ,true,crdb-v1-count)>, -sql-auth: ,true,crdb-v1-count)>, -sql-exec: ,false,crdb-v1)>, -sql-slow: ,false,crdb-v1)>, -sql-slow-internal-only: ,false,crdb-v1)>}, +sinks: {file-groups: {default: ,true,crdb-v1)>, +pebble: ,true,crdb-v1)>, +sql-audit: ,false,crdb-v1-count)>, +sql-auth: ,false,crdb-v1-count)>, +sql-exec: ,true,crdb-v1)>, +sql-slow: ,true,crdb-v1)>, +sql-slow-internal-only: ,true,crdb-v1)>}, }} # Logging to stderr without stderr capture causes an error in the default config. @@ -236,13 +236,13 @@ start --log-dir=/mypath ---- config: {, -sinks: {file-groups: {default: , -pebble: , -sql-audit: , -sql-auth: , -sql-exec: , -sql-slow: , -sql-slow-internal-only: }, +sinks: {file-groups: {default: , +pebble: , +sql-audit: , +sql-auth: , +sql-exec: , +sql-slow: , +sql-slow-internal-only: }, }, } @@ -254,13 +254,13 @@ start --log-dir=/pathA ---- config: {, -sinks: {file-groups: {default: , -pebble: , -sql-audit: , -sql-auth: , -sql-exec: , -sql-slow: , -sql-slow-internal-only: }, +sinks: {file-groups: {default: , +pebble: , +sql-audit: , +sql-auth: , +sql-exec: , +sql-slow: , +sql-slow-internal-only: }, }, } @@ -274,7 +274,7 @@ init config: {, sinks: {file-groups: {default: {channels: all, dir: /mypath, -sync-writes: false, +buffered-writes: true, filter: INFO, format: crdb-v1, redactable: true}}, @@ -287,13 +287,13 @@ start --logtostderr=INFO ---- config: {)>, -sinks: {file-groups: {default: ,false,crdb-v1)>, -pebble: ,false,crdb-v1)>, -sql-audit: ,true,crdb-v1-count)>, -sql-auth: ,true,crdb-v1-count)>, -sql-exec: ,false,crdb-v1)>, -sql-slow: ,false,crdb-v1)>, -sql-slow-internal-only: ,false,crdb-v1)>}, +sinks: {file-groups: {default: ,true,crdb-v1)>, +pebble: ,true,crdb-v1)>, +sql-audit: ,false,crdb-v1-count)>, +sql-auth: ,false,crdb-v1-count)>, +sql-exec: ,true,crdb-v1)>, +sql-slow: ,true,crdb-v1)>, +sql-slow-internal-only: ,true,crdb-v1)>}, }, )>} @@ -303,13 +303,13 @@ start --logtostderr ---- config: {)>, -sinks: {file-groups: {default: ,false,crdb-v1)>, -pebble: ,false,crdb-v1)>, -sql-audit: ,true,crdb-v1-count)>, -sql-auth: ,true,crdb-v1-count)>, -sql-exec: ,false,crdb-v1)>, -sql-slow: ,false,crdb-v1)>, -sql-slow-internal-only: ,false,crdb-v1)>}, +sinks: {file-groups: {default: ,true,crdb-v1)>, +pebble: ,true,crdb-v1)>, +sql-audit: ,false,crdb-v1-count)>, +sql-auth: ,false,crdb-v1-count)>, +sql-exec: ,true,crdb-v1)>, +sql-slow: ,true,crdb-v1)>, +sql-slow-internal-only: ,true,crdb-v1)>}, }, )>} diff --git a/pkg/util/log/clog.go b/pkg/util/log/clog.go index ce4ed0c6f75a..97b24fa1e6ba 100644 --- a/pkg/util/log/clog.go +++ b/pkg/util/log/clog.go @@ -264,10 +264,10 @@ func (l *loggerT) outputLogEntry(entry logEntry) { // are disabled. See IsActive() and its callers for details. setActive() var fatalTrigger chan struct{} - extraSync := false + extraFlush := false if entry.sev == severity.FATAL { - extraSync = true + extraFlush = true logging.signalFatalCh() switch traceback { @@ -377,7 +377,7 @@ func (l *loggerT) outputLogEntry(entry logEntry) { // The sink was not accepting entries at this level. Nothing to do. continue } - if err := s.sink.output(extraSync, bufs.b[i].Bytes()); err != nil { + if err := s.sink.output(extraFlush, bufs.b[i].Bytes()); err != nil { if !s.criticality { // An error on this sink is not critical. Just report // the error and move on. diff --git a/pkg/util/log/file.go b/pkg/util/log/file.go index f792a96e8bbe..3bddcbab22b3 100644 --- a/pkg/util/log/file.go +++ b/pkg/util/log/file.go @@ -64,12 +64,12 @@ type fileSink struct { // name prefix for log files. prefix string - // syncWrites if true calls file.Flush and file.Sync on every log + // bufferedWrites if false calls file.Flush on every log // write. This can be set per-logger e.g. for audit logging. // // Note that synchronization for all log files simultaneously can - // also be configured via logging.syncWrites, see SetSync(). - syncWrites bool + // also be configured via logging.flushWrites, see SetAlwaysFlush(). + bufferedWrites bool // logFileMaxSize is the maximum size of a log file in bytes. logFileMaxSize int64 @@ -128,7 +128,7 @@ type fileSink struct { // newFileSink creates a new file sink. func newFileSink( dir, fileNamePrefix string, - forceSyncWrites bool, + bufferedWrites bool, fileMaxSize, combinedMaxSize int64, getStartLines func(time.Time) []*buffer, ) *fileSink { @@ -138,7 +138,7 @@ func newFileSink( } f := &fileSink{ prefix: prefix, - syncWrites: forceSyncWrites, + bufferedWrites: bufferedWrites, logFileMaxSize: fileMaxSize, logFilesCombinedMaxSize: combinedMaxSize, gcNotify: make(chan struct{}, 1), @@ -168,7 +168,7 @@ func (l *fileSink) attachHints(stacks []byte) []byte { } // output implements the logSink interface. -func (l *fileSink) output(extraSync bool, b []byte) error { +func (l *fileSink) output(extraFlush bool, b []byte) error { if !l.enabled.Get() { // NB: we need to check filesink.enabled a second time here in // case a test Scope() has disabled it asynchronously while @@ -187,8 +187,8 @@ func (l *fileSink) output(extraSync bool, b []byte) error { return err } - if extraSync || l.syncWrites || logging.syncWrites.Get() { - l.flushAndSyncLocked(true /*doSync*/) + if extraFlush || !l.bufferedWrites || logging.flushWrites.Get() { + l.flushAndMaybeSyncLocked(false /*doSync*/) } return nil } @@ -214,24 +214,24 @@ func (l *fileSink) emergencyOutput(b []byte) { // During an emergency, we flush to get the data out to the OS, but // we don't care as much about persistence. In fact, trying too hard // to sync may cause additional stoppage. - l.flushAndSyncLocked(false /*doSync*/) + l.flushAndMaybeSyncLocked(false /*doSync*/) } -// lockAndFlushAndSync is like flushAndSync but locks l.mu first. -func (l *fileSink) lockAndFlushAndSync(doSync bool) { +// lockAndFlushAndMaybeSync is like flushAndMaybeSyncLocked but locks l.mu first. +func (l *fileSink) lockAndFlushAndMaybeSync(doSync bool) { if l == nil { return } l.mu.Lock() defer l.mu.Unlock() - l.flushAndSyncLocked(doSync) + l.flushAndMaybeSyncLocked(doSync) } -// flushAndSync flushes the current log and, if doSync is set, +// flushAndMaybeSyncLocked flushes the current log and, if doSync is set, // attempts to sync its data to disk. // // l.mu is held. -func (l *fileSink) flushAndSyncLocked(doSync bool) { +func (l *fileSink) flushAndMaybeSyncLocked(doSync bool) { if l.mu.file == nil { return } diff --git a/pkg/util/log/file_log_gc_test.go b/pkg/util/log/file_log_gc_test.go index 94e88a73718f..bf105d08ae84 100644 --- a/pkg/util/log/file_log_gc_test.go +++ b/pkg/util/log/file_log_gc_test.go @@ -47,13 +47,13 @@ func TestSecondaryGC(t *testing.T) { f := logconfig.DefaultFileFormat common := logconfig.DefaultConfig().FileDefaults.CommonSinkConfig common.Format = &f - bt := true + bf := false fc := logconfig.FileConfig{ CommonSinkConfig: common, Dir: &s.logDir, MaxFileSize: &m, MaxGroupSize: &m, - SyncWrites: &bt, + BufferedWrites: &bf, } logger := &loggerT{} si, fileSink, err := newFileSinkInfo("gctest", fc) diff --git a/pkg/util/log/flags.go b/pkg/util/log/flags.go index 24b80f906114..3a831a750df1 100644 --- a/pkg/util/log/flags.go +++ b/pkg/util/log/flags.go @@ -30,10 +30,10 @@ type config struct { // used for testing. showLogs bool - // syncWrites can be set asynchronously to force all file output to - // synchronize to disk. This is set via SetSync() and used e.g. in - // start.go upon encountering errors. - syncWrites syncutil.AtomicBool + // flushWrites can be set asynchronously to force all file output to + // be flushed to disk immediately. This is set via SetAlwaysFlush() + // and used e.g. in start.go upon encountering errors. + flushWrites syncutil.AtomicBool } var debugLog *loggerT @@ -165,10 +165,10 @@ func ApplyConfig(config logconfig.Config) (cleanupFn func(), err error) { // impression to the entry parser. Redactable: &bf, }, - Dir: config.CaptureFd2.Dir, - MaxGroupSize: config.CaptureFd2.MaxGroupSize, - MaxFileSize: &mf, - SyncWrites: &bt, + Dir: config.CaptureFd2.Dir, + MaxGroupSize: config.CaptureFd2.MaxGroupSize, + MaxFileSize: &mf, + BufferedWrites: &bf, } fileSinkInfo, fileSink, err := newFileSinkInfo("stderr", fakeConfig) if err != nil { @@ -295,7 +295,7 @@ func newFileSinkInfo(fileNamePrefix string, c logconfig.FileConfig) (*sinkInfo, fileSink := newFileSink( *c.Dir, fileNamePrefix, - *c.SyncWrites, + *c.BufferedWrites, int64(*c.MaxFileSize), int64(*c.MaxGroupSize), info.getStartLines) @@ -404,7 +404,7 @@ func DescribeAppliedConfig() string { dir := fileSink.mu.logDir fileSink.mu.Unlock() fc.Dir = &dir - fc.SyncWrites = &fileSink.syncWrites + fc.BufferedWrites = &fileSink.bufferedWrites // Describe the connections to this file sink. for ch, logger := range chans { diff --git a/pkg/util/log/log_flush.go b/pkg/util/log/log_flush.go index bebf85277b64..13729b2fd330 100644 --- a/pkg/util/log/log_flush.go +++ b/pkg/util/log/log_flush.go @@ -32,7 +32,7 @@ type flushSyncWriter interface { // user signal. func Flush() { _ = allSinkInfos.iterFileSinks(func(l *fileSink) error { - l.lockAndFlushAndSync(true /*doSync*/) + l.lockAndFlushAndMaybeSync(true /*doSync*/) return nil }) } @@ -84,7 +84,7 @@ func flushDaemon() { if !disableDaemons { // Flush the loggers. _ = allSinkInfos.iterFileSinks(func(l *fileSink) error { - l.lockAndFlushAndSync(doSync) + l.lockAndFlushAndMaybeSync(doSync) return nil }) } @@ -101,13 +101,13 @@ func signalFlusher() { } } -// StartSync configures all loggers to start synchronizing writes. +// StartAlwaysFlush configures all loggers to start flushing writes. // This is used e.g. in `cockroach start` when an error occurs, // to ensure that all log writes from the point the error // occurs are flushed to logs (in case the error degenerates // into a panic / segfault on the way out). -func StartSync() { - logging.syncWrites.Set(true) +func StartAlwaysFlush() { + logging.flushWrites.Set(true) // There may be something in the buffers already; flush it. Flush() } diff --git a/pkg/util/log/logconfig/config.go b/pkg/util/log/logconfig/config.go index 060cb5cc3825..1de794316759 100644 --- a/pkg/util/log/logconfig/config.go +++ b/pkg/util/log/logconfig/config.go @@ -43,6 +43,7 @@ file-defaults: max-file-size: 10mib max-group-size: 100mib exit-on-error: true + buffered-writes: true sinks: stderr: filter: NONE @@ -188,9 +189,10 @@ type FileDefaults struct { // If zero, old files are not removed. MaxGroupSize ByteSize `yaml:"max-group-size,omitempty"` - // SyncWrites stores the default setting for sync-writes on file - // sinks, which implies synchronization on every log write. - SyncWrites bool `yaml:"sync-writes,omitempty"` + // BufferedWrites stores the default setting for buffered-writes on + // file sinks, which implies keeping a buffer of log entries in memory. + // Conversely, setting this to false flushes log writes upon every entry. + BufferedWrites *bool `yaml:"buffered-writes,omitempty"` // CommonSinkConfig is the configuration common to all sinks. Note // that although the idiom in Go is to place embedded fields at the @@ -220,8 +222,8 @@ type FileConfig struct { // beyond this specified size. MaxGroupSize *ByteSize `yaml:"max-group-size,omitempty"` - // SyncWrites specifies whether to sync on every log write. - SyncWrites *bool `yaml:"sync-writes,omitempty"` + // BufferedWrites specifies whether to flush on every log write. + BufferedWrites *bool `yaml:"buffered-writes,omitempty"` // CommonSinkConfig is the configuration common to all sinks. Note // that although the idiom in Go is to place embedded fields at the diff --git a/pkg/util/log/logconfig/export.go b/pkg/util/log/logconfig/export.go index ab82d1dca0f3..a2c0f4b30946 100644 --- a/pkg/util/log/logconfig/export.go +++ b/pkg/util/log/logconfig/export.go @@ -121,10 +121,10 @@ func (c *Config) Export(onlyChans ChannelList) (string, string) { target := fileKey var syncproc, synclink []string - if *fc.SyncWrites { - skey := fmt.Sprintf("sync%d", fileNum) + if *fc.BufferedWrites { + skey := fmt.Sprintf("buffer%d", fileNum) fileNum++ - syncproc = append(syncproc, fmt.Sprintf("card %s as \"sync\"", skey)) + syncproc = append(syncproc, fmt.Sprintf("card %s as \"buffer\"", skey)) synclink = append(synclink, fmt.Sprintf("%s --> %s", skey, target)) fileNum++ target = skey diff --git a/pkg/util/log/logconfig/testdata/export b/pkg/util/log/logconfig/testdata/export index ffdb22894a02..a027a41a4339 100644 --- a/pkg/util/log/logconfig/testdata/export +++ b/pkg/util/log/logconfig/testdata/export @@ -19,6 +19,7 @@ component sources { cloud stray as "stray\nerrors" } queue stderr +card buffer2 as "buffer" card p__1 as "format:crdb-v1" artifact files { folder "/default-dir" { @@ -38,17 +39,18 @@ SENSITIVE_ACCESS --> p__1 SQL_EXEC --> p__1 SQL_PERF --> p__1 SQL_INTERNAL_PERF --> p__1 -p__1 --> f1 +p__1 --> buffer2 +buffer2 --> f1 stray --> stderrfile @enduml -# http://www.plantuml.com/plantuml/uml/L59DZvim5BpxLpnnQW-qohM7QX5vtI1bWMAAUgY4N7_ieXAyVPXALPN_NjcGTJXvPjxmW1a_p4wGMouZ6xLnu2pGyFpYG0safHlinAIzlDf9Jmvckv3KC_nZxzx34Jj_L3NtnmcpStloYBTLaprYWCXvKPN1mB-UUvwVy2dpx1l7fi-EJqNfMTqKNN76HroTMFAYBJhiipn7_lOqViVyXcjiljnmKRRObDacoj4k2cP7uY-86PA0VaoQoDASi2lxlUX5m-oK9c9Ia8BNlZy4hx5q4UwJ9FKp_ND8c20t624Tc64Clmt6ZaeJ91-LDc8PNQe6IlmWMC2iUwILlyWA-V9XjC_9RRY-Ci2TALq7c_VAhko87QJfv_1XhAfv96lCaMzXtmbRv7VAM_1HY_57BPPmlsrVnNwhBsIl-0RsogwSE0gHUsOEREtAavWGUz2JyfVoVm000F__ +# http://www.plantuml.com/plantuml/uml/L99FZzCm4CNl_XIZdE0GK3boG1krmpPIDornYJWWHLx_sOr8usNY827Kxuvi9tLpydjlNYnhVdaOdI0tNNhMQ-F0MQ3kvTM1waXBrzc1INjviuCU78ns8gb7-CVUlOSTDluudhW_zbWSpdkl-FbO5uyO979U7gjGuDyEBT_kyQdmxZl7kYrsJsNbtQakc_A0ZnYRMF7oN3RO5jijyjlN-0EtYpvX_NNHPNN6kYhcKFPsKZ0w4dz1Z905zNFGH9R6Z5tPxqbF6aQdD16JWXGyJyPe-XJgKMTnyDQscv0QInVXFajIp_cVJSO4kSu8wS1qVTW76DihJP1zLDg8gNUvwYZpWr014--HLluYA-JhXzw-PCimlYN8LIcFWxbxPLUsmmRo_4ju63kUU39n_ydF5Ew2cSLTSYEIiiGbPOdE_MrhHkjRFQbruj6ianMrD0ehzsw6mXOuFZCR5i5waxR67lIW_1_yFm00__y0 # Capture everything to one file with sync and warnings only to stderr. yaml only-channels=DEV,SESSIONS sinks: file-groups: everything: - sync-writes: true + buffered-writes: false redact: true channels: ALL stderr: @@ -62,7 +64,6 @@ component sources { cloud stray as "stray\nerrors" } queue stderr -card sync2 as "sync" card p__1 as "redact" card p__2 as "format:crdb-v1" card p__3 as "format:crdb-v1-tty" @@ -75,13 +76,12 @@ artifact files { } DEV --> p__2 SESSIONS --> p__2 -p__1 --> sync2 +p__1 --> f1 p__2 --> p__1 -sync2 --> f1 stray --> stderrfile DEV --> p__4 SESSIONS --> p__4 p__3 --> stderr p__4 --> p__3 @enduml -# http://www.plantuml.com/plantuml/uml/R98nJ_Cm48Rt-nKdJzyt11JQgGFgq0uiC4Gg2r9bx7DhuThbVAaKeVvt5Bib89XYl-yJ9_V8oooQfJy42EG49I7xtLxGUYOZFaKmwN1CaQ9WJZqRolW1__xZQhqP7zswwnwU7Zim8VKMix0UK6TKPVKIYJbnLd26zvvwmYoMcC5ejfY7QEugF4IZQdZSRjkICLbjP4ehwH8Vj2mCszVcr4xjx8-s4HacObu97uHuyQn0itYdZQ3peGo5BWLBZEhMajDzaCPwLcDH47JrlqmoRvoqsJTq8Xvax-Fk9gITkd9rnBByoTVYmfxX3Alr1flclam7LvDJKbICko8AYeDBsKALDsvT2rLxGRy-_ltq-Q_Jvr2aJQz0KNHfPx2aQCTRyHa00F__ +# http://www.plantuml.com/plantuml/uml/R94nJpCn38Pt_mehq_SD1phQgGFgq0uiC9nK5gGg94uRaPwBuwjKeVvtTBah1u8fjjydZlrccTMATeS4YOAYCahSxHLz578QkGN7XoEtr2fcxiHHnW_uznzNwqr_DEkcUNXwRC0bxZnc5Nj6cz6KwAKb4PPiu0Bl7NM4MJs9WBFYyRZTreKLyjQf-QhUbMfWELXTEF6lrQcUrDaVQgLwdeZvGCIa98jd0rOq1kiKGqnbVWoSF0cQMq_1Taah7yNqGa4m37CvTc_2rkqhEf6STH_RtKtYdRbompOb_CaFmiXUu0AhzGQhwhvi1rVJfagneiz23SM0KQbXSBHFfyU-Tvl_wZQ7Oj9q1Oebepg39RM-__3F0000__y0 diff --git a/pkg/util/log/logconfig/testdata/validate b/pkg/util/log/logconfig/testdata/validate index 870a8d45683b..5dda45ff89dd 100644 --- a/pkg/util/log/logconfig/testdata/validate +++ b/pkg/util/log/logconfig/testdata/validate @@ -5,6 +5,7 @@ file-defaults: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -18,7 +19,7 @@ sinks: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB - sync-writes: false + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -47,6 +48,7 @@ file-defaults: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -60,7 +62,7 @@ sinks: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB - sync-writes: false + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -91,6 +93,7 @@ file-defaults: dir: /custom max-file-size: 10MiB max-group-size: 100MiB + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -104,7 +107,7 @@ sinks: dir: /custom max-file-size: 10MiB max-group-size: 100MiB - sync-writes: false + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -136,6 +139,7 @@ file-defaults: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB + buffered-writes: true filter: WARNING format: crdb-v1 redact: false @@ -149,7 +153,7 @@ sinks: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB - sync-writes: false + buffered-writes: true filter: WARNING format: crdb-v1 redact: false @@ -179,6 +183,7 @@ file-defaults: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -192,7 +197,7 @@ sinks: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB - sync-writes: false + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -222,6 +227,7 @@ file-defaults: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -235,7 +241,7 @@ sinks: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB - sync-writes: true + buffered-writes: false filter: INFO format: crdb-v1-count redact: false @@ -266,6 +272,7 @@ file-defaults: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -279,7 +286,7 @@ sinks: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB - sync-writes: false + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -305,6 +312,7 @@ file-defaults: dir: /default-dir max-file-size: 10MiB max-group-size: 100MiB + buffered-writes: true filter: NONE format: crdb-v1 redact: false diff --git a/pkg/util/log/logconfig/validate.go b/pkg/util/log/logconfig/validate.go index 9cd2b1982025..52dbfefdbd79 100644 --- a/pkg/util/log/logconfig/validate.go +++ b/pkg/util/log/logconfig/validate.go @@ -50,6 +50,10 @@ func (c *Config) Validate(defaultLogDir *string) (resErr error) { if c.FileDefaults.Auditable == nil { c.FileDefaults.Auditable = &bf } + // File sinks are buffered by default. + if c.FileDefaults.BufferedWrites == nil { + c.FileDefaults.BufferedWrites = &bt + } // No format -> populate defaults. if c.FileDefaults.Format == nil { s := DefaultFileFormat @@ -229,8 +233,8 @@ func (c *Config) validateFileConfig(fc *FileConfig, defaultLogDir *string) error if fc.MaxGroupSize == nil { fc.MaxGroupSize = &c.FileDefaults.MaxGroupSize } - if fc.SyncWrites == nil { - fc.SyncWrites = &c.FileDefaults.SyncWrites + if fc.BufferedWrites == nil { + fc.BufferedWrites = c.FileDefaults.BufferedWrites } // Set up the directory. @@ -253,8 +257,8 @@ func (c *Config) validateFileConfig(fc *FileConfig, defaultLogDir *string) error // Apply the auditable flag if set. if *fc.Auditable { - bt := true - fc.SyncWrites = &bt + bf, bt := false, true + fc.BufferedWrites = &bf fc.Criticality = &bt if *fc.Format == DefaultFileFormat { s := DefaultFileFormat + "-count" diff --git a/pkg/util/log/testdata/config b/pkg/util/log/testdata/config index f837102bc6cc..79856113aac1 100644 --- a/pkg/util/log/testdata/config +++ b/pkg/util/log/testdata/config @@ -8,7 +8,7 @@ sinks: dir: TMPDIR max-file-size: 10MiB max-group-size: 100MiB - sync-writes: false + buffered-writes: true filter: INFO format: crdb-v1 redact: false @@ -40,7 +40,7 @@ sinks: dir: TMPDIR max-file-size: 10MiB max-group-size: 100MiB - sync-writes: true + buffered-writes: false filter: INFO format: crdb-v1-count redact: false @@ -50,7 +50,7 @@ sinks: dir: TMPDIR max-file-size: 10MiB max-group-size: 100MiB - sync-writes: false + buffered-writes: true filter: INFO format: crdb-v1 redact: false