From db48bd58ad5013a41a255919e7a6bd6ea419a4a8 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Fri, 6 Nov 2020 19:36:45 -0400 Subject: [PATCH 1/3] sqlsmith: add schema-related operations User-defined schemas are now supported in sqlsmith. UDSs will be randomly generated, and new tables will be randomly included in the available UDSs. Queries will select from any table, including those inside of UDSs as well. Release note: None --- pkg/internal/sqlsmith/alter.go | 14 +++++- pkg/internal/sqlsmith/schema.go | 76 +++++++++++++++++++------------ pkg/internal/sqlsmith/sqlsmith.go | 6 +++ 3 files changed, 66 insertions(+), 30 deletions(-) diff --git a/pkg/internal/sqlsmith/alter.go b/pkg/internal/sqlsmith/alter.go index 131337c0444f..e6ec48b0e19f 100644 --- a/pkg/internal/sqlsmith/alter.go +++ b/pkg/internal/sqlsmith/alter.go @@ -21,6 +21,7 @@ var ( alters = append(append(altersTableExistence, altersExistingTable...), altersTypeExistence...) altersTableExistence = []statementWeight{ {10, makeCreateTable}, + {2, makeCreateSchema}, {1, makeDropTable}, } altersExistingTable = []statementWeight{ @@ -62,9 +63,20 @@ func makeAlter(s *Smither) (tree.Statement, bool) { return nil, false } +func makeCreateSchema(s *Smither) (tree.Statement, bool) { + return &tree.CreateSchema{ + Schema: tree.ObjectNamePrefix{ + SchemaName: s.name("schema"), + ExplicitSchema: true, + }, + }, true +} + func makeCreateTable(s *Smither) (tree.Statement, bool) { table := rowenc.RandCreateTable(s.rnd, "", 0) - table.Table = tree.MakeUnqualifiedTableName(s.name("tab")) + schemaOrd := s.rnd.Intn(len(s.schemas)) + schema := s.schemas[schemaOrd] + table.Table = tree.MakeTableNameWithSchema(tree.Name(s.dbName), schema.SchemaName, s.name("tab")) return table, true } diff --git a/pkg/internal/sqlsmith/schema.go b/pkg/internal/sqlsmith/schema.go index 7d8cda0465f2..7ae12b0d55dd 100644 --- a/pkg/internal/sqlsmith/schema.go +++ b/pkg/internal/sqlsmith/schema.go @@ -55,6 +55,10 @@ func (s *Smither) ReloadSchemas() error { if err != nil { return err } + s.schemas, err = s.extractSchemas() + if err != nil { + return err + } s.indexes, err = s.extractIndexes(s.tables) s.columns = make(map[tree.TableName]map[tree.Name]*tree.ColumnTableDef) for _, ref := range s.tables { @@ -208,6 +212,31 @@ FROM }, nil } +type schemaRef struct { + SchemaName tree.Name +} + +func (s *Smither) extractSchemas() ([]*schemaRef, error) { + rows, err := s.db.Query(` +SELECT nspname FROM pg_catalog.pg_namespace +WHERE nspname NOT IN ('crdb_internal', 'pg_catalog', 'pg_extension', + 'information_schema')`) + if err != nil { + return nil, err + } + defer rows.Close() + + var ret []*schemaRef + for rows.Next() { + var schema tree.Name + if err := rows.Scan(&schema); err != nil { + return nil, err + } + ret = append(ret, &schemaRef{SchemaName: schema}) + } + return ret, nil +} + func (s *Smither) extractTables() ([]*tableRef, error) { rows, err := s.db.Query(` SELECT @@ -222,7 +251,8 @@ SELECT FROM information_schema.columns WHERE - table_schema = 'public' + table_schema NOT IN ('crdb_internal', 'pg_catalog', 'pg_extension', + 'information_schema') ORDER BY table_catalog, table_schema, table_name `) @@ -241,9 +271,6 @@ ORDER BY var tables []*tableRef var currentCols []*tree.ColumnTableDef emit := func() error { - if lastSchema != "public" { - return nil - } if len(currentCols) == 0 { return fmt.Errorf("zero columns for %s.%s", lastCatalog, lastName) } @@ -255,8 +282,9 @@ ORDER BY Type: col.Type, }) } + tableName := tree.MakeTableNameWithSchema(lastCatalog, lastSchema, lastName) tables = append(tables, &tableRef{ - TableName: tree.NewTableName(lastCatalog, lastName), + TableName: &tableName, Columns: currentCols, }) return nil @@ -324,9 +352,13 @@ func (s *Smither) extractIndexes( // sqlsmith. rows, err := s.db.Query(fmt.Sprintf(` SELECT - index_name, column_name, storing, direction = 'ASC' + si.index_name, column_name, storing, direction = 'ASC', + is_inverted FROM - [SHOW INDEXES FROM %s] + [SHOW INDEXES FROM %s] si + JOIN crdb_internal.table_indexes ti + ON si.table_name = ti.descriptor_name + AND si.index_name = ti.index_name WHERE column_name != 'rowid' `, t.TableName)) @@ -335,15 +367,16 @@ func (s *Smither) extractIndexes( } for rows.Next() { var idx, col tree.Name - var storing, ascending bool - if err := rows.Scan(&idx, &col, &storing, &ascending); err != nil { + var storing, ascending, inverted bool + if err := rows.Scan(&idx, &col, &storing, &ascending, &inverted); err != nil { rows.Close() return nil, err } if _, ok := indexes[idx]; !ok { indexes[idx] = &tree.CreateIndex{ - Name: idx, - Table: *t.TableName, + Name: idx, + Table: *t.TableName, + Inverted: inverted, } } create := indexes[idx] @@ -359,25 +392,10 @@ func (s *Smither) extractIndexes( Direction: dir, }) } - row := s.db.QueryRow(fmt.Sprintf(` - SELECT - is_inverted - FROM - crdb_internal.table_indexes - WHERE - descriptor_name = '%s' AND index_name = '%s' -`, t.TableName.Table(), idx)) - var isInverted bool - if err = row.Scan(&isInverted); err != nil { - // We got an error which likely indicates that 'is_inverted' column is - // not present in crdb_internal.table_indexes vtable (probably because - // we're running 19.2 version). We will use a heuristic to determine - // whether the index is inverted. - isInverted = strings.Contains(strings.ToLower(idx.String()), "jsonb") - } - indexes[idx].Inverted = isInverted } - rows.Close() + if err := rows.Close(); err != nil { + return nil, err + } if err := rows.Err(); err != nil { return nil, err } diff --git a/pkg/internal/sqlsmith/sqlsmith.go b/pkg/internal/sqlsmith/sqlsmith.go index 86f114c088bc..27c9d79885c6 100644 --- a/pkg/internal/sqlsmith/sqlsmith.go +++ b/pkg/internal/sqlsmith/sqlsmith.go @@ -60,6 +60,8 @@ type Smither struct { rnd *rand.Rand db *gosql.DB lock syncutil.RWMutex + dbName string + schemas []*schemaRef tables []*tableRef columns map[tree.TableName]map[tree.Name]*tree.ColumnTableDef indexes map[tree.TableName]map[tree.Name]*tree.CreateIndex @@ -128,6 +130,10 @@ func NewSmither(db *gosql.DB, rnd *rand.Rand, opts ...SmitherOption) (*Smither, s.scalarExprSampler = newWeightedScalarExprSampler(s.scalarExprWeights, rnd.Int63()) s.boolExprSampler = newWeightedScalarExprSampler(s.boolExprWeights, rnd.Int63()) s.enableBulkIO() + row := s.db.QueryRow("SELECT current_database()") + if err := row.Scan(&s.dbName); err != nil { + return nil, err + } return s, s.ReloadSchemas() } From 94a429b124c953285a69c948c7d9666252089a04 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 30 Nov 2020 13:19:13 +0100 Subject: [PATCH 2/3] Revert "Revert "util/log: more misc cleanups"" This re-instates #57000, as it did not "introduce a race in crdb" (Instead it merely outlined a bug in a test, which is to be fixed in a subsequent commit.) Release note: None --- pkg/ccl/changefeedccl/BUILD.bazel | 1 + .../changefeedccl/changefeed_processors.go | 3 +- pkg/cli/BUILD.bazel | 1 + pkg/cli/cli.go | 5 +- pkg/cli/cpuprofile.go | 3 +- pkg/cli/start.go | 3 +- pkg/jobs/BUILD.bazel | 1 + pkg/jobs/adopt.go | 5 +- pkg/jobs/deprecated.go | 2 +- pkg/server/BUILD.bazel | 1 + pkg/server/admin.go | 3 +- pkg/server/debug/logspy.go | 7 +- pkg/server/updates.go | 7 +- pkg/sql/BUILD.bazel | 1 + pkg/sql/catalog/lease/BUILD.bazel | 1 + pkg/sql/catalog/lease/lease.go | 3 +- pkg/sql/colexec/colbuilder/BUILD.bazel | 1 + pkg/sql/colexec/colbuilder/execplan.go | 3 +- pkg/sql/colflow/colrpc/BUILD.bazel | 1 + pkg/sql/colflow/colrpc/inbox.go | 3 +- pkg/sql/conn_executor.go | 7 +- pkg/sql/execinfra/BUILD.bazel | 1 + pkg/sql/execinfra/base.go | 3 +- pkg/sql/execinfra/processorsbase.go | 7 +- pkg/sql/gcjob/gcjobnotifier/BUILD.bazel | 2 +- pkg/sql/gcjob/gcjobnotifier/notifier.go | 6 +- pkg/sql/sqltelemetry/BUILD.bazel | 1 + pkg/sql/sqltelemetry/report.go | 5 +- .../lint/passes/fmtsafe/functions.go | 3 +- pkg/util/errorutil/BUILD.bazel | 2 +- pkg/util/errorutil/error.go | 6 +- pkg/util/log/BUILD.bazel | 63 +------------ pkg/util/log/clog.go | 67 +++++++------ pkg/util/log/clog_test.go | 17 ++-- pkg/util/log/exit_override.go | 6 +- pkg/util/log/file.go | 25 +---- pkg/util/log/file_sync_buffer.go | 21 +++-- pkg/util/log/flags.go | 91 +++++++++--------- pkg/util/log/log_bridge.go | 2 +- pkg/util/log/log_entry.go | 50 +++++----- pkg/util/log/logcrash/BUILD.bazel | 93 +++++++++++++++++++ .../log/{ => logcrash}/crash_reporting.go | 25 +++-- .../crash_reporting_packet_test.go | 21 +++-- .../{ => logcrash}/crash_reporting_test.go | 55 +++++------ .../crash_reporting_unix_test.go | 2 +- pkg/util/log/logcrash/main_test.go | 39 ++++++++ pkg/util/log/main_test.go | 15 --- pkg/util/log/secondary_log.go | 50 +++++----- pkg/util/log/secondary_log_test.go | 22 +++-- pkg/util/log/sinks.go | 13 +-- pkg/util/log/stderr_redirect.go | 2 +- pkg/util/log/stderr_sink.go | 20 +--- pkg/util/log/structured.go | 16 ++-- pkg/util/log/test_log_scope.go | 6 +- pkg/util/log/trace.go | 3 - pkg/util/mon/BUILD.bazel | 1 + pkg/util/mon/bytes_usage.go | 7 +- pkg/util/stop/BUILD.bazel | 1 + pkg/util/stop/stopper.go | 3 +- 59 files changed, 448 insertions(+), 386 deletions(-) create mode 100644 pkg/util/log/logcrash/BUILD.bazel rename pkg/util/log/{ => logcrash}/crash_reporting.go (94%) rename pkg/util/log/{ => logcrash}/crash_reporting_packet_test.go (93%) rename pkg/util/log/{ => logcrash}/crash_reporting_test.go (86%) rename pkg/util/log/{ => logcrash}/crash_reporting_unix_test.go (97%) create mode 100644 pkg/util/log/logcrash/main_test.go diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index aec95e4f7c73..d08c4c6f7b51 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -75,6 +75,7 @@ go_library( "//pkg/util/humanizeutil", "//pkg/util/json", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/metric", "//pkg/util/mon", "//pkg/util/protoutil", diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index 9f093c78518f..8c07705b9100 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/span" @@ -715,7 +716,7 @@ func (cf *changeFrontier) noteResolvedSpan(d rowenc.EncDatum) error { // job progress update closure, but it currently doesn't pass along the info // we'd need to do it that way. if !resolved.Timestamp.IsEmpty() && resolved.Timestamp.Less(cf.highWaterAtStart) { - log.ReportOrPanic(cf.Ctx, &cf.flowCtx.Cfg.Settings.SV, + logcrash.ReportOrPanic(cf.Ctx, &cf.flowCtx.Cfg.Settings.SV, `got a span level timestamp %s for %s that is less than the initial high-water %s`, log.Safe(resolved.Timestamp), resolved.Span, log.Safe(cf.highWaterAtStart)) return nil diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel index c59b561971ab..a9326215c212 100644 --- a/pkg/cli/BUILD.bazel +++ b/pkg/cli/BUILD.bazel @@ -144,6 +144,7 @@ go_library( "//pkg/util/iterutil", "//pkg/util/keysutil", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/log/logflags", "//pkg/util/log/logpb", "//pkg/util/log/severity", diff --git a/pkg/cli/cli.go b/pkg/cli/cli.go index efd4bf33a122..0691eb9a5fc2 100644 --- a/pkg/cli/cli.go +++ b/pkg/cli/cli.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/cli/exit" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/log/logflags" "github.com/cockroachdb/cockroach/pkg/util/randutil" // intentionally not all the workloads in pkg/ccl/workloadccl/allccl @@ -58,12 +59,12 @@ func Main() { cmdName := commandName(os.Args[1:]) - log.SetupCrashReporter( + logcrash.SetupCrashReporter( context.Background(), cmdName, ) - defer log.RecoverAndReportPanic(context.Background(), &serverCfg.Settings.SV) + defer logcrash.RecoverAndReportPanic(context.Background(), &serverCfg.Settings.SV) err := Run(os.Args[1:]) diff --git a/pkg/cli/cpuprofile.go b/pkg/cli/cpuprofile.go index ef5ed7d8aeb3..eb8cf3586c66 100644 --- a/pkg/cli/cpuprofile.go +++ b/pkg/cli/cpuprofile.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -76,7 +77,7 @@ func initCPUProfile(ctx context.Context, dir string, st *cluster.Settings) { // TODO(knz,tbg): The caller of initCPUProfile() also defines a stopper; // arguably this code would be better served by stopper.RunAsyncTask(). go func() { - defer log.RecoverAndReportPanic(ctx, &serverCfg.Settings.SV) + defer logcrash.RecoverAndReportPanic(ctx, &serverCfg.Settings.SV) ctx := context.Background() diff --git a/pkg/cli/start.go b/pkg/cli/start.go index c43c8a0cbd06..da6df24b432f 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/log/logflags" "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/cockroach/pkg/util/sdnotify" @@ -526,7 +527,7 @@ If problems persist, please see %s.` // actually been started successfully. If there's no server, // we won't know enough to decide whether reporting is // permitted. - log.RecoverAndReportPanic(ctx, &s.ClusterSettings().SV) + logcrash.RecoverAndReportPanic(ctx, &s.ClusterSettings().SV) } }() // When the start up goroutine completes, so can the start up span diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel index 240277147821..45fb832aebeb 100644 --- a/pkg/jobs/BUILD.bazel +++ b/pkg/jobs/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "//pkg/util/envutil", "//pkg/util/hlc", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/metric", "//pkg/util/protoutil", "//pkg/util/retry", diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go index bc6c4c0110d4..baa645422650 100644 --- a/pkg/jobs/adopt.go +++ b/pkg/jobs/adopt.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" ) @@ -259,7 +260,7 @@ func (r *Registry) runJob( // assertion errors, which shouldn't cause the test to panic. For now, // comment this out. // if errors.HasAssertionFailure(err) { - // log.ReportOrPanic(ctx, nil, err.Error()) + // logcrash.ReportOrPanic(ctx, nil, err.Error()) // } log.Errorf(ctx, "job %d: adoption completed with error %v", *job.ID(), err) } @@ -308,7 +309,7 @@ RETURNING id, status`, log.Infof(ctx, "job %d, session id: %s canceled: the job is now reverting", id, s.ID()) default: - log.ReportOrPanic(ctx, nil, "unexpected job status") + logcrash.ReportOrPanic(ctx, nil, "unexpected job status") } } return nil diff --git a/pkg/jobs/deprecated.go b/pkg/jobs/deprecated.go index 98d6e2ef5c9b..3faac387aa0b 100644 --- a/pkg/jobs/deprecated.go +++ b/pkg/jobs/deprecated.go @@ -375,7 +375,7 @@ func (r *Registry) deprecatedResume( // assertion errors, which shouldn't cause the test to panic. For now, // comment this out. // if errors.HasAssertionFailure(err) { - // log.ReportOrPanic(ctx, nil, err.Error()) + // logcrash.ReportOrPanic(ctx, nil, err.Error()) // } log.Errorf(ctx, "job %d: adoption completed with error %v", *job.ID(), err) } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 819dd1dcb63d..8060f12ea588 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -135,6 +135,7 @@ go_library( "//pkg/util/httputil", "//pkg/util/humanizeutil", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/log/logpb", "//pkg/util/log/severity", "//pkg/util/metric", diff --git a/pkg/server/admin.go b/pkg/server/admin.go index cafc0838736a..a1b02851be0a 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -51,6 +51,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -1322,7 +1323,7 @@ func (s *adminServer) Cluster( return &serverpb.ClusterResponse{ ClusterID: clusterID.String(), - ReportingEnabled: log.DiagnosticsReportingEnabled.Get(&s.server.st.SV), + ReportingEnabled: logcrash.DiagnosticsReportingEnabled.Get(&s.server.st.SV), EnterpriseEnabled: enterpriseEnabled, }, nil } diff --git a/pkg/server/debug/logspy.go b/pkg/server/debug/logspy.go index 0403146227d0..53c3dda34165 100644 --- a/pkg/server/debug/logspy.go +++ b/pkg/server/debug/logspy.go @@ -160,7 +160,7 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er if err == nil { if dropped := atomic.LoadInt32(&countDropped); dropped > 0 { entry := log.MakeEntry( - ctx, severity.WARNING, nil /* LogCounter */, 0 /* depth */, false, /* redactable */ + ctx, severity.WARNING, 0 /* depth */, false, /* redactable */ "%d messages were dropped", log.Safe(dropped)) err = log.FormatEntry(entry, w) // modify return value } @@ -175,7 +175,7 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er { entry := log.MakeEntry( - ctx, severity.INFO, nil /* LogCounter */, 0 /* depth */, false, /* redactable */ + ctx, severity.INFO, 0 /* depth */, false, /* redactable */ "intercepting logs with options %+v", opts) entries <- entry } @@ -212,8 +212,8 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er for { select { case <-done: - return + case entry := <-entries: if err := log.FormatEntry(entry, w); err != nil { return errors.Wrapf(err, "while writing entry %v", entry) @@ -225,6 +225,7 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er if done == nil { done = ctx.Done() } + case <-flushTimer.C: flushTimer.Read = true flushTimer.Reset(flushInterval) diff --git a/pkg/server/updates.go b/pkg/server/updates.go index bb054a054a69..ca9b7e3c5b1d 100644 --- a/pkg/server/updates.go +++ b/pkg/server/updates.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/cloudinfo" "github.com/cockroachdb/cockroach/pkg/util/httputil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -80,7 +81,7 @@ type versionInfo struct { // phones home to check for updates and report usage. func (s *Server) PeriodicallyCheckForUpdates(ctx context.Context) { s.stopper.RunWorker(ctx, func(ctx context.Context) { - defer log.RecoverAndReportNonfatalPanic(ctx, &s.st.SV) + defer logcrash.RecoverAndReportNonfatalPanic(ctx, &s.st.SV) nextUpdateCheck := s.startTime nextDiagnosticReport := s.startTime @@ -120,7 +121,7 @@ func (s *Server) maybeCheckForUpdates( // if diagnostics reporting is disabled, we should assume that means that the // user doesn't want us phoning home for new-version checks either. - if !log.DiagnosticsReportingEnabled.Get(&s.st.SV) { + if !logcrash.DiagnosticsReportingEnabled.Get(&s.st.SV) { return now.Add(updateCheckFrequency) } @@ -251,7 +252,7 @@ func (s *Server) maybeReportDiagnostics(ctx context.Context, now, scheduled time // TODO(dt): we should allow tuning the reset and report intervals separately. // Consider something like rand.Float() > resetFreq/reportFreq here to sample // stat reset periods for reporting. - if log.DiagnosticsReportingEnabled.Get(&s.st.SV) { + if logcrash.DiagnosticsReportingEnabled.Get(&s.st.SV) { s.ReportDiagnostics(ctx) } diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 6f8447e9f864..4a1d27e1b0b2 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -341,6 +341,7 @@ go_library( "//pkg/util/humanizeutil", "//pkg/util/json", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/log/severity", "//pkg/util/metric", "//pkg/util/mon", diff --git a/pkg/sql/catalog/lease/BUILD.bazel b/pkg/sql/catalog/lease/BUILD.bazel index 292bf4f47796..7f93ed08e588 100644 --- a/pkg/sql/catalog/lease/BUILD.bazel +++ b/pkg/sql/catalog/lease/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/sql/sqlutil", "//pkg/util/hlc", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/quotapool", "//pkg/util/retry", "//pkg/util/stop", diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 2a94c9d772f4..7caf31da086b 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -1846,7 +1847,7 @@ func (m *Manager) watchForRangefeedUpdates( } var descriptor descpb.Descriptor if err := ev.Value.GetProto(&descriptor); err != nil { - log.ReportOrPanic(ctx, &m.storage.settings.SV, + logcrash.ReportOrPanic(ctx, &m.storage.settings.SV, "%s: unable to unmarshal descriptor %v", ev.Key, ev.Value) return } diff --git a/pkg/sql/colexec/colbuilder/BUILD.bazel b/pkg/sql/colexec/colbuilder/BUILD.bazel index bdf85025c4ff..b6c6114dba78 100644 --- a/pkg/sql/colexec/colbuilder/BUILD.bazel +++ b/pkg/sql/colexec/colbuilder/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/sql/types", "//pkg/util", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/mon", "//vendor/github.com/cockroachdb/errors", ], diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 5e26d9229809..ffccc92413fd 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/errors" ) @@ -628,7 +629,7 @@ func NewColOperator( result.OpMonitors = result.OpMonitors[:0] } if panicErr != nil { - colexecerror.InternalError(log.PanicAsError(0, panicErr)) + colexecerror.InternalError(logcrash.PanicAsError(0, panicErr)) } }() spec := args.Spec diff --git a/pkg/sql/colflow/colrpc/BUILD.bazel b/pkg/sql/colflow/colrpc/BUILD.bazel index 5c3d9a5df011..714d42618f9c 100644 --- a/pkg/sql/colflow/colrpc/BUILD.bazel +++ b/pkg/sql/colflow/colrpc/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/sql/execinfrapb", "//pkg/sql/types", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/timeutil", "//vendor/github.com/apache/arrow/go/arrow/array", "//vendor/github.com/cockroachdb/errors", diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index 5a3ba333f77b..a59816bfa93a 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/logtags" ) @@ -260,7 +261,7 @@ func (i *Inbox) Next(ctx context.Context) coldata.Batch { // during normal termination. if err := recover(); err != nil { i.close() - colexecerror.InternalError(log.PanicAsError(0, err)) + colexecerror.InternalError(logcrash.PanicAsError(0, err)) } }() diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index be0bfc239923..911e63ca2bbf 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/fsm" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -771,7 +772,7 @@ const ( func (ex *connExecutor) closeWrapper(ctx context.Context, recovered interface{}) { if recovered != nil { - panicErr := log.PanicAsError(1, recovered) + panicErr := logcrash.PanicAsError(1, recovered) // If there's a statement currently being executed, we'll report // on it. @@ -788,7 +789,7 @@ func (ex *connExecutor) closeWrapper(ctx context.Context, recovered interface{}) } // Report the panic to telemetry in any case. - log.ReportPanic(ctx, &ex.server.cfg.Settings.SV, panicErr, 1 /* depth */) + logcrash.ReportPanic(ctx, &ex.server.cfg.Settings.SV, panicErr, 1 /* depth */) // Close the executor before propagating the panic further. ex.close(ctx, panicClose) @@ -2695,7 +2696,7 @@ func statementFromCtx(ctx context.Context) tree.Statement { func init() { // Register a function to include the anonymized statement in crash reports. - log.RegisterTagFn("statement", func(ctx context.Context) string { + logcrash.RegisterTagFn("statement", func(ctx context.Context) string { stmt := statementFromCtx(ctx) if stmt == nil { return "" diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index c467b2b94418..e821c9177c70 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -50,6 +50,7 @@ go_library( "//pkg/storage/fs", "//pkg/util", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/metric", "//pkg/util/mon", "//pkg/util/optional", diff --git a/pkg/sql/execinfra/base.go b/pkg/sql/execinfra/base.go index 625a954abeb0..82fb69d51893 100644 --- a/pkg/sql/execinfra/base.go +++ b/pkg/sql/execinfra/base.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -394,7 +395,7 @@ func (rb *rowSourceBase) consumerDone() { func (rb *rowSourceBase) consumerClosed(name string) { status := ConsumerStatus(atomic.LoadUint32((*uint32)(&rb.ConsumerStatus))) if status == ConsumerClosed { - log.ReportOrPanic(context.Background(), nil, "%s already closed", log.Safe(name)) + logcrash.ReportOrPanic(context.Background(), nil, "%s already closed", log.Safe(name)) } atomic.StoreUint32((*uint32)(&rb.ConsumerStatus), uint32(ConsumerClosed)) } diff --git a/pkg/sql/execinfra/processorsbase.go b/pkg/sql/execinfra/processorsbase.go index 28d349c4801e..310bb6dca70f 100644 --- a/pkg/sql/execinfra/processorsbase.go +++ b/pkg/sql/execinfra/processorsbase.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/optional" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -612,7 +613,7 @@ func (pb *ProcessorBase) MoveToDraining(err error) { // However, calling it with an error in states other than StateRunning is // not permitted. if err != nil { - log.ReportOrPanic( + logcrash.ReportOrPanic( pb.Ctx, &pb.FlowCtx.Cfg.Settings.SV, "MoveToDraining called in state %s with err: %+v", @@ -642,7 +643,7 @@ func (pb *ProcessorBase) MoveToDraining(err error) { // also moves from StateDraining to StateTrailingMeta when appropriate. func (pb *ProcessorBase) DrainHelper() *execinfrapb.ProducerMetadata { if pb.State == StateRunning { - log.ReportOrPanic( + logcrash.ReportOrPanic( pb.Ctx, &pb.FlowCtx.Cfg.Settings.SV, "drain helper called in StateRunning", @@ -716,7 +717,7 @@ func (pb *ProcessorBase) popTrailingMeta() *execinfrapb.ProducerMetadata { // draining its inputs (if it wants to drain them). func (pb *ProcessorBase) moveToTrailingMeta() { if pb.State == StateTrailingMeta || pb.State == StateExhausted { - log.ReportOrPanic( + logcrash.ReportOrPanic( pb.Ctx, &pb.FlowCtx.Cfg.Settings.SV, "moveToTrailingMeta called in state: %s", diff --git a/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel b/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel index fc3aab445c1b..71b0bf619994 100644 --- a/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel +++ b/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel @@ -11,7 +11,7 @@ go_library( "//pkg/keys", "//pkg/roachpb", "//pkg/settings/cluster", - "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/stop", "//pkg/util/syncutil", ], diff --git a/pkg/sql/gcjob/gcjobnotifier/notifier.go b/pkg/sql/gcjob/gcjobnotifier/notifier.go index 01b20bbefeeb..b62ef235abf4 100644 --- a/pkg/sql/gcjob/gcjobnotifier/notifier.go +++ b/pkg/sql/gcjob/gcjobnotifier/notifier.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -77,7 +77,7 @@ func (n *Notifier) AddNotifyee(ctx context.Context) (onChange <-chan struct{}, c n.mu.Lock() defer n.mu.Unlock() if !n.mu.started { - log.ReportOrPanic(ctx, &n.settings.SV, + logcrash.ReportOrPanic(ctx, &n.settings.SV, "adding a notifyee to a Notifier before starting") } if n.mu.stopped { @@ -121,7 +121,7 @@ func (n *Notifier) markStarted() (alreadyStarted bool) { // Start must not be called more than once. func (n *Notifier) Start(ctx context.Context) { if alreadyStarted := n.markStarted(); alreadyStarted { - log.ReportOrPanic(ctx, &n.settings.SV, "started Notifier more than once") + logcrash.ReportOrPanic(ctx, &n.settings.SV, "started Notifier more than once") return } if err := n.stopper.RunAsyncTask(ctx, "gcjob.Notifier", n.run); err != nil { diff --git a/pkg/sql/sqltelemetry/BUILD.bazel b/pkg/sql/sqltelemetry/BUILD.bazel index a300bc27f01c..96edcffb4542 100644 --- a/pkg/sql/sqltelemetry/BUILD.bazel +++ b/pkg/sql/sqltelemetry/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/util/log", + "//pkg/util/log/logcrash", "//vendor/github.com/cockroachdb/errors", ], ) diff --git a/pkg/sql/sqltelemetry/report.go b/pkg/sql/sqltelemetry/report.go index 38421a914cc3..75dcd8654dc9 100644 --- a/pkg/sql/sqltelemetry/report.go +++ b/pkg/sql/sqltelemetry/report.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/errors" ) @@ -55,9 +56,9 @@ func RecordError(ctx context.Context, err error, sv *settings.Values) { // report is sent to sentry below. log.Errorf(ctx, "encountered internal error:\n%+v", err) - if log.ShouldSendReport(sv) { + if logcrash.ShouldSendReport(sv) { event, extraDetails := errors.BuildSentryReport(err) - log.SendReport(ctx, log.ReportTypeError, event, extraDetails) + logcrash.SendReport(ctx, logcrash.ReportTypeError, event, extraDetails) } } } diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index b54225ee5c46..b087ff183152 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -74,12 +74,13 @@ var requireConstFmt = map[string]bool{ "github.com/cockroachdb/cockroach/pkg/util/log.FatalfDepth": true, "github.com/cockroachdb/cockroach/pkg/util/log.VEventfDepth": true, "github.com/cockroachdb/cockroach/pkg/util/log.VErrEventfDepth": true, - "github.com/cockroachdb/cockroach/pkg/util/log.ReportOrPanic": true, "github.com/cockroachdb/cockroach/pkg/util/log.MakeEntry": true, "github.com/cockroachdb/cockroach/pkg/util/log.FormatWithContextTags": true, "github.com/cockroachdb/cockroach/pkg/util/log.renderArgsAsRedactable": true, "github.com/cockroachdb/cockroach/pkg/util/log.formatArgs": true, + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash.ReportOrPanic": true, + "(*github.com/cockroachdb/cockroach/pkg/util/log.loggerT).makeStartLine": true, "(*github.com/cockroachdb/cockroach/pkg/util/log.SecondaryLogger).output": true, "(*github.com/cockroachdb/cockroach/pkg/util/log.SecondaryLogger).Logf": true, diff --git a/pkg/util/errorutil/BUILD.bazel b/pkg/util/errorutil/BUILD.bazel index 793fddf3d46f..5047374bb4c2 100644 --- a/pkg/util/errorutil/BUILD.bazel +++ b/pkg/util/errorutil/BUILD.bazel @@ -13,7 +13,7 @@ go_library( deps = [ "//pkg/settings", "//pkg/util/errorutil/unimplemented", - "//pkg/util/log", + "//pkg/util/log/logcrash", "//vendor/github.com/cockroachdb/errors", ], ) diff --git a/pkg/util/errorutil/error.go b/pkg/util/errorutil/error.go index 5472c49cf87d..48c8618bcb40 100644 --- a/pkg/util/errorutil/error.go +++ b/pkg/util/errorutil/error.go @@ -15,7 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/errors" ) @@ -42,9 +42,9 @@ func UnexpectedWithIssueErrorf(issue int, format string, args ...interface{}) er // The format string will be reproduced ad litteram in the report; the arguments // will be sanitized. func SendReport(ctx context.Context, sv *settings.Values, err error) { - if !log.ShouldSendReport(sv) { + if !logcrash.ShouldSendReport(sv) { return } event, extraDetails := errors.BuildSentryReport(err) - log.SendReport(ctx, log.ReportTypeError, event, extraDetails) + logcrash.SendReport(ctx, logcrash.ReportTypeError, event, extraDetails) } diff --git a/pkg/util/log/BUILD.bazel b/pkg/util/log/BUILD.bazel index 7e96aba3c56e..af77af009683 100644 --- a/pkg/util/log/BUILD.bazel +++ b/pkg/util/log/BUILD.bazel @@ -5,7 +5,6 @@ go_library( srcs = [ "ambient_context.go", "clog.go", - "crash_reporting.go", "doc.go", "every_n.go", "exit_override.go", @@ -44,7 +43,6 @@ go_library( "//pkg/build", "//pkg/cli/cliflags", "//pkg/cli/exit", - "//pkg/settings", "//pkg/util", "//pkg/util/caller", "//pkg/util/encoding/encodingtype", @@ -62,7 +60,6 @@ go_library( "//vendor/github.com/cockroachdb/errors/oserror", "//vendor/github.com/cockroachdb/logtags", "//vendor/github.com/cockroachdb/redact", - "//vendor/github.com/cockroachdb/sentry-go", "//vendor/github.com/cockroachdb/ttycolor", "//vendor/github.com/opentracing/opentracing-go/log", "//vendor/github.com/petermattis/goid", @@ -122,9 +119,6 @@ go_test( srcs = [ "ambient_context_test.go", "clog_test.go", - "crash_reporting_packet_test.go", - "crash_reporting_test.go", - "crash_reporting_unix_test.go", "file_log_gc_test.go", "file_test.go", "main_test.go", @@ -132,17 +126,10 @@ go_test( "secondary_log_test.go", "trace_test.go", ], + data = glob(["testdata/**"]), embed = [":log"], deps = [ - "//pkg/base", "//pkg/cli/exit", - "//pkg/security", - "//pkg/security/securitytest", - "//pkg/server", - "//pkg/settings/cluster", - "//pkg/testutils", - "//pkg/testutils/serverutils", - "//pkg/util", "//pkg/util/fileutil", "//pkg/util/leaktest", "//pkg/util/log/logpb", @@ -153,55 +140,9 @@ go_test( "//vendor/github.com/cockroachdb/errors", "//vendor/github.com/cockroachdb/logtags", "//vendor/github.com/cockroachdb/redact", - "//vendor/github.com/cockroachdb/sentry-go", "//vendor/github.com/kr/pretty", - "//vendor/github.com/pmezard/go-difflib/difflib", "//vendor/github.com/stretchr/testify/assert", "//vendor/github.com/stretchr/testify/require", "//vendor/golang.org/x/net/trace", - ] + select({ - "@io_bazel_rules_go//go/platform:aix": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:android": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:darwin": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:dragonfly": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:freebsd": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:illumos": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:ios": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:js": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:linux": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:nacl": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:netbsd": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:openbsd": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:plan9": [ - "//vendor/golang.org/x/sys/unix", - ], - "@io_bazel_rules_go//go/platform:solaris": [ - "//vendor/golang.org/x/sys/unix", - ], - "//conditions:default": [], - }), + ], ) diff --git a/pkg/util/log/clog.go b/pkg/util/log/clog.go index 07e81031f96b..8b9243eb540e 100644 --- a/pkg/util/log/clog.go +++ b/pkg/util/log/clog.go @@ -50,7 +50,8 @@ type loggingT struct { vmoduleConfig vmoduleConfig // The common stderr sink. - stderrSink stderrSink + stderrSink stderrSink + stderrSinkInfo sinkInfo // mu protects the remaining elements of this structure and is // used to synchronize logging. @@ -92,10 +93,22 @@ type sinkInfo struct { // sink is where the log entries should be written. sink logSink + // Level at or beyond which entries are output to this sink. + threshold Severity + // editor is the optional step that occurs prior to emitting the log // entry. editor redactEditor + // formatter for entries written via this sink. + formatter logFormatter + + // msgCount supports the generation of a per-entry log entry + // counter. This is needed in audit logs to hinder malicious + // repudiation of log events by manually erasing log files or log + // entries. + msgCount uint64 + // criticality indicates whether a failure to output some log // entries should incur the process to terminate. criticality bool @@ -104,39 +117,30 @@ type sinkInfo struct { // loggerT represents the logging source for a given log channel. type loggerT struct { // sinkInfos stores the destinations for log entries. - sinkInfos []sinkInfo - - // logCounter supports the generation of a per-entry log entry - // counter. This is needed in audit logs to hinder malicious - // repudiation of log events by manually erasing log files or log - // entries. - logCounter EntryCounter + sinkInfos []*sinkInfo // outputMu is used to coordinate output to the sinks, to guarantee // that the ordering of events the the same on all sinks. outputMu syncutil.Mutex } -// getFileSink retrieves the file sink if defined. -func (l *loggerT) getFileSink() *fileSink { - for _, s := range l.sinkInfos { - if fs, ok := s.sink.(*fileSink); ok { - return fs +// getFileSinkIndex retrieves the index of the fileSink, if defined, +// in the sinkInfos. Returns -1 if there is no file sink. +func (l *loggerT) getFileSinkIndex() int { + for i, s := range l.sinkInfos { + if _, ok := s.sink.(*fileSink); ok { + return i } } - return nil + return -1 } -// EntryCounter supports the generation of a per-entry log entry -// counter. This is needed in audit logs to hinder malicious -// repudiation of log events by manually erasing log files or log -// entries. -type EntryCounter struct { - // EnableMsgCount, if true, enables the production of entry - // counters. - EnableMsgCount bool - // msgCount is the current value of the counter. - msgCount uint64 +// getFileSink retrieves the file sink if defined. +func (l *loggerT) getFileSink() *fileSink { + if i := l.getFileSinkIndex(); i != -1 { + return l.sinkInfos[i].sink.(*fileSink) + } + return nil } // FatalChan is closed when Fatal is called. This can be used to make @@ -269,11 +273,18 @@ func (l *loggerT) outputLogEntry(entry logpb.Entry) { // not eliminate the event. someSinkActive := false for i, s := range l.sinkInfos { - if s.sink.activeAtSeverity(entry.Severity) { - editedEntry := maybeRedactEntry(entry, s.editor) - bufs.b[i] = s.sink.getFormatter().formatEntry(editedEntry, stacks) - someSinkActive = true + if entry.Severity < s.threshold || !s.sink.active() { + continue } + editedEntry := maybeRedactEntry(entry, s.editor) + + // Add a counter. This is important for e.g. the SQL audit logs. + // Note: whether the counter is displayed or not depends on + // the formatter. + editedEntry.Counter = atomic.AddUint64(&s.msgCount, 1) + + bufs.b[i] = s.formatter.formatEntry(editedEntry, stacks) + someSinkActive = true } // If any of the sinks is active, it is now time to send it out. diff --git a/pkg/util/log/clog_test.go b/pkg/util/log/clog_test.go index c14ce7dc3363..08cf7f8e023e 100644 --- a/pkg/util/log/clog_test.go +++ b/pkg/util/log/clog_test.go @@ -104,7 +104,7 @@ func setFlags() { ResetExitFunc() // Make all logged errors go to the external stderr, in addition to // the log file. - logging.stderrSink.threshold = severity.ERROR + logging.stderrSinkInfo.threshold = severity.ERROR } // Test that Info works as advertised. @@ -622,7 +622,7 @@ func TestFatalStacktraceStderr(t *testing.T) { defer s.Close(t) setFlags() - logging.stderrSink.threshold = severity.NONE + logging.stderrSinkInfo.threshold = severity.NONE SetExitFunc(false /* hideStack */, func(exit.Code) {}) defer setFlags() @@ -660,7 +660,7 @@ func TestRedirectStderr(t *testing.T) { defer s.Close(t) setFlags() - logging.stderrSink.threshold = severity.NONE + logging.stderrSinkInfo.threshold = severity.NONE Infof(context.Background(), "test") @@ -688,15 +688,16 @@ func TestFileSeverityFilter(t *testing.T) { defer s.Close(t) setFlags() - debugFileSink := debugLog.getFileSink() - defer func(save Severity) { debugFileSink.threshold = save }(debugFileSink.threshold) - debugFileSink.threshold = severity.ERROR + debugFileSinkInfo := debugLog.sinkInfos[debugLogFileSinkIndex] + defer func(save Severity) { debugFileSinkInfo.threshold = save }(debugFileSinkInfo.threshold) + debugFileSinkInfo.threshold = severity.ERROR Infof(context.Background(), "test1") Errorf(context.Background(), "test2") Flush() + debugFileSink := debugFileSinkInfo.sink.(*fileSink) contents, err := ioutil.ReadFile(debugFileSink.mu.file.(*syncBuffer).file.Name()) if err != nil { t.Fatal(err) @@ -726,7 +727,7 @@ func TestExitOnFullDisk(t *testing.T) { }) fs := &fileSink{} - l := &loggerT{sinkInfos: []sinkInfo{{ + l := &loggerT{sinkInfos: []*sinkInfo{{ sink: fs, editor: func(r redactablePackage) redactablePackage { return r }, criticality: true, @@ -799,7 +800,7 @@ func TestLogEntryPropagation(t *testing.T) { const specialMessage = `CAPTAIN KIRK` // Enable output to stderr (the Scope disabled it). - logging.stderrSink.threshold.SetValue(severity.INFO) + logging.stderrSinkInfo.threshold.SetValue(severity.INFO) // Make stderr non-critical. // We assume that the stderr sink is the first one. diff --git a/pkg/util/log/exit_override.go b/pkg/util/log/exit_override.go index 730e7315fe48..df495ecceeb7 100644 --- a/pkg/util/log/exit_override.go +++ b/pkg/util/log/exit_override.go @@ -77,7 +77,7 @@ func (l *loggerT) exitLocked(err error, code exit.Code) { func (l *loggerT) reportErrorEverywhereLocked(ctx context.Context, err error) { // Make a valid log entry for this error. entry := MakeEntry( - ctx, severity.ERROR, &l.logCounter, 2 /* depth */, true, /* redactable */ + ctx, severity.ERROR, 2 /* depth */, true, /* redactable */ "logging error: %v", err) // Either stderr or our log file is broken. Try writing the error to both @@ -93,8 +93,8 @@ func (l *loggerT) reportErrorEverywhereLocked(ctx context.Context, err error) { for _, s := range l.sinkInfos { sink := s.sink - if sink.activeAtSeverity(logpb.Severity_ERROR) { - buf := sink.getFormatter().formatEntry(entry, nil /*stack*/) + if logpb.Severity_ERROR >= s.threshold && sink.active() { + buf := s.formatter.formatEntry(entry, nil /*stack*/) sink.emergencyOutput(buf.Bytes()) putBuffer(buf) } diff --git a/pkg/util/log/file.go b/pkg/util/log/file.go index 247e422be10e..b1886ab62be9 100644 --- a/pkg/util/log/file.go +++ b/pkg/util/log/file.go @@ -27,7 +27,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/cli/exit" - "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -82,20 +81,12 @@ type fileSink struct { // temporarily be up to logFileMaxSize larger. logFilesCombinedMaxSize int64 - // Level beyond which entries submitted to this sink are written - // to the output file. This acts as a filter between the log entry - // producers and the file sink. - threshold Severity - - // formatter for entries. - formatter logFormatter - // notify GC daemon that a new log file was created. gcNotify chan struct{} // getStartLines retrieves a list of log entries to // include at the start of a log file. - getStartLines func(time.Time) []logpb.Entry + getStartLines func(time.Time) []*buffer // mu protects the remaining elements of this structure and is // used to synchronize output to this file sink.. @@ -139,9 +130,8 @@ type fileSink struct { func newFileSink( dir, fileNamePrefix string, forceSyncWrites bool, - fileThreshold Severity, fileMaxSize, combinedMaxSize int64, - getStartLines func(time.Time) []logpb.Entry, + getStartLines func(time.Time) []*buffer, ) *fileSink { prefix := program if fileNamePrefix != "" { @@ -149,8 +139,6 @@ func newFileSink( } f := &fileSink{ prefix: prefix, - threshold: fileThreshold, - formatter: formatCrdbV1WithCounter{}, syncWrites: forceSyncWrites, logFileMaxSize: fileMaxSize, logFilesCombinedMaxSize: combinedMaxSize, @@ -163,8 +151,8 @@ func newFileSink( } // activeAtSeverity implements the logSink interface. -func (l *fileSink) activeAtSeverity(sev logpb.Severity) bool { - return l.enabled.Get() && sev >= l.threshold +func (l *fileSink) active() bool { + return l.enabled.Get() } // attachHints implements the logSink interface. @@ -180,11 +168,6 @@ func (l *fileSink) attachHints(stacks []byte) []byte { return stacks } -// getFormatter implements the logSink interface. -func (l *fileSink) getFormatter() logFormatter { - return l.formatter -} - // output implements the logSink interface. func (l *fileSink) output(extraSync bool, b []byte) error { if !l.enabled.Get() { diff --git a/pkg/util/log/file_sync_buffer.go b/pkg/util/log/file_sync_buffer.go index 27e5e1d4ef12..5c9d3fac3b36 100644 --- a/pkg/util/log/file_sync_buffer.go +++ b/pkg/util/log/file_sync_buffer.go @@ -235,18 +235,23 @@ func (l *fileSink) initializeNewOutputFile( newWriter = bufio.NewWriterSize(file, bufferSize) if l.getStartLines != nil { - messages := l.getStartLines(now) - for _, entry := range messages { - buf := l.formatter.formatEntry(entry, nil) + bufs := l.getStartLines(now) + for _, buf := range bufs { var n int - n, err = file.Write(buf.Bytes()) - putBuffer(buf) + var thisErr error + n, thisErr = file.Write(buf.Bytes()) nbytes += int64(n) - if err != nil { - return nil, 0, err - } + // Note: we combine the errors, instead of stopping at the first + // error encountered, to ensure that all the buffers get + // released back to the pool. + err = errors.CombineErrors(err, thisErr) + putBuffer(buf) } } + if err != nil { + return nil, nbytes, err + } + return newWriter, nbytes, nil } diff --git a/pkg/util/log/flags.go b/pkg/util/log/flags.go index 768c9ac1392d..b6924f2769bf 100644 --- a/pkg/util/log/flags.go +++ b/pkg/util/log/flags.go @@ -62,6 +62,10 @@ type config struct { var debugLog *loggerT +// debugLogFileSinkIndex is the index of the fileSink inside +// debugLog.sinkInfos. +var debugLogFileSinkIndex int + // stderrLog is the logger where writes performed directly // to the stderr file descriptor (such as that performed // by the go runtime) *may* be redirected. @@ -74,8 +78,21 @@ func init() { // both to the output file and to the process' external stderr // (OrigStderr). logging.fileThreshold = severity.INFO - logging.stderrSink.threshold = severity.INFO - logging.stderrSink.formatter = formatCrdbV1TTYWithCounter{} + // TODO(knz): make the stderr settings below configurable. + logging.stderrSinkInfo = sinkInfo{ + sink: &logging.stderrSink, + threshold: severity.INFO, + // stderr editor. + // We don't redact upfront, and we keep the redaction markers. + editor: getEditor(SelectEditMode(false /* redact */, true /* keepRedactable */)), + formatter: formatCrdbV1TTY{}, + // failure to write to stderr is critical for now. We may want + // to make this non-critical in the future, since it's common + // for folk to close the terminal where they launched 'cockroach + // start --background'. + // We keep this true for now for backward-compatibility. + criticality: true, + } // Default maximum size of individual log files. logging.logFileMaxSize = 10 << 20 // 10MiB @@ -87,31 +104,18 @@ func init() { "", /* dir */ "", /* fileNamePrefix */ false, /* forceSyncWrites */ - logging.fileThreshold, logging.logFileMaxSize, logging.logFilesCombinedMaxSize, debugLog.getStartLines, ) // TODO(knz): Make all this configurable. // (As done in https://github.com/cockroachdb/cockroach/pull/51987.) - debugLog.sinkInfos = []sinkInfo{ - { - // Nb: some tests in this package assume that the stderr sink is - // the first one. If this changes, the tests need to be updated - // as well. - sink: &logging.stderrSink, - // stderr editor. - // We don't redact upfront, and we keep the redaction markers. - editor: getEditor(SelectEditMode(false /* redact */, true /* keepRedactable */)), - // failure to write to stderr is critical for now. We may want - // to make this non-critical in the future, since it's common - // for folk to close the terminal where they launched 'cockroach - // start --background'. - // We keep this true for now for backward-compatibility. - criticality: true, - }, + debugLog.sinkInfos = []*sinkInfo{ + &logging.stderrSinkInfo, { - sink: debugFileSink, + sink: debugFileSink, + threshold: logging.fileThreshold, + formatter: formatCrdbV1{}, // file editor. // We don't redact upfront, and the "--redactable-logs" flag decides // whether to keep the redaction markers in the output. @@ -120,6 +124,7 @@ func init() { criticality: true, }, } + debugLogFileSinkIndex = 1 allLoggers.put(debugLog) allFileSinks.put(debugFileSink) @@ -136,7 +141,7 @@ func init() { ) // We define these flags here because they have the type Severity // which we can't pass to logflags without creating an import cycle. - flag.Var(&logging.stderrSink.threshold, logflags.LogToStderrName, + flag.Var(&logging.stderrSinkInfo.threshold, logflags.LogToStderrName, "logs at or above this threshold go to stderr") flag.Var(&logging.fileThreshold, logflags.LogFileVerbosityThresholdName, "minimum verbosity of messages written to the log file") @@ -148,30 +153,24 @@ func init() { // during SetupRedactionAndStderrRedirects() after the custom // logging configuration has been selected. func initDebugLogFromDefaultConfig() { - for i := range debugLog.sinkInfos { - fileSink, ok := debugLog.sinkInfos[i].sink.(*fileSink) - if !ok { - continue - } - // Re-configure the redaction editor. This may have changed - // after SetupRedactionAndStderrRedirects() reconfigures - // logging.redactableLogs. - // - // TODO(knz): Remove this initialization when - // https://github.com/cockroachdb/cockroach/pull/51987 introduces - // proper configurability. - debugLog.sinkInfos[i].editor = getEditor(SelectEditMode(false /* redact */, logging.redactableLogs /* keepRedactable */)) - func() { - fileSink.mu.Lock() - defer fileSink.mu.Unlock() - fileSink.prefix = program - fileSink.mu.logDir = logging.logDir.String() - fileSink.enabled.Set(fileSink.mu.logDir != "") - fileSink.logFileMaxSize = logging.logFileMaxSize - fileSink.logFilesCombinedMaxSize = logging.logFilesCombinedMaxSize - fileSink.threshold = logging.fileThreshold - }() - } + debugLogFileSinkInfo := debugLog.sinkInfos[debugLogFileSinkIndex] + // Re-configure the redaction editor. This may have changed + // after SetupRedactionAndStderrRedirects() reconfigures + // logging.redactableLogs. + // + // TODO(knz): Remove this initialization when + // https://github.com/cockroachdb/cockroach/pull/51987 introduces + // proper configurability. + debugLogFileSinkInfo.editor = getEditor(SelectEditMode(false /* redact */, logging.redactableLogs /* keepRedactable */)) + debugLogFileSinkInfo.threshold = logging.fileThreshold + fileSink := debugLogFileSinkInfo.sink.(*fileSink) + fileSink.mu.Lock() + defer fileSink.mu.Unlock() + fileSink.prefix = program + fileSink.mu.logDir = logging.logDir.String() + fileSink.enabled.Set(fileSink.mu.logDir != "") + fileSink.logFileMaxSize = logging.logFileMaxSize + fileSink.logFilesCombinedMaxSize = logging.logFilesCombinedMaxSize } // IsActive returns true iff the main logger already has some events @@ -297,7 +296,7 @@ func SetupRedactionAndStderrRedirects() (cleanupForTestingOnly func(), err error // If redaction is requested and we have a chance to produce some // log entries on stderr, that's a configuration we cannot support // safely. Reject it. - if logging.redactableLogsRequested && logging.stderrSink.threshold != severity.NONE { + if logging.redactableLogsRequested && logging.stderrSinkInfo.threshold != severity.NONE { return nil, errors.WithHintf( errors.New("cannot enable redactable logging without a logging directory"), "You can pass --%s to set up a logging directory explicitly.", cliflags.LogDir.Name) diff --git a/pkg/util/log/log_bridge.go b/pkg/util/log/log_bridge.go index 051028d930e8..ba80cc5bd12d 100644 --- a/pkg/util/log/log_bridge.go +++ b/pkg/util/log/log_bridge.go @@ -75,7 +75,7 @@ func (lb logBridge) Write(b []byte) (n int, err error) { } entry := MakeEntry(context.Background(), - Severity(lb), &debugLog.logCounter, 0, /* depth */ + Severity(lb), 0, /* depth */ // Note: because the caller is using the stdLog interface, they are // bypassing all the log marker logic. This means that the entire // log message should be assumed to contain confidential diff --git a/pkg/util/log/log_entry.go b/pkg/util/log/log_entry.go index a5842599dde6..14fe7dcd1393 100644 --- a/pkg/util/log/log_entry.go +++ b/pkg/util/log/log_entry.go @@ -15,7 +15,6 @@ import ( "fmt" "os" "strings" - "sync/atomic" "time" "github.com/cockroachdb/cockroach/pkg/build" @@ -29,54 +28,55 @@ import ( const severityChar = "IWEF" -// makeStartLine creates a log entry suitable for the start of a logging -// output using the canonical logging format. -func (l *loggerT) makeStartLine(format string, args ...interface{}) logpb.Entry { +// makeStartLine creates a formatted log entry suitable for the start +// of a logging output using the canonical logging format. +func (l *loggerT) makeStartLine( + formatter logFormatter, format string, args ...interface{}, +) *buffer { entry := MakeEntry( context.Background(), severity.INFO, - nil, /* logCounter */ 2, /* depth */ true, /* redactable */ format, args...) entry.Tags = "config" - return entry + var f formatCrdbV1 + return f.formatEntry(entry, nil) } // getStartLines retrieves the log entries for the start -// of a new logging output. -func (l *loggerT) getStartLines(now time.Time) []logpb.Entry { - messages := make([]logpb.Entry, 0, 6) +// of a new log file output. +func (l *loggerT) getStartLines(now time.Time) []*buffer { + idx := l.getFileSinkIndex() + if idx == -1 { + return nil + } + f := l.sinkInfos[idx].formatter + messages := make([]*buffer, 0, 6) messages = append(messages, - l.makeStartLine("file created at: %s", Safe(now.Format("2006/01/02 15:04:05"))), - l.makeStartLine("running on machine: %s", host), - l.makeStartLine("binary: %s", Safe(build.GetInfo().Short())), - l.makeStartLine("arguments: %s", os.Args), + l.makeStartLine(f, "file created at: %s", Safe(now.Format("2006/01/02 15:04:05"))), + l.makeStartLine(f, "running on machine: %s", host), + l.makeStartLine(f, "binary: %s", Safe(build.GetInfo().Short())), + l.makeStartLine(f, "arguments: %s", os.Args), ) logging.mu.Lock() if logging.mu.clusterID != "" { - messages = append(messages, l.makeStartLine("clusterID: %s", logging.mu.clusterID)) + messages = append(messages, l.makeStartLine(f, "clusterID: %s", logging.mu.clusterID)) } logging.mu.Unlock() // Including a non-ascii character in the first 1024 bytes of the log helps // viewers that attempt to guess the character encoding. messages = append(messages, - l.makeStartLine("line format: [IWEF]yymmdd hh:mm:ss.uuuuuu goid file:line msg utf8=\u2713")) + l.makeStartLine(f, "line format: [IWEF]yymmdd hh:mm:ss.uuuuuu goid file:line msg utf8=\u2713")) return messages } // MakeEntry creates an logpb.Entry. func MakeEntry( - ctx context.Context, - s Severity, - lc *EntryCounter, - depth int, - redactable bool, - format string, - args ...interface{}, + ctx context.Context, s Severity, depth int, redactable bool, format string, args ...interface{}, ) (res logpb.Entry) { res = logpb.Entry{ Severity: s, @@ -90,12 +90,6 @@ func MakeEntry( res.File = file res.Line = int64(line) - // Optionally populate the counter. - if lc != nil && lc.EnableMsgCount { - // Add a counter. This is important for e.g. the SQL audit logs. - res.Counter = atomic.AddUint64(&lc.msgCount, 1) - } - // Populate the tags. var buf strings.Builder if redactable { diff --git a/pkg/util/log/logcrash/BUILD.bazel b/pkg/util/log/logcrash/BUILD.bazel new file mode 100644 index 000000000000..f810720f9f30 --- /dev/null +++ b/pkg/util/log/logcrash/BUILD.bazel @@ -0,0 +1,93 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "logcrash", + srcs = ["crash_reporting.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/util/log/logcrash", + visibility = ["//visibility:public"], + deps = [ + "//pkg/build", + "//pkg/settings", + "//pkg/util/envutil", + "//pkg/util/log", + "//pkg/util/log/severity", + "//pkg/util/timeutil", + "//vendor/github.com/cockroachdb/errors", + "//vendor/github.com/cockroachdb/sentry-go", + ], +) + +go_test( + name = "logcrash_test", + srcs = [ + "crash_reporting_packet_test.go", + "crash_reporting_test.go", + "crash_reporting_unix_test.go", + "main_test.go", + ], + embed = [":logcrash"], + deps = [ + "//pkg/base", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/settings/cluster", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/util", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + "//pkg/util/timeutil", + "//vendor/github.com/cockroachdb/errors", + "//vendor/github.com/cockroachdb/redact", + "//vendor/github.com/cockroachdb/sentry-go", + "//vendor/github.com/kr/pretty", + "//vendor/github.com/pmezard/go-difflib/difflib", + "//vendor/github.com/stretchr/testify/assert", + ] + select({ + "@io_bazel_rules_go//go/platform:aix": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:android": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:darwin": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:dragonfly": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:freebsd": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:illumos": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:ios": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:js": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:linux": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:nacl": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:netbsd": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:openbsd": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:plan9": [ + "//vendor/golang.org/x/sys/unix", + ], + "@io_bazel_rules_go//go/platform:solaris": [ + "//vendor/golang.org/x/sys/unix", + ], + "//conditions:default": [], + }), +) diff --git a/pkg/util/log/crash_reporting.go b/pkg/util/log/logcrash/crash_reporting.go similarity index 94% rename from pkg/util/log/crash_reporting.go rename to pkg/util/log/logcrash/crash_reporting.go index 614bad55bbf2..1b8d56688fb5 100644 --- a/pkg/util/log/crash_reporting.go +++ b/pkg/util/log/logcrash/crash_reporting.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package log +package logcrash import ( "context" @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -136,14 +137,14 @@ func ReportPanic(ctx context.Context, sv *settings.Values, r interface{}, depth // sure, whether some other caller further in the call stack is // catching the panic object in the end or not. panicErr := PanicAsError(depth+1, r) - Shoutf(ctx, severity.ERROR, "a panic has occurred!\n%+v", panicErr) + log.Shoutf(ctx, severity.ERROR, "a panic has occurred!\n%+v", panicErr) // In addition to informing the user, also report the details to telemetry. sendCrashReport(ctx, sv, panicErr, ReportTypePanic) // Ensure that the logs are flushed before letting a panic // terminate the server. - Flush() + log.Flush() } // PanicAsError turns r into an error if it is not one already. @@ -319,10 +320,10 @@ func SendReport( res := sentry.CaptureEvent(event) if res != nil { - Shoutf(ctx, severity.ERROR, "Queued as error %v", string(*res)) + log.Shoutf(ctx, severity.ERROR, "Queued as error %v", string(*res)) } if !sentry.Flush(10 * time.Second) { - Shout(ctx, severity.ERROR, "Timeout trying to submit crash report") + log.Shout(ctx, severity.ERROR, "Timeout trying to submit crash report") } } @@ -340,7 +341,7 @@ func ReportOrPanic( if !build.IsRelease() || (sv != nil && PanicOnAssertions.Get(sv)) { panic(err) } - Warningf(ctx, "%v", err) + log.Warningf(ctx, "%v", err) sendCrashReport(ctx, sv, err, ReportTypeError) } @@ -368,3 +369,15 @@ var tagFns []tagFn func RegisterTagFn(key string, value func(context.Context) string) { tagFns = append(tagFns, tagFn{key, value}) } + +func maybeSendCrashReport(ctx context.Context, err error) { + // We load the ReportingSettings from the a global singleton in this + // call path. See the singleton's comment for a rationale. + if sv := settings.TODO(); sv != nil { + sendCrashReport(ctx, sv, err, ReportTypeLogFatal) + } +} + +func init() { + log.MaybeSendCrashReport = maybeSendCrashReport +} diff --git a/pkg/util/log/crash_reporting_packet_test.go b/pkg/util/log/logcrash/crash_reporting_packet_test.go similarity index 93% rename from pkg/util/log/crash_reporting_packet_test.go rename to pkg/util/log/logcrash/crash_reporting_packet_test.go index 205d20e6fe18..63fc397012dc 100644 --- a/pkg/util/log/crash_reporting_packet_test.go +++ b/pkg/util/log/logcrash/crash_reporting_packet_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package log_test +package logcrash_test import ( "context" @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/redact" "github.com/cockroachdb/sentry-go" "github.com/kr/pretty" @@ -61,11 +62,11 @@ func TestCrashReportingPacket(t *testing.T) { st := cluster.MakeTestingClusterSettings() // Enable all crash-reporting settings. - log.DiagnosticsReportingEnabled.Override(&st.SV, true) + logcrash.DiagnosticsReportingEnabled.Override(&st.SV, true) - defer log.TestingSetCrashReportingURL("https://ignored:ignored@ignored/1234")() + defer logcrash.TestingSetCrashReportingURL("https://ignored:ignored@ignored/1234")() - log.SetupCrashReporter(ctx, "test") + logcrash.SetupCrashReporter(ctx, "test") // Install a Transport that locally records events rather than sending them // to Sentry over HTTP. @@ -91,13 +92,13 @@ func TestCrashReportingPacket(t *testing.T) { func() { defer expectPanic("before server start") - defer log.RecoverAndReportPanic(ctx, &st.SV) + defer logcrash.RecoverAndReportPanic(ctx, &st.SV) panic(log.Safe(panicPre)) }() func() { defer expectPanic("after server start") - defer log.RecoverAndReportPanic(ctx, &st.SV) + defer logcrash.RecoverAndReportPanic(ctx, &st.SV) s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) s.Stopper().Stop(ctx) panic(log.Safe(panicPost)) @@ -151,7 +152,7 @@ func TestCrashReportingPacket(t *testing.T) { p := packets[0] packets = packets[1:] t.Run("", func(t *testing.T) { - if !log.ReportSensitiveDetails { + if !logcrash.ReportSensitiveDetails { e, a := "", p.ServerName if e != a { t.Errorf("expected ServerName to be '', but got '%s'", a) @@ -209,11 +210,11 @@ func TestInternalErrorReporting(t *testing.T) { st := cluster.MakeTestingClusterSettings() // Enable all crash-reporting settings. - log.DiagnosticsReportingEnabled.Override(&st.SV, true) + logcrash.DiagnosticsReportingEnabled.Override(&st.SV, true) - defer log.TestingSetCrashReportingURL("https://ignored:ignored@ignored/1234")() + defer logcrash.TestingSetCrashReportingURL("https://ignored:ignored@ignored/1234")() - log.SetupCrashReporter(ctx, "test") + logcrash.SetupCrashReporter(ctx, "test") // Install a Transport that locally records packets rather than sending them // to Sentry over HTTP. diff --git a/pkg/util/log/crash_reporting_test.go b/pkg/util/log/logcrash/crash_reporting_test.go similarity index 86% rename from pkg/util/log/crash_reporting_test.go rename to pkg/util/log/logcrash/crash_reporting_test.go index 3700b2235f77..120f70acbdbd 100644 --- a/pkg/util/log/crash_reporting_test.go +++ b/pkg/util/log/logcrash/crash_reporting_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package log +package logcrash import ( "context" @@ -20,6 +20,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -77,9 +78,9 @@ Error types: (1) *runtime.TypeAssertionError`, expErr: `some visible detail: interface conversion: interface {} is nil, not int (1) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN - | github.com/cockroachdb/cockroach/pkg/util/log.init + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | ...crash_reporting_test.go:NN | runtime.doInit | ...proc.go:NN @@ -99,9 +100,9 @@ Error types: (1) *withstack.withStack (2) *errutil.withPrefix (3) *runtime.TypeA expErr: `interface conversion: interface {} is nil, not int (1) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN - | github.com/cockroachdb/cockroach/pkg/util/log.init + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | ...crash_reporting_test.go:NN | runtime.doInit | ...proc.go:NN @@ -127,13 +128,13 @@ Wraps: (2) interface conversion: interface {} is nil, not int Error types: (1) *safedetails.withSafeDetails (2) *runtime.TypeAssertionError`, }, { - err: errors.Newf("I like %s and my pin code is %d or %d", Safe("A"), 1234, Safe(9999)), + err: errors.Newf("I like %s and my pin code is %v or %v", log.Safe("A"), "1234", log.Safe("9999")), expErr: `I like A and my pin code is ` + rm + ` or 9999 (1) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN - | github.com/cockroachdb/cockroach/pkg/util/log.init + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | ...crash_reporting_test.go:NN | runtime.doInit | ...proc.go:NN @@ -147,13 +148,13 @@ Wraps: (2) I like A and my pin code is ` + rm + ` or 9999 Error types: (1) *withstack.withStack (2) *errutil.leafError`, }, { - err: errors.Wrapf(context.Canceled, "this is preserved: %d", Safe(6)), + err: errors.Wrapf(context.Canceled, "this is preserved: %d", log.Safe(6)), expErr: `this is preserved: 6: context canceled (1) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN - | github.com/cockroachdb/cockroach/pkg/util/log.init + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | ...crash_reporting_test.go:NN | runtime.doInit | ...proc.go:NN @@ -174,7 +175,7 @@ Error types: (1) *withstack.withStack (2) *errutil.withPrefix (3) *errors.errorS (1) moo ` + rm + ` ` + rm + ` Wraps: (2) assumed safe Wraps: (3) ` + rm + ` -Error types: (1) *os.LinkError (2) *safedetails.withSafeDetails (3) log.leafErr`, +Error types: (1) *os.LinkError (2) *safedetails.withSafeDetails (3) logcrash.leafErr`, }, { // Verify that invalid sentinel errors print something and don't @@ -187,9 +188,9 @@ Error types: (1) *os.LinkError (2) *safedetails.withSafeDetails (3) log.leafErr` expErr: `this is reportable as well: this is reportable too: this is reportable: ` + rm + ` (1) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN - | github.com/cockroachdb/cockroach/pkg/util/log.init + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | ...crash_reporting_test.go:NN | runtime.doInit | ...proc.go:NN @@ -200,9 +201,9 @@ Error types: (1) *os.LinkError (2) *safedetails.withSafeDetails (3) log.leafErr` Wraps: (2) this is reportable as well Wraps: (3) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN - | github.com/cockroachdb/cockroach/pkg/util/log.init + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | ...crash_reporting_test.go:NN | runtime.doInit | ...proc.go:NN @@ -213,15 +214,15 @@ Wraps: (3) attached stack trace Wraps: (4) this is reportable too Wraps: (5) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN | [...repeated from below...] Wraps: (6) this is reportable Wraps: (7) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN - | github.com/cockroachdb/cockroach/pkg/util/log.init + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | ...crash_reporting_test.go:NN | runtime.doInit | ...proc.go:NN @@ -245,16 +246,16 @@ Error types: (1) *withstack.withStack (2) *errutil.withPrefix (3) *withstack.wit expErr: `write tcp ` + rm + ` -> ` + rm + `: ` + rm + ` (1) write tcp ` + rm + ` -> ` + rm + ` Wraps: (2) ` + rm + ` -Error types: (1) *net.OpError (2) log.leafErr`, +Error types: (1) *net.OpError (2) logcrash.leafErr`, }, { err: errFormatted, expErr: `this embed an error: this is reportable too: this is reportable: ` + rm + ` (1) attached stack trace -- stack trace: - | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | ...crash_reporting_test.go:NN - | github.com/cockroachdb/cockroach/pkg/util/log.init + | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | ...crash_reporting_test.go:NN | runtime.doInit | ...proc.go:NN @@ -268,9 +269,9 @@ Wraps: (2) secondary error attachment | this is reportable too: this is reportable: ` + rm + ` | (1) attached stack trace | -- stack trace: - | | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | | ...crash_reporting_test.go:NN - | | github.com/cockroachdb/cockroach/pkg/util/log.init + | | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | | ...crash_reporting_test.go:NN | | runtime.doInit | | ...proc.go:NN @@ -281,15 +282,15 @@ Wraps: (2) secondary error attachment | Wraps: (2) this is reportable too | Wraps: (3) attached stack trace | -- stack trace: - | | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | | ...crash_reporting_test.go:NN | | [...repeated from below...] | Wraps: (4) this is reportable | Wraps: (5) attached stack trace | -- stack trace: - | | github.com/cockroachdb/cockroach/pkg/util/log.glob..func4 + | | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.glob..func2 | | ...crash_reporting_test.go:NN - | | github.com/cockroachdb/cockroach/pkg/util/log.init + | | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.init | | ...crash_reporting_test.go:NN | | runtime.doInit | | ...proc.go:NN diff --git a/pkg/util/log/crash_reporting_unix_test.go b/pkg/util/log/logcrash/crash_reporting_unix_test.go similarity index 97% rename from pkg/util/log/crash_reporting_unix_test.go rename to pkg/util/log/logcrash/crash_reporting_unix_test.go index b15a4d15fd85..d4712237af7e 100644 --- a/pkg/util/log/crash_reporting_unix_test.go +++ b/pkg/util/log/logcrash/crash_reporting_unix_test.go @@ -10,7 +10,7 @@ // +build !windows -package log +package logcrash import ( "os" diff --git a/pkg/util/log/logcrash/main_test.go b/pkg/util/log/logcrash/main_test.go new file mode 100644 index 000000000000..c680db5abb7a --- /dev/null +++ b/pkg/util/log/logcrash/main_test.go @@ -0,0 +1,39 @@ +// Copyright 2017 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package logcrash_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + randutil.SeedForTests() + security.SetAssetLoader(securitytest.EmbeddedAssets) + serverutils.InitTestServerFactory(server.TestServerFactory) + + // MakeTestingClusterSettings initializes log.ReportingSettings to this + // instance of setting values. + // TODO(knz): This comment appears to be untrue. + st := cluster.MakeTestingClusterSettings() + logcrash.DiagnosticsReportingEnabled.Override(&st.SV, false) + logcrash.CrashReports.Override(&st.SV, false) + + os.Exit(m.Run()) +} diff --git a/pkg/util/log/main_test.go b/pkg/util/log/main_test.go index fb79290e3ea8..f552e40a48d8 100644 --- a/pkg/util/log/main_test.go +++ b/pkg/util/log/main_test.go @@ -14,26 +14,11 @@ import ( "os" "testing" - "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/security/securitytest" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) func TestMain(m *testing.M) { randutil.SeedForTests() - security.SetAssetLoader(securitytest.EmbeddedAssets) - serverutils.InitTestServerFactory(server.TestServerFactory) - - // MakeTestingClusterSettings initializes log.ReportingSettings to this - // instance of setting values. - // TODO(knz): This comment appears to be untrue. - st := cluster.MakeTestingClusterSettings() - log.DiagnosticsReportingEnabled.Override(&st.SV, false) - log.CrashReports.Override(&st.SV, false) os.Exit(m.Run()) } diff --git a/pkg/util/log/secondary_log.go b/pkg/util/log/secondary_log.go index 70164fb6a4f7..8261bec1520b 100644 --- a/pkg/util/log/secondary_log.go +++ b/pkg/util/log/secondary_log.go @@ -54,43 +54,35 @@ func NewSecondaryLogger( dir = logging.logDir.String() } l := &SecondaryLogger{ - logger: loggerT{ - logCounter: EntryCounter{EnableMsgCount: enableMsgCount}, - }, + logger: loggerT{}, } + // TODO(knz): Make all this configurable. + // (As done in https://github.com/cockroachdb/cockroach/pull/51987.) fileSink := newFileSink( dir, fileNamePrefix, forceSyncWrites, - severity.INFO, logging.logFileMaxSize, logging.logFilesCombinedMaxSize, l.logger.getStartLines, ) - // TODO(knz): Make all this configurable. - // (As done in https://github.com/cockroachdb/cockroach/pull/51987.) - l.logger.sinkInfos = []sinkInfo{ - { - sink: &logging.stderrSink, - // stderr editor. - // We don't redact upfront, and we keep the redaction markers. - editor: getEditor(SelectEditMode(false /* redact */, true /* keepRedactable */)), - // failure to write to stderr is critical for now. We may want - // to make this non-critical in the future, since it's common - // for folk to close the terminal where they launched 'cockroach - // start --background'. - // We keep this true for now for backward-compatibility. - criticality: true, - }, - { - sink: fileSink, - // file editor. - // We don't redact upfront, and the "--redactable-logs" flag decides - // whether to keep the redaction markers in the output. - editor: getEditor(SelectEditMode(false /* redact */, logging.redactableLogs /* keepRedactable */)), - // failure to write to file is definitely critical. - criticality: true, - }, + fileSinkInfo := &sinkInfo{ + sink: fileSink, + threshold: severity.INFO, + formatter: formatCrdbV1{}, + // file editor. + // We don't redact upfront, and the "--redactable-logs" flag decides + // whether to keep the redaction markers in the output. + editor: getEditor(SelectEditMode(false /* redact */, logging.redactableLogs /* keepRedactable */)), + // failure to write to file is definitely critical. + criticality: true, + } + if enableMsgCount { + fileSinkInfo.formatter = formatCrdbV1WithCounter{} + } + l.logger.sinkInfos = []*sinkInfo{ + &logging.stderrSinkInfo, + fileSinkInfo, } // Ensure the registry knows about this logger. @@ -117,7 +109,7 @@ func (l *SecondaryLogger) output( ctx context.Context, depth int, sev Severity, format string, args ...interface{}, ) { entry := MakeEntry( - ctx, sev, &l.logger.logCounter, depth+1, true /* redactable */, format, args...) + ctx, sev, depth+1, true /* redactable */, format, args...) l.logger.outputLogEntry(entry) } diff --git a/pkg/util/log/secondary_log_test.go b/pkg/util/log/secondary_log_test.go index 5e51c65d0172..0b3c40a99071 100644 --- a/pkg/util/log/secondary_log_test.go +++ b/pkg/util/log/secondary_log_test.go @@ -49,27 +49,29 @@ func TestSecondaryLog(t *testing.T) { // Check that the messages indeed made it to different files. - contents, err := ioutil.ReadFile(debugLog.getFileSink().mu.file.(*syncBuffer).file.Name()) + bcontents, err := ioutil.ReadFile(debugLog.getFileSink().mu.file.(*syncBuffer).file.Name()) if err != nil { t.Fatal(err) } - if !strings.Contains(string(contents), "test1") || !strings.Contains(string(contents), "test2") { + contents := string(bcontents) + if !strings.Contains(contents, "test1") || !strings.Contains(contents, "test2") { t.Errorf("log does not contain error text\n%s", contents) } - if strings.Contains(string(contents), "world") { + if strings.Contains(contents, "world") { t.Errorf("secondary log spilled into debug log\n%s", contents) } - contents, err = ioutil.ReadFile(l.logger.getFileSink().mu.file.(*syncBuffer).file.Name()) + bcontents, err = ioutil.ReadFile(l.logger.getFileSink().mu.file.(*syncBuffer).file.Name()) if err != nil { t.Fatal(err) } - if !strings.Contains(string(contents), "hello") || - !strings.Contains(string(contents), "world") || - !strings.Contains(string(contents), "story time") { - t.Errorf("secondary log does not contain text\n%s", contents) + contents = string(bcontents) + if !strings.Contains(contents, "hello") || + !strings.Contains(contents, "world") || + !strings.Contains(contents, "1 " /* entry counter */ +"story time") { + t.Errorf("secondary log does not contain text or counter\n%s", contents) } - if strings.Contains(string(contents), "test1") { + if strings.Contains(contents, "test1") { t.Errorf("primary log spilled into secondary\n%s", contents) } @@ -80,7 +82,7 @@ func TestRedirectStderrWithSecondaryLoggersActive(t *testing.T) { defer s.Close(t) setFlags() - logging.stderrSink.threshold = severity.NONE + logging.stderrSinkInfo.threshold = severity.NONE // Take over stderr. TestingResetActive() diff --git a/pkg/util/log/sinks.go b/pkg/util/log/sinks.go index ce4b8be80076..29143e3522af 100644 --- a/pkg/util/log/sinks.go +++ b/pkg/util/log/sinks.go @@ -10,27 +10,20 @@ package log -import ( - "github.com/cockroachdb/cockroach/pkg/cli/exit" - "github.com/cockroachdb/cockroach/pkg/util/log/logpb" -) +import "github.com/cockroachdb/cockroach/pkg/cli/exit" // logSink abstracts the destination of logging events, after all // their details have been collected into a logpb.Entry. // // Each logger can have zero or more logSinks attached to it. type logSink interface { - // activeAtSeverity returns true if this sink accepts entries at - // severity sev or higher. - activeAtSeverity(sev logpb.Severity) bool + // active returns true if this sink is currently active. + active() bool // attachHints attaches some hints about the location of the message // to the stack message. attachHints([]byte) []byte - // getFormatter retrieves the entry formatter for this sink. - getFormatter() logFormatter - // output emits some formatted bytes to this sink. // the sink is invited to perform an extra flush if indicated // by the argument. This is set to true for e.g. Fatal diff --git a/pkg/util/log/stderr_redirect.go b/pkg/util/log/stderr_redirect.go index 624c97e479c0..5bc2da12829c 100644 --- a/pkg/util/log/stderr_redirect.go +++ b/pkg/util/log/stderr_redirect.go @@ -41,7 +41,7 @@ var OrigStderr = func() *os.File { // // This is also the logic used by Shout calls. func LoggingToStderr(s Severity) bool { - return s >= logging.stderrSink.threshold + return s >= logging.stderrSinkInfo.threshold } // hijackStderr replaces stderr with the given file descriptor. diff --git a/pkg/util/log/stderr_sink.go b/pkg/util/log/stderr_sink.go index 747ba31497ae..cafc83987808 100644 --- a/pkg/util/log/stderr_sink.go +++ b/pkg/util/log/stderr_sink.go @@ -10,39 +10,23 @@ package log -import ( - "github.com/cockroachdb/cockroach/pkg/cli/exit" - "github.com/cockroachdb/cockroach/pkg/util/log/logpb" -) +import "github.com/cockroachdb/cockroach/pkg/cli/exit" // Type of a stderr copy sink. type stderrSink struct { // the --no-color flag. When set it disables escapes code on the // stderr copy. noColor bool - - // Level at or beyond which entries are output to this sink. - threshold Severity - - // formatter for entries written via this sink. - formatter logFormatter } // activeAtSeverity implements the logSink interface. -func (l *stderrSink) activeAtSeverity(sev logpb.Severity) bool { - return sev >= l.threshold.Get() -} +func (l *stderrSink) active() bool { return true } // attachHints implements the logSink interface. func (l *stderrSink) attachHints(stacks []byte) []byte { return stacks } -// getFormatter implements the logSink interface. -func (l *stderrSink) getFormatter() logFormatter { - return l.formatter -} - // output implements the logSink interface. func (l *stderrSink) output(_ bool, b []byte) error { _, err := OrigStderr.Write(b) diff --git a/pkg/util/log/structured.go b/pkg/util/log/structured.go index 1d8c079c580a..1ff489d48f94 100644 --- a/pkg/util/log/structured.go +++ b/pkg/util/log/structured.go @@ -14,7 +14,6 @@ import ( "context" "strings" - "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/errors" ) @@ -36,19 +35,18 @@ func FormatWithContextTags(ctx context.Context, format string, args ...interface func addStructured( ctx context.Context, sev Severity, depth int, format string, args ...interface{}, ) { - if sev == severity.FATAL { - // We load the ReportingSettings from the a global singleton in this - // call path. See the singleton's comment for a rationale. - if sv := settings.TODO(); sv != nil { - err := errors.NewWithDepthf(depth+1, "log.Fatal: "+format, args...) - sendCrashReport(ctx, sv, err, ReportTypeLogFatal) - } + if sev == severity.FATAL && MaybeSendCrashReport != nil { + err := errors.NewWithDepthf(depth+1, "log.Fatal: "+format, args...) + MaybeSendCrashReport(ctx, err) } entry := MakeEntry( - ctx, sev, &debugLog.logCounter, depth+1, true /* redactable */, format, args...) + ctx, sev, depth+1, true /* redactable */, format, args...) if sp, el, ok := getSpanOrEventLog(ctx); ok { eventInternal(sp, el, (sev >= severity.ERROR), entry) } debugLog.outputLogEntry(entry) } + +// MaybeSendCrashReport is injected by package logcrash +var MaybeSendCrashReport func(ctx context.Context, err error) diff --git a/pkg/util/log/test_log_scope.go b/pkg/util/log/test_log_scope.go index a4c9d6b3ad8f..a3ef8dcc6a14 100644 --- a/pkg/util/log/test_log_scope.go +++ b/pkg/util/log/test_log_scope.go @@ -117,7 +117,7 @@ func ScopeWithoutShowLogs(t tShim) (sc *TestLogScope) { sc = &TestLogScope{ // Remember the stderr threshold. Close() will restore it. - stderrThreshold: logging.stderrSink.threshold.Get(), + stderrThreshold: logging.stderrSinkInfo.threshold.Get(), } defer func() { // If any of the following initialization fails, we close the scope. @@ -145,7 +145,7 @@ func ScopeWithoutShowLogs(t tShim) (sc *TestLogScope) { // Override the stderr threshold for the main logger. // From this point log entries do not show up on stderr any more; // they only go to files. - logging.stderrSink.threshold.SetValue(severity.NONE) + logging.stderrSinkInfo.threshold.SetValue(severity.NONE) t.Logf("test logs captured to: %s", tempDir) return sc @@ -170,7 +170,7 @@ func (l *TestLogScope) Rotate(t tShim) { // restoreStderrThreshold restores the stderr output threshold at the end // of a scope. func (l *TestLogScope) restoreStderrThreshold() { - logging.stderrSink.threshold.SetValue(l.stderrThreshold) + logging.stderrSinkInfo.threshold.SetValue(l.stderrThreshold) } // Close cleans up a TestLogScope. The directory and its contents are diff --git a/pkg/util/log/trace.go b/pkg/util/log/trace.go index 4dde3ec62a50..3ba0e454c62b 100644 --- a/pkg/util/log/trace.go +++ b/pkg/util/log/trace.go @@ -199,7 +199,6 @@ func Event(ctx context.Context, msg string) { // Format the tracing event and add it to the trace. entry := MakeEntry(ctx, severity.INFO, /* unused for trace events */ - nil, /* logCounter, unused for trace events */ 1, /* depth */ // redactable is false because we want to flatten the data in traces // -- we don't have infrastructure yet for trace redaction. @@ -222,7 +221,6 @@ func Eventf(ctx context.Context, format string, args ...interface{}) { // Format the tracing event and add it to the trace. entry := MakeEntry(ctx, severity.INFO, /* unused for trace events */ - nil, /* logCounter, unused for trace events */ 1, /* depth */ // redactable is false because we want to flatten the data in traces // -- we don't have infrastructure yet for trace redaction. @@ -249,7 +247,6 @@ func vEventf( } entry := MakeEntry(ctx, severity.INFO, /* unused for trace events */ - nil, /* logCounter, unused for trace events */ depth+1, // redactable is false because we want to flatten the data in traces // -- we don't have infrastructure yet for trace redaction. diff --git a/pkg/util/mon/BUILD.bazel b/pkg/util/mon/BUILD.bazel index bc182f86d59c..08ab01bed2fc 100644 --- a/pkg/util/mon/BUILD.bazel +++ b/pkg/util/mon/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/util/envutil", "//pkg/util/humanizeutil", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/metric", "//pkg/util/syncutil", "//vendor/github.com/cockroachdb/errors", diff --git a/pkg/util/mon/bytes_usage.go b/pkg/util/mon/bytes_usage.go index 124f4cfdfff8..74b3c24f9e82 100644 --- a/pkg/util/mon/bytes_usage.go +++ b/pkg/util/mon/bytes_usage.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -403,7 +404,7 @@ func (mm *BytesMonitor) doStop(ctx context.Context, check bool) { } if check && mm.mu.curAllocated != 0 { - log.ReportOrPanic( + logcrash.ReportOrPanic( ctx, &mm.settings.SV, "%s: unexpected %d leftover bytes", log.Safe(mm.name), log.Safe(mm.mu.curAllocated)) @@ -582,7 +583,7 @@ func (b *BoundAccount) Grow(ctx context.Context, x int64) error { // Shrink releases part of the cumulated allocations by the specified size. func (b *BoundAccount) Shrink(ctx context.Context, delta int64) { if b.used < delta { - log.ReportOrPanic(ctx, &b.mon.settings.SV, + logcrash.ReportOrPanic(ctx, &b.mon.settings.SV, "%s: no bytes in account to release, current %d, free %d", b.mon.name, b.used, delta) delta = b.used @@ -655,7 +656,7 @@ func (mm *BytesMonitor) releaseBytes(ctx context.Context, sz int64) { mm.mu.Lock() defer mm.mu.Unlock() if mm.mu.curAllocated < sz { - log.ReportOrPanic(ctx, &mm.settings.SV, + logcrash.ReportOrPanic(ctx, &mm.settings.SV, "%s: no bytes to release, current %d, free %d", mm.name, mm.mu.curAllocated, sz) sz = mm.mu.curAllocated diff --git a/pkg/util/stop/BUILD.bazel b/pkg/util/stop/BUILD.bazel index 887cf65b3f9e..949074839988 100644 --- a/pkg/util/stop/BUILD.bazel +++ b/pkg/util/stop/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/util/caller", "//pkg/util/leaktest", "//pkg/util/log", + "//pkg/util/log/logcrash", "//pkg/util/quotapool", "//pkg/util/syncutil", "//pkg/util/tracing", diff --git a/pkg/util/stop/stopper.go b/pkg/util/stop/stopper.go index 2a4467f02078..b21e7a823598 100644 --- a/pkg/util/stop/stopper.go +++ b/pkg/util/stop/stopper.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -201,7 +202,7 @@ func (s *Stopper) Recover(ctx context.Context) { return } if sv := settings.TODO(); sv != nil { - log.ReportPanic(ctx, sv, r, 1) + logcrash.ReportPanic(ctx, sv, r, 1) } panic(r) } From ef184b7dc08b35b6902ca6d34b7724f58ad5eb70 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 30 Nov 2020 13:53:25 +0100 Subject: [PATCH 3/3] util/log: deflake tests that use SQL connections The pgwire code uses goroutines whose execution extends past the server's stopper's `Stop()` call. This is arguably a mis-design which needs to be fixed -- but in the meantime the logging package needs to tolerate it. This patch ensures that asynchronous `log` calls can be performed concurrently with a `TestLogScope`'s `Close()` call. Release note: None --- pkg/util/log/clog.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/util/log/clog.go b/pkg/util/log/clog.go index 8b9243eb540e..6de774b6804c 100644 --- a/pkg/util/log/clog.go +++ b/pkg/util/log/clog.go @@ -273,7 +273,13 @@ func (l *loggerT) outputLogEntry(entry logpb.Entry) { // not eliminate the event. someSinkActive := false for i, s := range l.sinkInfos { - if entry.Severity < s.threshold || !s.sink.active() { + // Note: we need to use the .Get() method instead of reading the + // severity threshold directly, because some tests are unruly and + // let goroutines live and perform log calls beyond their + // Stopper's Stop() call (e.g. the pgwire async processing + // goroutine). These asynchronous log calls are concurrent with + // the stderrSinkInfo update in (*TestLogScope).Close(). + if entry.Severity < s.threshold.Get() || !s.sink.active() { continue } editedEntry := maybeRedactEntry(entry, s.editor)