Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
57879: sql: make writes to `system.eventlog` conditional r=tbg a=knz

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.

57913: build: move `.pb.go` contents munging from `make` to `protoc-gen-gogoroach` r=rickystewart a=rickystewart

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

57942: tracing: fix child span optimization r=knz a=tbg

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


Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
Co-authored-by: Ricky Stewart <ricky@cockroachlabs.com>
Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
  • Loading branch information
4 people committed Dec 15, 2020
4 parents 33091b3 + 951aba2 + 82b1e29 + 947bd9c commit 744f66c
Show file tree
Hide file tree
Showing 11 changed files with 77 additions and 46 deletions.
16 changes: 0 additions & 16 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -1279,24 +1274,13 @@ 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 $@

bin/.gw_protobuf_sources: $(PROTOC) $(GW_SERVER_PROTOS) $(GW_TS_PROTOS) $(GO_PROTOS) $(GOGOPROTO_PROTO) $(ERRORS_PROTO) bin/.bootstrap
$(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)
Expand Down
1 change: 0 additions & 1 deletion build/variables.mk
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,6 @@ define VALID_VARS
PROTOC_DIR
PROTO_MAPPINGS
RACETIMEOUT
SED_INPLACE
SETTINGS_DOC_PAGE
SHELL
SQLPARSER_TARGETS
Expand Down
3 changes: 2 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,8 @@
<tr><td><code>server.clock.forward_jump_check_enabled</code></td><td>boolean</td><td><code>false</code></td><td>if enabled, forward clock jumps > max_offset/2 will cause a panic</td></tr>
<tr><td><code>server.clock.persist_upper_bound_interval</code></td><td>duration</td><td><code>0s</code></td><td>the 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.</td></tr>
<tr><td><code>server.consistency_check.max_rate</code></td><td>byte size</td><td><code>8.0 MiB</code></td><td>the 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.</td></tr>
<tr><td><code>server.eventlog.ttl</code></td><td>duration</td><td><code>2160h0m0s</code></td><td>if nonzero, event log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours.</td></tr>
<tr><td><code>server.eventlog.enabled</code></td><td>boolean</td><td><code>true</code></td><td>if set, logged notable events are also stored in the table system.eventlog</td></tr>
<tr><td><code>server.eventlog.ttl</code></td><td>duration</td><td><code>2160h0m0s</code></td><td>if nonzero, entries in system.eventlog older than this duration are deleted every 10m0s. Should not be lowered below 24 hours.</td></tr>
<tr><td><code>server.host_based_authentication.configuration</code></td><td>string</td><td><code></code></td><td>host-based authentication configuration to use during connection authentication</td></tr>
<tr><td><code>server.oidc_authentication.autologin</code></td><td>boolean</td><td><code>false</code></td><td>if true, logged-out visitors to the DB Console will be automatically redirected to the OIDC login endpoint (this feature is experimental)</td></tr>
<tr><td><code>server.oidc_authentication.button_text</code></td><td>string</td><td><code>Login with your OIDC provider</code></td><td>text 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)</td></tr>
Expand Down
33 changes: 33 additions & 0 deletions pkg/cmd/protoc-gen-gogoroach/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -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<capture>(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()
Expand Down Expand Up @@ -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)
}
2 changes: 1 addition & 1 deletion pkg/server/server_systemlog_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
),
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/catalog/descpb/structured.pb.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

12 changes: 12 additions & 0 deletions pkg/sql/event_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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.
//
Expand Down Expand Up @@ -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
Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/event_log
Original file line number Diff line number Diff line change
Expand Up @@ -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
1 change: 0 additions & 1 deletion pkg/util/metric/metric.pb.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

17 changes: 0 additions & 17 deletions pkg/util/tracing/span_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@
package tracing

import (
"context"
"regexp"
"strings"
"testing"
Expand Down Expand Up @@ -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)
}
}
15 changes: 7 additions & 8 deletions pkg/util/tracing/tracer.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down

0 comments on commit 744f66c

Please sign in to comment.