From 951aba2520cd2e64c8125318cb8280ae386ab847 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Sun, 13 Dec 2020 15:35:12 +0100 Subject: [PATCH 1/3] sql: make writes to `system.eventlog` conditional This patch is meant to help recovering partial availability in clusters where the `system.eventlog` table / range are unsavailable. Prior to this patch, when any SQL action was causing a notable event, that event would be written transactionally (in the same transaction) to the table `system.eventlog`. If that table happened to be unavailable, the action would not complete. This was true of even basic operations like changing a cluster setting, changing privileges on unrelated tables, etc. This patch changes that by introducing a new cluster setting `server.eventlog.enabled` to make these writes conditional. Release note (general change): The new cluster setting `server.eventlog.enabled` controls whether notable events are also written to the table `system.eventlog`. Its default value is `true`. Changing this cluster setting can help recovering partial cluster availability when the `system.eventlog` table becomes unavailable. Note that even when `false`, notable events are still propagated to the logging system, where they can be e.g. redirected to files. --- docs/generated/settings/settings.html | 3 ++- pkg/server/server_systemlog_gc.go | 2 +- pkg/sql/event_log.go | 12 ++++++++++ .../logictest/testdata/logic_test/event_log | 22 +++++++++++++++++++ 4 files changed, 37 insertions(+), 2 deletions(-) diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 176845d2550e..154099c0194f 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -43,7 +43,8 @@ server.clock.forward_jump_check_enabledbooleanfalseif enabled, forward clock jumps > max_offset/2 will cause a panic server.clock.persist_upper_bound_intervalduration0sthe interval between persisting the wall time upper bound of the clock. The clock does not generate a wall time greater than the persisted timestamp and will panic if it sees a wall time greater than this value. When cockroach starts, it waits for the wall time to catch-up till this persisted timestamp. This guarantees monotonic wall time across server restarts. Not setting this or setting a value of 0 disables this feature. server.consistency_check.max_ratebyte size8.0 MiBthe rate limit (bytes/sec) to use for consistency checks; used in conjunction with server.consistency_check.interval to control the frequency of consistency checks. Note that setting this too high can negatively impact performance. -server.eventlog.ttlduration2160h0m0sif nonzero, event log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours. +server.eventlog.enabledbooleantrueif set, logged notable events are also stored in the table system.eventlog +server.eventlog.ttlduration2160h0m0sif nonzero, entries in system.eventlog older than this duration are deleted every 10m0s. Should not be lowered below 24 hours. server.host_based_authentication.configurationstringhost-based authentication configuration to use during connection authentication server.oidc_authentication.autologinbooleanfalseif true, logged-out visitors to the DB Console will be automatically redirected to the OIDC login endpoint (this feature is experimental) server.oidc_authentication.button_textstringLogin with your OIDC providertext to show on button on DB Console login page to login with your OIDC provider (only shown if OIDC is enabled) (this feature is experimental) diff --git a/pkg/server/server_systemlog_gc.go b/pkg/server/server_systemlog_gc.go index 19b9f3301e60..a6d9db263342 100644 --- a/pkg/server/server_systemlog_gc.go +++ b/pkg/server/server_systemlog_gc.go @@ -50,7 +50,7 @@ var ( eventLogTTL = settings.RegisterPublicDurationSetting( "server.eventlog.ttl", fmt.Sprintf( - "if nonzero, event log entries older than this duration are deleted every %s. "+ + "if nonzero, entries in system.eventlog older than this duration are deleted every %s. "+ "Should not be lowered below 24 hours.", systemLogGCPeriod, ), diff --git a/pkg/sql/event_log.go b/pkg/sql/event_log.go index 35cdc609788e..249af64428fd 100644 --- a/pkg/sql/event_log.go +++ b/pkg/sql/event_log.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -105,6 +106,12 @@ func logEventInternalForSQLStatements( event) } +var eventLogEnabled = settings.RegisterPublicBoolSetting( + "server.eventlog.enabled", + "if set, logged notable events are also stored in the table system.eventlog", + true, +) + // InsertEventRecord inserts a single event into the event log as part // of the provided transaction, using the provided internal executor. // @@ -142,6 +149,11 @@ func InsertEventRecord( log.StructuredEvent(ctx, info) }) + // If writes to the event log table are disabled, take a shortcut. + if !eventLogEnabled.Get(&ex.s.cfg.Settings.SV) { + return nil + } + const insertEventTableStmt = ` INSERT INTO system.eventlog ( timestamp, "eventType", "targetID", "reportingID", info diff --git a/pkg/sql/logictest/testdata/logic_test/event_log b/pkg/sql/logictest/testdata/logic_test/event_log index 95088f6f0b83..4d6abbca392e 100644 --- a/pkg/sql/logictest/testdata/logic_test/event_log +++ b/pkg/sql/logictest/testdata/logic_test/event_log @@ -713,3 +713,25 @@ ORDER BY "timestamp", info 1 {"EventType": "drop_schema", "InstanceID": 1, "SchemaName": "sc", "Statement": "DROP SCHEMA \"\".sc", "User": "root"} 1 {"EventType": "drop_schema", "InstanceID": 1, "SchemaName": "s", "Statement": "DROP SCHEMA \"\".s, \"\".t", "User": "root"} 1 {"EventType": "drop_schema", "InstanceID": 1, "SchemaName": "t", "Statement": "DROP SCHEMA \"\".s, \"\".t", "User": "root"} + + +subtest eventlog_setting_disable + +statement ok +SET CLUSTER SETTING server.eventlog.enabled = false + +statement ok +CREATE ROLE rinvisible + +statement ok +DROP ROLE rinvisible + +query ITT +SELECT "reportingID", "eventType", info::JSONB - 'Timestamp' - 'DescriptorID' +FROM system.eventlog +WHERE "eventType" LIKE '%_role' AND info LIKE '%invisible%' +---- + + +statement ok +SET CLUSTER SETTING server.eventlog.enabled = false From 82b1e293961544e48c85e5166ddbc5387b9cb47a Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Mon, 14 Dec 2020 12:39:15 -0600 Subject: [PATCH 2/3] build: move `.pb.go` contents munging from `make` to `protoc-gen-gogoroach` This worked fine, but it complicated #56067 because [the standard Go protobuf Bazel support](https://github.com/bazelbuild/rules_go/blob/master/proto/core.rst#go_proto_library) doesn't easily allow us to just inject a random `sed` command into the middle of the (`.proto` -> `.pb.go` -> compiled Go library) pipeline. With this logic in the actual `protoc` plugin proper, we can then safely use Bazel's built-in stuff without a lot of monkey-patching or code duplication. Release note: None --- Makefile | 16 ------------ build/variables.mk | 1 - pkg/cmd/protoc-gen-gogoroach/main.go | 33 +++++++++++++++++++++++++ pkg/sql/catalog/descpb/structured.pb.go | 1 - pkg/util/metric/metric.pb.go | 1 - 5 files changed, 33 insertions(+), 19 deletions(-) diff --git a/Makefile b/Makefile index 48c7d3422e01..4b1d46d4e7c0 100644 --- a/Makefile +++ b/Makefile @@ -290,11 +290,6 @@ override make-lazy = $(eval $1 = $$(eval $1 := $(value $1))$$($1)) TAR_XFORM_FLAG = $(shell $(TAR) --version | grep -q GNU && echo "--xform='flags=r;s'" || echo "-s") $(call make-lazy,TAR_XFORM_FLAG) -# To edit in-place without creating a backup file, GNU sed requires a bare -i, -# while BSD sed requires an empty string as the following argument. -SED_INPLACE = sed $(shell sed --version 2>&1 | grep -q GNU && echo -i || echo "-i ''") -$(call make-lazy,SED_INPLACE) - # MAKE_TERMERR is set automatically in Make v4.1+, but macOS is still shipping # v3.81. MAKE_TERMERR ?= $(shell [[ -t 2 ]] && echo true) @@ -1279,15 +1274,6 @@ bin/.go_protobuf_sources: $(PROTOC) $(GO_PROTOS) $(GOGOPROTO_PROTO) $(ERRORS_PRO set -e; for dir in $(sort $(dir $(GO_PROTOS))); do \ build/werror.sh $(PROTOC) -Ipkg:./vendor/github.com:$(GOGO_PROTOBUF_PATH):$(PROTOBUF_PATH):$(COREOS_PATH):$(GRPC_GATEWAY_GOOGLEAPIS_PATH):$(ERRORS_PATH) --gogoroach_out=$(PROTO_MAPPINGS),plugins=grpc,import_prefix=github.com/cockroachdb/cockroach/pkg/:./pkg $$dir/*.proto; \ done - $(SED_INPLACE) -E \ - -e '/import _ /d' \ - -e 's!import (fmt|math) "github.com/cockroachdb/cockroach/pkg/(fmt|math)"! !g' \ - -e 's!github\.com/cockroachdb/cockroach/pkg/(etcd)!go.etcd.io/\1!g' \ - -e 's!github.com/cockroachdb/cockroach/pkg/((bytes|encoding/binary|errors|fmt|io|math|github\.com|(google\.)?golang\.org)([^a-z]|$$))!\1!g' \ - -e 's!github.com/cockroachdb/cockroach/pkg/errorspb!github.com/cockroachdb/errors/errorspb!g' \ - -e 's!golang.org/x/net/context!context!g' \ - $(GO_SOURCES) - @# TODO(benesch): Remove the last sed command after https://github.com/grpc/grpc-go/issues/711. gofmt -s -w $(GO_SOURCES) touch $@ @@ -1295,8 +1281,6 @@ bin/.gw_protobuf_sources: $(PROTOC) $(GW_SERVER_PROTOS) $(GW_TS_PROTOS) $(GO_PRO $(FIND_RELEVANT) -type f -name '*.pb.gw.go' -exec rm {} + build/werror.sh $(PROTOC) -Ipkg:./vendor/github.com:$(GOGO_PROTOBUF_PATH):$(PROTOBUF_PATH):$(ERRORS_PATH):$(COREOS_PATH):$(GRPC_GATEWAY_GOOGLEAPIS_PATH) --grpc-gateway_out=logtostderr=true,request_context=true:./pkg $(GW_SERVER_PROTOS) build/werror.sh $(PROTOC) -Ipkg:./vendor/github.com:$(GOGO_PROTOBUF_PATH):$(PROTOBUF_PATH):$(ERRORS_PATH):$(COREOS_PATH):$(GRPC_GATEWAY_GOOGLEAPIS_PATH) --grpc-gateway_out=logtostderr=true,request_context=true:./pkg $(GW_TS_PROTOS) - @# TODO(benesch): Remove after https://github.com/grpc/grpc-go/issues/711. - $(SED_INPLACE) -E 's!golang.org/x/net/context!context!g' $(GW_SOURCES) gofmt -s -w $(GW_SOURCES) @# TODO(jordan,benesch) This can be removed along with the above TODO. goimports -w $(GW_SOURCES) diff --git a/build/variables.mk b/build/variables.mk index d81d9a1284c7..49f0850da952 100644 --- a/build/variables.mk +++ b/build/variables.mk @@ -119,7 +119,6 @@ define VALID_VARS PROTOC_DIR PROTO_MAPPINGS RACETIMEOUT - SED_INPLACE SETTINGS_DOC_PAGE SHELL SQLPARSER_TARGETS diff --git a/pkg/cmd/protoc-gen-gogoroach/main.go b/pkg/cmd/protoc-gen-gogoroach/main.go index 3490f2c91210..4fc5515d43d4 100644 --- a/pkg/cmd/protoc-gen-gogoroach/main.go +++ b/pkg/cmd/protoc-gen-gogoroach/main.go @@ -11,11 +11,41 @@ package main import ( + "regexp" + "strings" + "github.com/gogo/protobuf/protoc-gen-gogo/descriptor" "github.com/gogo/protobuf/vanity" "github.com/gogo/protobuf/vanity/command" ) +// As we invoke it, the generator will sometimes prepend the cockroachdb github +// URL to what should be unqualified standard library imports. This regexp +// allows us to identify and fix those bad imports. +var builtinRegex *regexp.Regexp = regexp.MustCompile(`github.com/cockroachdb/cockroach/pkg/(?P(bytes|context|encoding/binary|errors|fmt|io|math|github\.com|(google\.)?golang\.org)([^a-z]|$$))`) + +func fixImports(s string) string { + lines := strings.Split(s, "\n") + var builder strings.Builder + for _, line := range lines { + if strings.Contains(line, "import _ ") || + strings.Contains(line, "import fmt \"github.com/cockroachdb/cockroach/pkg/fmt\"") || + strings.Contains(line, "import math \"github.com/cockroachdb/cockroach/pkg/math\"") { + continue + } + + line = strings.ReplaceAll(line, "github.com/cockroachdb/cockroach/pkg/etcd", "go.etcd.io/etcd") + line = strings.ReplaceAll(line, "github.com/cockroachdb/cockroach/pkg/errorspb", "github.com/cockroachdb/errors/errorspb") + line = strings.ReplaceAll(line, "golang.org/x/net/context", "context") + if builtinRegex.MatchString(line) { + line = builtinRegex.ReplaceAllString(line, "$1") + } + builder.WriteString(line) + builder.WriteByte('\n') + } + return builder.String() +} + func main() { req := command.Read() files := req.GetProtoFile() @@ -92,5 +122,8 @@ func main() { } resp := command.Generate(req) + for i := 0; i < len(resp.File); i++ { + *resp.File[i].Content = fixImports(*resp.File[i].Content) + } command.Write(resp) } diff --git a/pkg/sql/catalog/descpb/structured.pb.go b/pkg/sql/catalog/descpb/structured.pb.go index d880eded7ec1..135037fe2662 100644 --- a/pkg/sql/catalog/descpb/structured.pb.go +++ b/pkg/sql/catalog/descpb/structured.pb.go @@ -17,7 +17,6 @@ import bytes "bytes" import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" import io "io" - import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" // Reference imports to suppress errors if they are not otherwise used. diff --git a/pkg/util/metric/metric.pb.go b/pkg/util/metric/metric.pb.go index 7ad634790868..25133cd56978 100644 --- a/pkg/util/metric/metric.pb.go +++ b/pkg/util/metric/metric.pb.go @@ -9,7 +9,6 @@ import math "math" import _go "github.com/prometheus/client_model/go" import io "io" - import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" // Reference imports to suppress errors if they are not otherwise used. From 947bd9c0136e861657d15618c2fa739f0a67a690 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 15 Dec 2020 11:44:34 +0100 Subject: [PATCH 3/3] tracing: fix child span optimization When given a context with a non-recording (but real) Span, we would return the incoming context. This would lead to an extra premature call to `Finish()` and subsequent use-after-Finish of the Span, which can blow up and/or hang, most of the time within net/trace code. Prior to this commit, the crash reproduced readily (within seconds) via ``` make stress PKG=./pkg/sql TESTS=TestTrace ``` and I had no issues for ~10 minutes with this commit. Fixes #57875. Release note: None --- pkg/util/tracing/span_test.go | 17 ----------------- pkg/util/tracing/tracer.go | 15 +++++++-------- 2 files changed, 7 insertions(+), 25 deletions(-) diff --git a/pkg/util/tracing/span_test.go b/pkg/util/tracing/span_test.go index ff96717bef9d..ff58c448a4f5 100644 --- a/pkg/util/tracing/span_test.go +++ b/pkg/util/tracing/span_test.go @@ -11,7 +11,6 @@ package tracing import ( - "context" "regexp" "strings" "testing" @@ -182,19 +181,3 @@ Span grandchild: ` require.Equal(t, exp, recToStrippedString(childRec)) } - -func TestChildSpan(t *testing.T) { - tr := NewTracer() - // Set up non-recording span. - sp := tr.StartSpan("foo", WithForceRealSpan()) - ctx := ContextWithSpan(context.Background(), sp) - // Since the parent span was not recording, we would expect the - // noop span back. However - we don't, we get our inputs instead. - // This is a performance optimization; there is a to-do in - // childSpan asking for its removal, but for now it's here to stay. - { - newCtx, newSp := ChildSpan(ctx, "foo") - require.Equal(t, ctx, newCtx) - require.Equal(t, sp, newSp) - } -} diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 57e014d594d9..fcf78a155a8a 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -559,14 +559,13 @@ func childSpan(ctx context.Context, opName string, remote bool) (context.Context opts.RemoteParent = sp.Meta() } newSpan := tr.startSpanGeneric(opName, opts) - if newSpan.isNoop() { - // Optimization: if we end up with a noop, return the inputs - // to avoid ContextWithSpan call below. - // - // TODO(tbg): this is unsound. We are returning the incoming - // context which may have a Span that could later start recording. - // So in effect that span may capture parts of two goroutines - // accidentally. + if sp.isNoop() && newSpan.isNoop() { + // Optimization: if we started and end up with a noop, we can return + // the original context to save on the ContextWithSpan alloc. Note + // that it is important that the incoming span was the noop span. If + // it was a real, non-recording span, it might later start recording. + // Besides, the caller expects to get their own span, and will + // .Finish() it, leading to an extra, premature call to Finish(). return ctx, sp } return ContextWithSpan(ctx, newSpan), newSpan