From 0614a6d2bb51a480ecc8c60897056eb2d7c26c72 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Tue, 13 Sep 2022 12:56:22 -0400 Subject: [PATCH 01/13] tree: Add a JSON encoding benchmark Add a micro benchmark for `tree.AsJSON` method. Release note: None Release justification: test only change --- pkg/sql/sem/tree/BUILD.bazel | 1 + pkg/sql/sem/tree/json_test.go | 116 ++++++++++++++++++++++++++++++++++ 2 files changed, 117 insertions(+) create mode 100644 pkg/sql/sem/tree/json_test.go diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 67a57549fa6e..cdb40f077de8 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -182,6 +182,7 @@ go_test( "function_name_test.go", "indexed_vars_test.go", "interval_test.go", + "json_test.go", "main_test.go", "name_part_test.go", "name_resolution_test.go", diff --git a/pkg/sql/sem/tree/json_test.go b/pkg/sql/sem/tree/json_test.go new file mode 100644 index 000000000000..89b4d9d472fd --- /dev/null +++ b/pkg/sql/sem/tree/json_test.go @@ -0,0 +1,116 @@ +// Copyright 2022 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 tree_test + +import ( + "math/rand" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/sql/randgen" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func BenchmarkAsJSON(b *testing.B) { + // Use fixed seed so that each invocation of this benchmark + // produces exactly the same types, and datums streams. + // This number can be changed to an arbitrary value; doing so + // would result in new types/datums being produced. + rng := randutil.NewTestRandWithSeed(-4365865412074131521) + + const numDatums = 1024 + makeDatums := func(typ *types.T) tree.Datums { + const allowNulls = true + res := make(tree.Datums, numDatums) + for i := 0; i < numDatums; i++ { + res[i] = randgen.RandDatum(rng, typ, allowNulls) + } + return res + } + + bench := func(b *testing.B, typ *types.T) { + b.ReportAllocs() + b.StopTimer() + datums := makeDatums(typ) + b.StartTimer() + + for i := 0; i < b.N; i++ { + _, err := tree.AsJSON(datums[i%numDatums], sessiondatapb.DataConversionConfig{}, time.UTC) + if err != nil { + b.Fatal(err) + } + } + } + + for _, typ := range testTypes(rng) { + b.Run(typ.String(), func(b *testing.B) { + bench(b, typ) + }) + + if randgen.IsAllowedForArray(typ) { + typ = types.MakeArray(typ) + b.Run(typ.String(), func(b *testing.B) { + bench(b, typ) + }) + } + } +} + +// testTypes returns list of types to test against. +func testTypes(rng *rand.Rand) (typs []*types.T) { + for _, typ := range randgen.SeedTypes { + switch typ { + case types.AnyTuple: + // Ignore AnyTuple -- it's not very interesting; we'll generate test tuples below. + case types.RegClass, types.RegNamespace, types.RegProc, types.RegProcedure, types.RegRole, types.RegType: + // Ignore a bunch of pseudo-OID types (just want regular OID). + case types.Geometry, types.Geography: + // Ignore geometry/geography: these types are insanely inefficient; + // AsJson(Geo) -> MarshalGeo -> go JSON bytes -> ParseJSON -> Go native -> json.JSON + // Benchmarking this generates too much noise. + // TODO(yevgeniy): fix this. + default: + typs = append(typs, typ) + } + } + + // Add tuple types. + var tupleTypes []*types.T + makeTupleType := func() *types.T { + contents := make([]*types.T, rng.Intn(6)) // Up to 6 fields + for i := range contents { + contents[i] = randgen.RandTypeFromSlice(rng, typs) + } + candidateTuple := types.MakeTuple(contents) + // Ensure tuple type is unique. + for _, t := range tupleTypes { + if t.Equal(candidateTuple) { + return nil + } + } + tupleTypes = append(tupleTypes, candidateTuple) + return candidateTuple + } + + const numTupleTypes = 5 + for i := 0; i < numTupleTypes; i++ { + var typ *types.T + for typ == nil { + typ = makeTupleType() + } + typs = append(typs, typ) + } + + return typs +} From 922049be3966d582409f4e9313c4b454a88ca819 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Wed, 14 Sep 2022 17:00:40 -0400 Subject: [PATCH 02/13] tree: Improve performance of tree.AsJSON MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Improve performance of `tree.AsJSON` method. These improvements are important for any query that produces large number of JSON objects, as well as to changefeeds, which rely on this function when producing JSON encoded feed. Most of the changes revolved around modifying underlying types (s.a. date/timestamp types, box2d, etc) to favor using functions that append to bytes buffer, instead of relying on slower functions, such as `fmt.Sprintf`. The conversion performance improved around 5-10% for most of the types, and as high as 50% for time types: ``` Benchmark old t/op new t/op delta AsJSON/box2d-10 578ns ± 3% 414ns ± 2% -28.49% (p=0.000 n=10+9) AsJSON/box2d[]-10 1.64µs ± 3% 1.19µs ± 4% -27.14% (p=0.000 n=10+10) AsJSON/time-10 232ns ± 2% 103ns ± 1% -55.61% (p=0.000 n=10+10) AsJSON/time[]-10 687ns ± 4% 342ns ± 4% -50.17% (p=0.000 n=10+10) ``` Note: Some types in the local benchmark show slight slow down in speed. No changes were made in those types, and in general, the encoding speed of these types might be too fast to reliable detect changes: ``` Benchmark old t/op new t/op delta AsJSON/bool[]-10 65.9ns ± 1% 67.7ns ± 2% +2.79% (p=0.001 n=8+9) ``` The emphasis was also placed on reducing allocations. By relying more heavily on a pooled FmtCtx, which contains bytes buffer, some conversions resulted in amortized elimination of allocations (time): ``` Benchmark old B/op new t/op delta AsJSON/timestamp-10 42.1B ± 3% 0.0B -100.00% (p=0.000 n=10+10) AsJSON/timestamp[]-10 174B ± 4% 60B ± 1% -65.75% (p=0.000 n=10+10) ``` Release Note: None Release Justification: performance improvement --- pkg/BUILD.bazel | 4 ++++ pkg/geo/bbox.go | 24 +++++++++++-------- pkg/geo/bbox_test.go | 4 ++++ pkg/sql/sem/tree/datum.go | 31 +++++++++++++++++-------- pkg/sql/sem/tree/format.go | 11 +++++++++ pkg/util/strutil/BUILD.bazel | 19 ++++++++++++++++ pkg/util/strutil/util.go | 34 +++++++++++++++++++++++++++ pkg/util/strutil/util_test.go | 34 +++++++++++++++++++++++++++ pkg/util/timeofday/BUILD.bazel | 1 + pkg/util/timeofday/time_of_day.go | 21 +++++++++++++---- pkg/util/timetz/BUILD.bazel | 1 + pkg/util/timetz/timetz.go | 38 +++++++++++++++++++------------ pkg/util/uint128/uint128.go | 7 ++++++ pkg/util/uuid/uuid.go | 7 ++++-- 14 files changed, 196 insertions(+), 40 deletions(-) create mode 100644 pkg/util/strutil/BUILD.bazel create mode 100644 pkg/util/strutil/util.go create mode 100644 pkg/util/strutil/util_test.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 3b46a7d268b2..d4944daca34c 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -588,6 +588,7 @@ ALL_TESTS = [ "//pkg/util/span:span_test", "//pkg/util/stop:stop_test", "//pkg/util/stringarena:stringarena_test", + "//pkg/util/strutil:strutil_test", "//pkg/util/syncutil/singleflight:singleflight_test", "//pkg/util/syncutil:syncutil_test", "//pkg/util/sysutil:sysutil_test", @@ -2016,6 +2017,8 @@ GO_TARGETS = [ "//pkg/util/stringarena:stringarena", "//pkg/util/stringarena:stringarena_test", "//pkg/util/stringencoding:stringencoding", + "//pkg/util/strutil:strutil", + "//pkg/util/strutil:strutil_test", "//pkg/util/syncutil/singleflight:singleflight", "//pkg/util/syncutil/singleflight:singleflight_test", "//pkg/util/syncutil:syncutil", @@ -2969,6 +2972,7 @@ GET_X_DATA_TARGETS = [ "//pkg/util/stop:get_x_data", "//pkg/util/stringarena:get_x_data", "//pkg/util/stringencoding:get_x_data", + "//pkg/util/strutil:get_x_data", "//pkg/util/syncutil:get_x_data", "//pkg/util/syncutil/singleflight:get_x_data", "//pkg/util/system:get_x_data", diff --git a/pkg/geo/bbox.go b/pkg/geo/bbox.go index bc25ec25189d..b407bb80d37c 100644 --- a/pkg/geo/bbox.go +++ b/pkg/geo/bbox.go @@ -38,15 +38,21 @@ func NewCartesianBoundingBox() *CartesianBoundingBox { // Repr is the string representation of the CartesianBoundingBox. func (b *CartesianBoundingBox) Repr() string { - // fmt.Sprintf with %f does not truncate leading zeroes, so use - // FormatFloat instead. - return fmt.Sprintf( - "BOX(%s %s,%s %s)", - strconv.FormatFloat(b.LoX, 'f', -1, 64), - strconv.FormatFloat(b.LoY, 'f', -1, 64), - strconv.FormatFloat(b.HiX, 'f', -1, 64), - strconv.FormatFloat(b.HiY, 'f', -1, 64), - ) + return string(b.AppendFormat(nil)) +} + +// AppendFormat appends string representation of the CartesianBoundingBox +// to the buffer, and returns modified buffer. +func (b *CartesianBoundingBox) AppendFormat(buf []byte) []byte { + buf = append(buf, "BOX("...) + buf = strconv.AppendFloat(buf, b.LoX, 'f', -1, 64) + buf = append(buf, ' ') + buf = strconv.AppendFloat(buf, b.LoY, 'f', -1, 64) + buf = append(buf, ',') + buf = strconv.AppendFloat(buf, b.HiX, 'f', -1, 64) + buf = append(buf, ' ') + buf = strconv.AppendFloat(buf, b.HiY, 'f', -1, 64) + return append(buf, ')') } // ParseCartesianBoundingBox parses a box2d string into a bounding box. diff --git a/pkg/geo/bbox_test.go b/pkg/geo/bbox_test.go index 0ef94e9d7e6b..f099fb1e6bc7 100644 --- a/pkg/geo/bbox_test.go +++ b/pkg/geo/bbox_test.go @@ -14,6 +14,7 @@ import ( "fmt" "math" "strconv" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/geo/geopb" @@ -63,6 +64,9 @@ func TestParseCartesianBoundingBox(t *testing.T) { } else { require.NoError(t, err) require.Equal(t, tc.expected, ret) + // Test Repr/AppendFormat round trip. + require.Equal(t, strings.ToUpper(tc.s), ret.Repr()) + require.Equal(t, strings.ToUpper(tc.s), string(ret.AppendFormat(nil))) } }) } diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index 676a0f125267..a6a48548a485 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -1782,7 +1782,11 @@ func (d *DUuid) Format(ctx *FmtCtx) { if !bareStrings { ctx.WriteByte('\'') } - ctx.WriteString(d.UUID.String()) + + buf := ctx.scratch[:uuid.RFC4122StrSize] + d.UUID.StringBytes(buf) + ctx.Write(buf) + if !bareStrings { ctx.WriteByte('\'') } @@ -2344,7 +2348,7 @@ func (d *DTime) Format(ctx *FmtCtx) { if !bareStrings { ctx.WriteByte('\'') } - ctx.WriteString(timeofday.TimeOfDay(*d).String()) + ctx.Write(timeofday.TimeOfDay(*d).AppendFormat(ctx.scratch[:0])) if !bareStrings { ctx.WriteByte('\'') } @@ -2520,7 +2524,7 @@ func (d *DTimeTZ) Format(ctx *FmtCtx) { if !bareStrings { ctx.WriteByte('\'') } - ctx.WriteString(d.TimeTZ.String()) + ctx.Write(d.TimeTZ.AppendFormat(ctx.scratch[:0])) if !bareStrings { ctx.WriteByte('\'') } @@ -3603,7 +3607,7 @@ func (d *DBox2D) Format(ctx *FmtCtx) { if !bareStrings { ctx.WriteByte('\'') } - ctx.WriteString(d.CartesianBoundingBox.Repr()) + ctx.Write(d.CartesianBoundingBox.AppendFormat(ctx.scratch[:0])) if !bareStrings { ctx.WriteByte('\'') } @@ -3731,10 +3735,10 @@ func AsJSON( // without the T separator. This causes some compatibility problems // with certain JSON consumers, so we'll use an alternate formatting // path here to maintain consistency with PostgreSQL. - return json.FromString(t.Time.In(loc).Format(time.RFC3339Nano)), nil + return json.FromString(formatTime(t.Time.In(loc), time.RFC3339Nano)), nil case *DTimestamp: // This is RFC3339Nano, but without the TZ fields. - return json.FromString(t.UTC().Format("2006-01-02T15:04:05.999999999")), nil + return json.FromString(formatTime(t.UTC(), "2006-01-02T15:04:05.999999999")), nil case *DDate, *DUuid, *DOid, *DInterval, *DBytes, *DIPAddr, *DTime, *DTimeTZ, *DBitArray, *DBox2D: return json.FromString(AsStringWithFlags(t, FmtBareStrings, FmtDataConversionConfig(dcc))), nil case *DGeometry: @@ -3752,6 +3756,16 @@ func AsJSON( } } +// formatTime formats time with specified layout. +// TODO(yuzefovich): consider using this function in more places. +func formatTime(t time.Time, layout string) string { + // We only need FmtCtx to access its buffer so + // that we get 0 amortized allocations. + ctx := NewFmtCtx(FmtSimple) + ctx.Write(t.AppendFormat(ctx.scratch[:0], layout)) + return ctx.CloseAndGetString() +} + // ResolvedType implements the TypedExpr interface. func (*DJSON) ResolvedType() *types.T { return types.Jsonb @@ -5060,14 +5074,13 @@ func (d *DOid) CompareError(ctx CompareContext, other Datum) (int, error) { // Format implements the Datum interface. func (d *DOid) Format(ctx *FmtCtx) { - s := strconv.FormatUint(uint64(d.Oid), 10) if d.semanticType.Oid() == oid.T_oid || d.name == "" { - ctx.WriteString(s) + ctx.Write(strconv.AppendUint(ctx.scratch[:0], uint64(d.Oid), 10)) } else if ctx.HasFlags(fmtDisambiguateDatumTypes) { ctx.WriteString("crdb_internal.create_") ctx.WriteString(d.semanticType.SQLStandardName()) ctx.WriteByte('(') - ctx.WriteString(s) + ctx.Write(strconv.AppendUint(ctx.scratch[:0], uint64(d.Oid), 10)) ctx.WriteByte(',') lexbase.EncodeSQLStringWithFlags(&ctx.Buffer, d.name, lexbase.EncNoFlags) ctx.WriteByte(')') diff --git a/pkg/sql/sem/tree/format.go b/pkg/sql/sem/tree/format.go index 192354f711e6..457c106b20ef 100644 --- a/pkg/sql/sem/tree/format.go +++ b/pkg/sql/sem/tree/format.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -273,6 +274,8 @@ type FmtCtx struct { // indexedTypeFormatter is an optional interceptor for formatting // IDTypeReferences differently than normal. indexedTypeFormatter func(*FmtCtx, *OIDTypeReference) + // small scratch buffer to reduce allocations. + scratch [64]byte } // FmtCtxOption is an option to pass into NewFmtCtx. @@ -673,3 +676,11 @@ func (ctx *FmtCtx) formatNodeMaybeMarkRedaction(n NodeFormatter) { n.Format(ctx) } } + +func init() { + ctx := NewFmtCtx(FmtSimple) + if len(ctx.scratch) < uuid.RFC4122StrSize { + panic(errors.AssertionFailedf("FmtCtx scratch must be at least %d bytes", uuid.RFC4122StrSize)) + } + ctx.Close() +} diff --git a/pkg/util/strutil/BUILD.bazel b/pkg/util/strutil/BUILD.bazel new file mode 100644 index 000000000000..3d6bf1ae2fe5 --- /dev/null +++ b/pkg/util/strutil/BUILD.bazel @@ -0,0 +1,19 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "strutil", + srcs = ["util.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/util/strutil", + visibility = ["//visibility:public"], +) + +go_test( + name = "strutil_test", + srcs = ["util_test.go"], + args = ["-test.timeout=295s"], + embed = [":strutil"], + deps = ["@com_github_stretchr_testify//require"], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/util/strutil/util.go b/pkg/util/strutil/util.go new file mode 100644 index 000000000000..a1788a07b661 --- /dev/null +++ b/pkg/util/strutil/util.go @@ -0,0 +1,34 @@ +// Copyright 2022 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 strutil + +import "strconv" + +// AppendInt appends the decimal form of x to b and returns the result. +// If the decimal form is shorter than width, the result is padded with leading 0's. +// If the decimal is longer than width, returns formatted decimal without +// any truncation. +func AppendInt(b []byte, x int, width int) []byte { + if x < 0 { + width-- + x = -x + b = append(b, '-') + } + + var scratch [16]byte + xb := strconv.AppendInt(scratch[:0], int64(x), 10) + + // Add 0-padding. + for w := len(xb); w < width; w++ { + b = append(b, '0') + } + return append(b, xb...) +} diff --git a/pkg/util/strutil/util_test.go b/pkg/util/strutil/util_test.go new file mode 100644 index 000000000000..8a94ccfe94f6 --- /dev/null +++ b/pkg/util/strutil/util_test.go @@ -0,0 +1,34 @@ +// Copyright 2022 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 strutil + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestAppendInt(t *testing.T) { + for _, tc := range []struct { + n int + width int + fmt string + }{ + {0, 3, "%03d"}, + {5, 2, "%02d"}, + {10, 1, "%01d"}, + {-11, 4, "%04d"}, + {1234, 6, "%06d"}, + {-321, 2, "%02d"}, + } { + require.Equal(t, fmt.Sprintf(tc.fmt, tc.n), string(AppendInt(nil, tc.n, tc.width))) + } +} diff --git a/pkg/util/timeofday/BUILD.bazel b/pkg/util/timeofday/BUILD.bazel index a9424abf9319..0c723b337c29 100644 --- a/pkg/util/timeofday/BUILD.bazel +++ b/pkg/util/timeofday/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/util/duration", + "//pkg/util/strutil", "//pkg/util/timeutil", ], ) diff --git a/pkg/util/timeofday/time_of_day.go b/pkg/util/timeofday/time_of_day.go index d3de2de87c96..04694e4de4fb 100644 --- a/pkg/util/timeofday/time_of_day.go +++ b/pkg/util/timeofday/time_of_day.go @@ -11,12 +11,12 @@ package timeofday import ( - "fmt" + "bytes" "math/rand" - "strings" "time" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/strutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -52,12 +52,23 @@ func New(hour, min, sec, micro int) TimeOfDay { } func (t TimeOfDay) String() string { + return string(t.AppendFormat(nil)) +} + +// AppendFormat appends this TimeOfDay format to the specified buffer. +func (t TimeOfDay) AppendFormat(buf []byte) []byte { + buf = strutil.AppendInt(buf, t.Hour(), 2) + buf = append(buf, ':') + buf = strutil.AppendInt(buf, t.Minute(), 2) + buf = append(buf, ':') + buf = strutil.AppendInt(buf, t.Second(), 2) micros := t.Microsecond() if micros > 0 { - s := fmt.Sprintf("%02d:%02d:%02d.%06d", t.Hour(), t.Minute(), t.Second(), micros) - return strings.TrimRight(s, "0") + buf = append(buf, '.') + buf = strutil.AppendInt(buf, micros, 6) + buf = bytes.TrimRight(buf, "0") } - return fmt.Sprintf("%02d:%02d:%02d", t.Hour(), t.Minute(), t.Second()) + return buf } // FromInt constructs a TimeOfDay from an int64, representing microseconds since diff --git a/pkg/util/timetz/BUILD.bazel b/pkg/util/timetz/BUILD.bazel index 9d99cc9b2e8d..4482dbdf540d 100644 --- a/pkg/util/timetz/BUILD.bazel +++ b/pkg/util/timetz/BUILD.bazel @@ -9,6 +9,7 @@ go_library( deps = [ "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", + "//pkg/util/strutil", "//pkg/util/timeofday", "//pkg/util/timeutil", "//pkg/util/timeutil/pgdate", diff --git a/pkg/util/timetz/timetz.go b/pkg/util/timetz/timetz.go index 9969649fc190..71d62fcc48ba 100644 --- a/pkg/util/timetz/timetz.go +++ b/pkg/util/timetz/timetz.go @@ -11,12 +11,12 @@ package timetz import ( - "fmt" "regexp" "time" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/util/strutil" "github.com/cockroachdb/cockroach/pkg/util/timeofday" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" @@ -150,25 +150,33 @@ func ParseTimeTZ( // String implements the Stringer interface. func (t *TimeTZ) String() string { + return string(t.AppendFormat(nil)) +} + +// AppendFormat appends TimeTZ to the buffer, and returns modified buffer. +func (t *TimeTZ) AppendFormat(buf []byte) []byte { tTime := t.ToTime() - timeComponent := tTime.Format("15:04:05.999999") - // 24:00:00 gets returned as 00:00:00, which is incorrect. if t.TimeOfDay == timeofday.Time2400 { - timeComponent = "24:00:00" + // 24:00:00 gets returned as 00:00:00, which is incorrect. + buf = append(buf, "24:00:00"...) + } else { + buf = tTime.AppendFormat(buf, "15:04:05.999999") } - timeZoneComponent := tTime.Format("Z07:00:00") - // If it is UTC, .Format converts it to "Z". - // Fully expand this component. + if t.OffsetSecs == 0 { - timeZoneComponent = "+00:00:00" - } - // Go's time.Format functionality does not work for offsets which - // in the range -0s < offsetSecs < -60s, e.g. -22s offset prints as 00:00:-22. - // Manually correct for this. - if 0 < t.OffsetSecs && t.OffsetSecs < 60 { - timeZoneComponent = fmt.Sprintf("-00:00:%02d", t.OffsetSecs) + // If it is UTC, .Format converts it to "Z". + // Fully expand this component. + buf = append(buf, "+00:00:00"...) + } else if 0 < t.OffsetSecs && t.OffsetSecs < 60 { + // Go's time.Format functionality does not work for offsets which + // in the range -0s < offsetSecs < -60s, e.g. -22s offset prints as 00:00:-22. + // Manually correct for this. + buf = append(buf, "-00:00:"...) + buf = strutil.AppendInt(buf, int(t.OffsetSecs), 2) + } else { + buf = tTime.AppendFormat(buf, "Z07:00:00") } - return timeComponent + timeZoneComponent + return buf } // ToTime converts a DTimeTZ to a time.Time, corrected to the given location. diff --git a/pkg/util/uint128/uint128.go b/pkg/util/uint128/uint128.go index e54e24cac611..86fff00e7b62 100644 --- a/pkg/util/uint128/uint128.go +++ b/pkg/util/uint128/uint128.go @@ -30,6 +30,13 @@ func (u Uint128) GetBytes() []byte { return buf } +// AppendBytes appends big-endian byte representation to the +// buffer and returns the buffer. +func (u Uint128) AppendBytes(buf []byte) []byte { + buf = binary.BigEndian.AppendUint64(buf, u.Hi) + return binary.BigEndian.AppendUint64(buf, u.Lo) +} + // String returns a hexadecimal string representation. func (u Uint128) String() string { return hex.EncodeToString(u.GetBytes()) diff --git a/pkg/util/uuid/uuid.go b/pkg/util/uuid/uuid.go index f161a6b72b87..fd0951511ac4 100644 --- a/pkg/util/uuid/uuid.go +++ b/pkg/util/uuid/uuid.go @@ -30,6 +30,9 @@ import ( // Size of a UUID in bytes. const Size = 16 +// RFC4122StrSize of a size of the RFC-4122 string representation of UUID. +const RFC4122StrSize = 36 + // Bytes represents a byte slice which is intended to be interpreted as a binary // encoding of a UUID. type Bytes []byte @@ -144,7 +147,7 @@ func (u *UUID) bytesMut() []byte { // String returns a canonical RFC-4122 string representation of the UUID: // xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx. func (u UUID) String() string { - buf := make([]byte, 36) + buf := make([]byte, RFC4122StrSize) u.StringBytes(buf) return string(buf) } @@ -152,7 +155,7 @@ func (u UUID) String() string { // StringBytes writes the result of String directly into a buffer, which must // have a length of at least 36. func (u UUID) StringBytes(buf []byte) { - _ = buf[:36] + _ = buf[:RFC4122StrSize] hex.Encode(buf[0:8], u[0:4]) buf[8] = '-' hex.Encode(buf[9:13], u[4:6]) From 7ffed1eaf5311960fb27fc3fb1eadc1f726c7476 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Tue, 13 Sep 2022 08:02:27 -0400 Subject: [PATCH 03/13] changefeedccl: Add JSON encoder benchmark. Add JSON encoder benchmark. Release note: None Release justification: test only change --- pkg/ccl/changefeedccl/cdcevent/event.go | 55 +++++++ pkg/ccl/changefeedccl/encoder_test.go | 201 ++++++++++++++++++++++++ 2 files changed, 256 insertions(+) diff --git a/pkg/ccl/changefeedccl/cdcevent/event.go b/pkg/ccl/changefeedccl/cdcevent/event.go index 41123760f6a7..1e953dfdccf1 100644 --- a/pkg/ccl/changefeedccl/cdcevent/event.go +++ b/pkg/ccl/changefeedccl/cdcevent/event.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/cache" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -549,3 +550,57 @@ func TestingGetFamilyIDFromKey( _, familyID, err := decoder.(*eventDecoder).rfCache.tableDescForKey(context.Background(), key, ts) return familyID, err } + +// TestingMakeEventRowFromEncDatums creates event row from specified enc datum row. +// encRow assumed to contain *already decoded* datums. +// The first numKeyCols are assumed to be primary key columns. +// Columns names are generated based on the datum types. +func TestingMakeEventRowFromEncDatums( + encRow rowenc.EncDatumRow, colTypes []*types.T, numKeyCols int, deleted bool, +) Row { + if len(encRow) != len(colTypes) { + panic("unexpected length mismatch") + } + intRange := func(start, end int) (res []int) { + for i := 0; i < end; i++ { + res = append(res, i) + } + return res + } + ed := &EventDescriptor{ + Metadata: Metadata{ + TableID: 42, + TableName: "randtbl", + FamilyName: "primary", + }, + cols: func() (cols []ResultColumn) { + names := make(map[string]int, len(encRow)) + for i, typ := range colTypes { + colName := fmt.Sprintf("col_%s", typ.String()) + names[colName]++ + if names[colName] > 1 { + colName += fmt.Sprintf("_%d", names[colName]-1) + } + cols = append(cols, ResultColumn{ + ResultColumn: colinfo.ResultColumn{ + Name: colName, + Typ: typ, + TableID: 42, + }, + ord: i, + }) + } + return cols + }(), + keyCols: intRange(0, numKeyCols), + valueCols: intRange(0, len(encRow)), + } + + var alloc tree.DatumAlloc + return Row{ + EventDescriptor: ed, + datums: encRow, + deleted: deleted, + alloc: &alloc, + } +} diff --git a/pkg/ccl/changefeedccl/encoder_test.go b/pkg/ccl/changefeedccl/encoder_test.go index 9337cfa5c93d..16f29baf169c 100644 --- a/pkg/ccl/changefeedccl/encoder_test.go +++ b/pkg/ccl/changefeedccl/encoder_test.go @@ -12,6 +12,7 @@ import ( "context" gosql "database/sql" "fmt" + "math/rand" "net/url" "testing" @@ -20,13 +21,17 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/workload/ledger" "github.com/cockroachdb/cockroach/pkg/workload/workloadsql" "github.com/stretchr/testify/require" @@ -841,3 +846,199 @@ func TestAvroLedger(t *testing.T) { cdcTest(t, testFn, feedTestForceSink("kafka")) } + +func BenchmarkEncoders(b *testing.B) { + rng := randutil.NewTestRandWithSeed(2365865412074131521) + + // Initialize column types for tests; this is done before + // benchmark runs so that all reruns use the same types as generated + // by random seed above. + const maxKeyCols = 4 + const maxValCols = 2< MarshalGeo -> go JSON bytes -> ParseJSON -> Go native -> json.JSON + // Benchmarking this generates too much noise. + // TODO: fix this. + case types.Void: + // Just not a very interesting thing to encode + default: + allTypes = append(allTypes, typ) + } + } + + // Add tuple types. + var tupleTypes []*types.T + makeTupleType := func() *types.T { + contents := make([]*types.T, rng.Intn(6)) // Up to 6 fields + for i := range contents { + contents[i] = randgen.RandTypeFromSlice(rng, allTypes) + } + candidateTuple := types.MakeTuple(contents) + // Ensure tuple type is unique. + for _, t := range tupleTypes { + if t.Equal(candidateTuple) { + return nil + } + } + tupleTypes = append(tupleTypes, candidateTuple) + return candidateTuple + } + + const numTupleTypes = 5 + for i := 0; i < numTupleTypes; i++ { + var typ *types.T + for typ == nil { + typ = makeTupleType() + } + allTypes = append(allTypes, typ) + } + + randTypes := func(numTypes int, mustBeKeyType bool) []*types.T { + typs := make([]*types.T, numTypes) + for i := range typs { + typ := randgen.RandTypeFromSlice(rng, allTypes) + for mustBeKeyType && colinfo.MustBeValueEncoded(typ) { + typ = randgen.RandTypeFromSlice(rng, allTypes) + } + typs[i] = typ + } + return typs + } + + return randTypes(numKeyTypes, true), randTypes(numColTypes, false) +} From f1878cd8910c254eabcabad58ec4772a71d4fa2b Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Thu, 15 Sep 2022 11:42:01 -0400 Subject: [PATCH 04/13] changefeedccl: Improve JSON encoder performance Rewrite JSON encoder to improve its performance. Prior to this change JSON encoder was very inefficient. This inefficiency had multiple underlying reasons: * New Go map objects were constructed for each event. * Underlying json conversion functions had inefficiencies (tracked in https://github.com/cockroachdb/cockroach/pull/87968) * Conversion of Go maps to JSON incurs the cost of sorting the keys -- for each row. Sorting, particularly when rows are wide, has significant cost. * Each conversion to JSON allocated new array builder (to encode keys) and new object builder; that too has cost. * Underlying code structure, while attempting to reuse code when constructing different "envelope" formats, cause the code to be more inefficient. This PR addresses all of the above. In particular, since a schema version for the table is guaranteeed to have the same set of primary key and value columns, we can construct JSON builders once. The expensive sort operation can be performed once per version; builders can be memoized and cached. The performance impact is significant: * Key encoding speed up is 5-30%, depending on the number of primary keys. * Value encoding 30% - 60% faster (slowest being "wrapped" envelope with diff -- which effectively encodes 2x values) * Byte allocations per row reduces by over 70%, with the number of allocations reduced similarly. Release note (enterprise change): Changefeed JSON encoder performance improved by 50%. Release justification: performance improvement --- pkg/ccl/changefeedccl/bench_test.go | 2 +- pkg/ccl/changefeedccl/cdcevent/BUILD.bazel | 1 + pkg/ccl/changefeedccl/cdcevent/event.go | 4 +- .../cdcevent/rowfetcher_cache.go | 18 +- .../changefeedccl/cdcevent/version_cache.go | 42 +++ pkg/ccl/changefeedccl/changefeed_stmt.go | 3 +- pkg/ccl/changefeedccl/changefeed_test.go | 6 +- pkg/ccl/changefeedccl/encoder.go | 2 +- pkg/ccl/changefeedccl/encoder_json.go | 320 +++++++++++++----- pkg/ccl/changefeedccl/sink_cloudstorage.go | 2 +- .../changefeedccl/sink_cloudstorage_test.go | 2 +- pkg/ccl/changefeedccl/sink_webhook.go | 2 +- pkg/ccl/changefeedccl/sink_webhook_test.go | 2 +- pkg/ccl/changefeedccl/testfeed_test.go | 95 ++++-- pkg/util/json/BUILD.bazel | 1 + pkg/util/json/json.go | 56 +++ pkg/util/json/json_test.go | 60 +++- 17 files changed, 469 insertions(+), 149 deletions(-) create mode 100644 pkg/ccl/changefeedccl/cdcevent/version_cache.go diff --git a/pkg/ccl/changefeedccl/bench_test.go b/pkg/ccl/changefeedccl/bench_test.go index 08a7f06d1d33..1d4d3916b025 100644 --- a/pkg/ccl/changefeedccl/bench_test.go +++ b/pkg/ccl/changefeedccl/bench_test.go @@ -210,7 +210,7 @@ func createBenchmarkChangefeed( } initialHighWater := hlc.Timestamp{} encodingOpts := changefeedbase.EncodingOptions{Format: changefeedbase.OptFormatJSON, Envelope: changefeedbase.OptEnvelopeRow} - encoder, err := makeJSONEncoder(encodingOpts, AllTargets(details)) + encoder, err := makeJSONEncoder(encodingOpts) if err != nil { return nil, nil, err } diff --git a/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel b/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel index c851d9b62c88..ca9c8ea23c10 100644 --- a/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "event.go", "projection.go", "rowfetcher_cache.go", + "version_cache.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent", visibility = ["//visibility:public"], diff --git a/pkg/ccl/changefeedccl/cdcevent/event.go b/pkg/ccl/changefeedccl/cdcevent/event.go index 1e953dfdccf1..9534584a8ea7 100644 --- a/pkg/ccl/changefeedccl/cdcevent/event.go +++ b/pkg/ccl/changefeedccl/cdcevent/event.go @@ -346,7 +346,7 @@ func getEventDescriptorCached( schemaTS hlc.Timestamp, cache *cache.UnorderedCache, ) (*EventDescriptor, error) { - idVer := idVersion{id: desc.GetID(), version: desc.GetVersion(), family: family.ID} + idVer := CacheKey{ID: desc.GetID(), Version: desc.GetVersion(), FamilyID: family.ID} if v, ok := cache.Get(idVer); ok { ed := v.(*EventDescriptor) @@ -382,7 +382,7 @@ func NewEventDecoder( return nil, err } - eventDescriptorCache := cache.NewUnorderedCache(defaultCacheConfig) + eventDescriptorCache := cache.NewUnorderedCache(DefaultCacheConfig) getEventDescriptor := func( desc catalog.TableDescriptor, family *descpb.ColumnFamilyDescriptor, diff --git a/pkg/ccl/changefeedccl/cdcevent/rowfetcher_cache.go b/pkg/ccl/changefeedccl/cdcevent/rowfetcher_cache.go index 3f5523eeadb8..7d29b5af9c91 100644 --- a/pkg/ccl/changefeedccl/cdcevent/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/cdcevent/rowfetcher_cache.go @@ -57,20 +57,6 @@ type watchedFamily struct { familyName string } -var defaultCacheConfig = cache.Config{ - Policy: cache.CacheFIFO, - // TODO: If we find ourselves thrashing here in changefeeds on many tables, - // we can improve performance by eagerly evicting versions using Resolved notifications. - // A old version with a timestamp entirely before a notification can be safely evicted. - ShouldEvict: func(size int, _ interface{}, _ interface{}) bool { return size > 1024 }, -} - -type idVersion struct { - id descpb.ID - version descpb.DescriptorVersion - family descpb.FamilyID -} - // newRowFetcherCache constructs row fetcher cache. func newRowFetcherCache( ctx context.Context, @@ -96,7 +82,7 @@ func newRowFetcherCache( leaseMgr: leaseMgr, collection: cf.NewCollection(ctx, nil /* TemporarySchemaProvider */, nil /* monitor */), db: db, - fetchers: cache.NewUnorderedCache(defaultCacheConfig), + fetchers: cache.NewUnorderedCache(DefaultCacheConfig), watchedFamilies: watchedFamilies, }, err } @@ -192,7 +178,7 @@ var ErrUnwatchedFamily = errors.New("watched table but unwatched family") func (c *rowFetcherCache) RowFetcherForColumnFamily( tableDesc catalog.TableDescriptor, family descpb.FamilyID, ) (*row.Fetcher, *descpb.ColumnFamilyDescriptor, error) { - idVer := idVersion{id: tableDesc.GetID(), version: tableDesc.GetVersion(), family: family} + idVer := CacheKey{ID: tableDesc.GetID(), Version: tableDesc.GetVersion(), FamilyID: family} if v, ok := c.fetchers.Get(idVer); ok { f := v.(*cachedFetcher) if f.skip { diff --git a/pkg/ccl/changefeedccl/cdcevent/version_cache.go b/pkg/ccl/changefeedccl/cdcevent/version_cache.go new file mode 100644 index 000000000000..fabcb5ee68b3 --- /dev/null +++ b/pkg/ccl/changefeedccl/cdcevent/version_cache.go @@ -0,0 +1,42 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package cdcevent + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util/cache" +) + +// DefaultCacheConfig is the default configuration for unordered cache. +var DefaultCacheConfig = cache.Config{ + Policy: cache.CacheFIFO, + // TODO: If we find ourselves thrashing here in changefeeds on many tables, + // we can improve performance by eagerly evicting versions using Resolved notifications. + // A old Version with a timestamp entirely before a notification can be safely evicted. + ShouldEvict: func(size int, _ interface{}, _ interface{}) bool { return size > 1024 }, +} + +// CacheKey is the key for the event caches. +type CacheKey struct { + ID descpb.ID + Version descpb.DescriptorVersion + FamilyID descpb.FamilyID +} + +// GetCachedOrCreate returns cached object, or creates and caches new one. +func GetCachedOrCreate( + k CacheKey, c *cache.UnorderedCache, creator func() interface{}, +) interface{} { + if v, ok := c.Get(k); ok { + return v + } + v := creator() + c.Add(k, v) + return v +} diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index 8f344fef50f4..810fa3a420a7 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -525,7 +525,8 @@ func createChangefeedJobRecord( // the default error to avoid claiming the user set an option they didn't // explicitly set. Fortunately we know the only way to cause this is to // set envelope. - if isCloudStorageSink(parsedSink) || isWebhookSink(parsedSink) { + if (isCloudStorageSink(parsedSink) || isWebhookSink(parsedSink)) && + encodingOpts.Envelope != changefeedbase.OptEnvelopeBare { if err = opts.ForceKeyInValue(); err != nil { return nil, errors.Errorf(`this sink is incompatible with envelope=%s`, encodingOpts.Envelope) } diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index a81d9ff00f15..1b2a063f36bd 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -1048,7 +1048,7 @@ func TestChangefeedProjectionDelete(t *testing.T) { `foo: [0]->{}`, }) } - cdcTest(t, testFn) + cdcTest(t, testFn, feedTestForceSink("cloudstorage")) } // If we drop columns which are not targeted by the changefeed, it should not backfill. @@ -2476,9 +2476,7 @@ func TestChangefeedBareJSON(t *testing.T) { sqlDB.Exec(t, `INSERT INTO foo values (0, 'dog')`) foo := feed(t, f, `CREATE CHANGEFEED WITH schema_change_policy=stop AS SELECT * FROM foo`) defer closeFeed(t, foo) - assertPayloads(t, foo, []string{ - `foo: [0]->{"a": 0, "b": "dog"}`, - }) + assertPayloads(t, foo, []string{`foo: [0]->{"a": 0, "b": "dog"}`}) } cdcTest(t, testFn, feedTestForceSink("kafka")) cdcTest(t, testFn, feedTestForceSink("enterprise")) diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index a954fb41c4bb..c08f4dff42e1 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -46,7 +46,7 @@ func getEncoder( ) (Encoder, error) { switch opts.Format { case changefeedbase.OptFormatJSON: - return makeJSONEncoder(opts, targets) + return makeJSONEncoder(opts) case changefeedbase.OptFormatAvro, changefeedbase.DeprecatedOptFormatAvro: return newConfluentAvroEncoder(opts, targets) case changefeedbase.OptFormatCSV: diff --git a/pkg/ccl/changefeedccl/encoder_json.go b/pkg/ccl/changefeedccl/encoder_json.go index 2cd770818372..03f1a96ba1fe 100644 --- a/pkg/ccl/changefeedccl/encoder_json.go +++ b/pkg/ccl/changefeedccl/encoder_json.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/util/cache" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" @@ -32,8 +33,9 @@ type jsonEncoder struct { updatedField, mvccTimestampField, beforeField, keyInValue, topicInValue bool envelopeType changefeedbase.EnvelopeType - targets changefeedbase.Targets - buf bytes.Buffer + buf bytes.Buffer + versionEncoder func(ed *cdcevent.EventDescriptor) *versionEncoder + envelopeEncoder func(evCtx eventContext, updated, prev cdcevent.Row) (json.JSON, error) } var _ Encoder = &jsonEncoder{} @@ -45,20 +47,29 @@ func canJSONEncodeMetadata(e changefeedbase.EnvelopeType) bool { return e == changefeedbase.OptEnvelopeBare || e == changefeedbase.OptEnvelopeWrapped } -func makeJSONEncoder( - opts changefeedbase.EncodingOptions, targets changefeedbase.Targets, -) (*jsonEncoder, error) { +func makeJSONEncoder(opts changefeedbase.EncodingOptions) (*jsonEncoder, error) { + versionCache := cache.NewUnorderedCache(cdcevent.DefaultCacheConfig) e := &jsonEncoder{ - targets: targets, - envelopeType: opts.Envelope, - } - e.updatedField = opts.UpdatedTimestamps - e.mvccTimestampField = opts.MVCCTimestamps - // In the bare envelope we don't output diff directly, it's incorporated into the - // projection as desired. - e.beforeField = opts.Diff && opts.Envelope != changefeedbase.OptEnvelopeBare - e.keyInValue = opts.KeyInValue - e.topicInValue = opts.TopicInValue + envelopeType: opts.Envelope, + updatedField: opts.UpdatedTimestamps, + mvccTimestampField: opts.MVCCTimestamps, + // In the bare envelope we don't output diff directly, it's incorporated into the + // projection as desired. + beforeField: opts.Diff && opts.Envelope != changefeedbase.OptEnvelopeBare, + keyInValue: opts.KeyInValue, + topicInValue: opts.TopicInValue, + versionEncoder: func(ed *cdcevent.EventDescriptor) *versionEncoder { + key := cdcevent.CacheKey{ + ID: ed.TableID, + Version: ed.Version, + FamilyID: ed.FamilyID, + } + return cdcevent.GetCachedOrCreate(key, versionCache, func() interface{} { + return &versionEncoder{} + }).(*versionEncoder) + }, + } + if !canJSONEncodeMetadata(e.envelopeType) { if e.keyInValue { return nil, errors.Errorf(`%s is only usable with %s=%s`, @@ -69,16 +80,28 @@ func makeJSONEncoder( changefeedbase.OptTopicInValue, changefeedbase.OptEnvelope, changefeedbase.OptEnvelopeWrapped) } } + + if e.envelopeType == changefeedbase.OptEnvelopeWrapped { + if err := e.initWrappedEnvelope(); err != nil { + return nil, err + } + } else { + if err := e.initRawEnvelope(); err != nil { + return nil, err + } + } + return e, nil } +// versionEncoder memoizes version specific encoding state. +type versionEncoder struct { + valueBuilder *json.FixedKeysObjectBuilder +} + // EncodeKey implements the Encoder interface. func (e *jsonEncoder) EncodeKey(_ context.Context, row cdcevent.Row) ([]byte, error) { - jsonEntries, err := e.encodeKeyRaw(row) - if err != nil { - return nil, err - } - j, err := json.MakeJSON(jsonEntries) + j, err := e.versionEncoder(row.EventDescriptor).encodeKeyRaw(row) if err != nil { return nil, err } @@ -87,126 +110,243 @@ func (e *jsonEncoder) EncodeKey(_ context.Context, row cdcevent.Row) ([]byte, er return e.buf.Bytes(), nil } -func (e *jsonEncoder) encodeKeyRaw(row cdcevent.Row) ([]interface{}, error) { - var jsonEntries []interface{} +func (e *versionEncoder) encodeKeyRaw(row cdcevent.Row) (json.JSON, error) { + kb := json.NewArrayBuilder(1) if err := row.ForEachKeyColumn().Datum(func(d tree.Datum, col cdcevent.ResultColumn) error { j, err := tree.AsJSON(d, sessiondatapb.DataConversionConfig{}, time.UTC) if err != nil { return err } - jsonEntries = append(jsonEntries, j) + kb.Add(j) return nil }); err != nil { return nil, err } - return jsonEntries, nil + return kb.Build(), nil } -func rowAsGoNative(row cdcevent.Row) (map[string]interface{}, error) { +func (e *versionEncoder) encodeKeyInValue( + updated cdcevent.Row, b *json.FixedKeysObjectBuilder, +) error { + keyEntries, err := e.encodeKeyRaw(updated) + if err != nil { + return err + } + return b.Set("key", keyEntries) +} + +var emptyJSONValue = func() json.JSON { + j, err := json.MakeJSON(map[string]interface{}{}) + if err != nil { + panic(err) + } + return j +}() + +func (e *versionEncoder) rowAsGoNative(row cdcevent.Row, meta json.JSON) (json.JSON, error) { if !row.HasValues() || row.IsDeleted() { - return nil, nil + if meta != nil { + b := json.NewObjectBuilder(1) + b.Add(jsonMetaSentinel, meta) + return b.Build(), nil + } + return json.NullJSONValue, nil } - result := make(map[string]interface{}) - if err := row.ForEachColumn().Datum(func(d tree.Datum, col cdcevent.ResultColumn) (err error) { - result[col.Name], err = tree.AsJSON(d, sessiondatapb.DataConversionConfig{}, time.UTC) - return err + if e.valueBuilder == nil { + keys := make([]string, 0, len(row.ResultColumns())) + _ = row.ForEachColumn().Col(func(col cdcevent.ResultColumn) error { + keys = append(keys, col.Name) + return nil + }) + if meta != nil { + keys = append(keys, jsonMetaSentinel) + } + b, err := json.NewFixedKeysObjectBuilder(keys) + if err != nil { + return nil, err + } + e.valueBuilder = b + } + + if err := row.ForEachColumn().Datum(func(d tree.Datum, col cdcevent.ResultColumn) error { + j, err := tree.AsJSON(d, sessiondatapb.DataConversionConfig{}, time.UTC) + if err != nil { + return err + } + return e.valueBuilder.Set(col.Name, j) }); err != nil { return nil, err } - return result, nil -} -// EncodeValue implements the Encoder interface. -func (e *jsonEncoder) EncodeValue( - ctx context.Context, evCtx eventContext, updatedRow cdcevent.Row, prevRow cdcevent.Row, -) ([]byte, error) { - if e.envelopeType == changefeedbase.OptEnvelopeKeyOnly { - return nil, nil + if meta != nil { + if err := e.valueBuilder.Set(jsonMetaSentinel, meta); err != nil { + return nil, err + } } - if updatedRow.IsDeleted() && !canJSONEncodeMetadata(e.envelopeType) { - return nil, nil - } + return e.valueBuilder.Build() +} - after, err := rowAsGoNative(updatedRow) - if err != nil { - return nil, err +func (e *jsonEncoder) initRawEnvelope() error { + // Determine if we need to add crdb meta. + var metaKeys []string + if e.updatedField { + metaKeys = append(metaKeys, "updated") + } + if e.mvccTimestampField { + metaKeys = append(metaKeys, "mvcc_timestamp") + } + if e.keyInValue { + metaKeys = append(metaKeys, "key") + } + if e.topicInValue { + metaKeys = append(metaKeys, "topic") } - before, err := rowAsGoNative(prevRow) - if err != nil { - return nil, err + // Setup builder for crdb meta if needed. + var metaBuilder *json.FixedKeysObjectBuilder + if len(metaKeys) > 0 { + b, err := json.NewFixedKeysObjectBuilder(metaKeys) + if err != nil { + return err + } + metaBuilder = b } - var jsonEntries map[string]interface{} - var meta map[string]interface{} - if canJSONEncodeMetadata(e.envelopeType) { - if e.envelopeType == changefeedbase.OptEnvelopeWrapped { - if after != nil { - jsonEntries = map[string]interface{}{`after`: after} - } else { - jsonEntries = map[string]interface{}{`after`: nil} + e.envelopeEncoder = func(evCtx eventContext, updated, _ cdcevent.Row) (_ json.JSON, err error) { + ve := e.versionEncoder(updated.EventDescriptor) + if len(metaKeys) == 0 { + if updated.IsDeleted() { + return emptyJSONValue, nil } - meta = jsonEntries - } else { - meta = make(map[string]interface{}, 1) - if after != nil { - jsonEntries = after - } else { - jsonEntries = map[string]interface{}{} + return ve.rowAsGoNative(updated, nil) + } + + if e.updatedField { + if err := metaBuilder.Set("updated", json.FromString(evCtx.updated.AsOfSystemTime())); err != nil { + return nil, err } - jsonEntries[jsonMetaSentinel] = meta } - if e.beforeField { - if before != nil { - jsonEntries[`before`] = before - } else { - jsonEntries[`before`] = nil + + if e.mvccTimestampField { + if err := metaBuilder.Set("mvcc_timestamp", json.FromString(evCtx.mvcc.AsOfSystemTime())); err != nil { + return nil, err } } + if e.keyInValue { - keyEntries, err := e.encodeKeyRaw(updatedRow) - if err != nil { + if err := ve.encodeKeyInValue(updated, metaBuilder); err != nil { return nil, err } - jsonEntries[`key`] = keyEntries } + if e.topicInValue { - jsonEntries[`topic`] = evCtx.topic + if err := metaBuilder.Set("topic", json.FromString(evCtx.topic)); err != nil { + return nil, err + } } - } else { - jsonEntries = after + + meta, err := metaBuilder.Build() + if err != nil { + return nil, err + } + return ve.rowAsGoNative(updated, meta) } + return nil +} - // TODO (zinger): Existing behavior special-cases these fields for - // no particular reason. Fold this into the above block. - if e.updatedField || e.mvccTimestampField { - if meta == nil { - if e.envelopeType == changefeedbase.OptEnvelopeWrapped { - meta = jsonEntries +func (e *jsonEncoder) initWrappedEnvelope() error { + keys := []string{"after"} + if e.beforeField { + keys = append(keys, "before") + } + if e.keyInValue { + keys = append(keys, "key") + } + if e.topicInValue { + keys = append(keys, "topic") + } + if e.updatedField { + keys = append(keys, "updated") + } + if e.mvccTimestampField { + keys = append(keys, "mvcc_timestamp") + } + b, err := json.NewFixedKeysObjectBuilder(keys) + if err != nil { + return err + } + + e.envelopeEncoder = func(evCtx eventContext, updated, prev cdcevent.Row) (json.JSON, error) { + ve := e.versionEncoder(updated.EventDescriptor) + after, err := ve.rowAsGoNative(updated, nil) + if err != nil { + return nil, err + } + if err := b.Set("after", after); err != nil { + return nil, err + } + + if e.beforeField { + var before json.JSON + if prev.IsInitialized() && !prev.IsDeleted() { + before, err = e.versionEncoder(prev.EventDescriptor).rowAsGoNative(prev, nil) + if err != nil { + return nil, err + } } else { - meta = make(map[string]interface{}, 1) - jsonEntries[jsonMetaSentinel] = meta + before = json.NullJSONValue + } + + if err := b.Set("before", before); err != nil { + return nil, err + } + } + + if e.keyInValue { + if err := ve.encodeKeyInValue(updated, b); err != nil { + return nil, err } } + + if e.topicInValue { + if err := b.Set("topic", json.FromString(evCtx.topic)); err != nil { + return nil, err + } + } + if e.updatedField { - meta[`updated`] = evCtx.updated.AsOfSystemTime() + if err := b.Set("updated", json.FromString(evCtx.updated.AsOfSystemTime())); err != nil { + return nil, err + } } + if e.mvccTimestampField { - meta[`mvcc_timestamp`] = evCtx.mvcc.AsOfSystemTime() + if err := b.Set("mvcc_timestamp", json.FromString(evCtx.mvcc.AsOfSystemTime())); err != nil { + return nil, err + } } + + return b.Build() } + return nil +} - if metaFields, ok := jsonEntries[jsonMetaSentinel]; ok { - m, ok := metaFields.(map[string]interface{}) - if !ok || len(m) == 0 { - delete(jsonEntries, jsonMetaSentinel) - } +// EncodeValue implements the Encoder interface. +func (e *jsonEncoder) EncodeValue( + ctx context.Context, evCtx eventContext, updatedRow cdcevent.Row, prevRow cdcevent.Row, +) ([]byte, error) { + if e.envelopeType == changefeedbase.OptEnvelopeKeyOnly { + return nil, nil + } + + if updatedRow.IsDeleted() && !canJSONEncodeMetadata(e.envelopeType) { + return nil, nil } - j, err := json.MakeJSON(jsonEntries) + j, err := e.envelopeEncoder(evCtx, updatedRow, prevRow) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage.go b/pkg/ccl/changefeedccl/sink_cloudstorage.go index 60cbef569cb6..c4b91317353d 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage.go @@ -408,7 +408,7 @@ func makeCloudStorageSink( changefeedbase.OptEnvelope, encodingOpts.Envelope) } - if !encodingOpts.KeyInValue { + if encodingOpts.Envelope != changefeedbase.OptEnvelopeBare && !encodingOpts.KeyInValue { return nil, errors.Errorf(`this sink requires the WITH %s option`, changefeedbase.OptKeyInValue) } diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go index fba5f9bb4653..9fe5b318fd30 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go @@ -162,7 +162,7 @@ func TestCloudStorageSink(t *testing.T) { // NB: compression added in single-node subtest. } ts := func(i int64) hlc.Timestamp { return hlc.Timestamp{WallTime: i} } - e, err := makeJSONEncoder(opts, changefeedbase.Targets{}) + e, err := makeJSONEncoder(opts) require.NoError(t, err) clientFactory := blobs.TestBlobServiceClient(settings.ExternalIODir) diff --git a/pkg/ccl/changefeedccl/sink_webhook.go b/pkg/ccl/changefeedccl/sink_webhook.go index 06bb9a9a81f4..c8d7f723e7a5 100644 --- a/pkg/ccl/changefeedccl/sink_webhook.go +++ b/pkg/ccl/changefeedccl/sink_webhook.go @@ -295,7 +295,7 @@ func makeWebhookSink( changefeedbase.OptEnvelope, encodingOpts.Envelope) } - if !encodingOpts.KeyInValue { + if encodingOpts.Envelope != changefeedbase.OptEnvelopeBare && !encodingOpts.KeyInValue { return nil, errors.Errorf(`this sink requires the WITH %s option`, changefeedbase.OptKeyInValue) } diff --git a/pkg/ccl/changefeedccl/sink_webhook_test.go b/pkg/ccl/changefeedccl/sink_webhook_test.go index 470abc49f973..85b8ce91ede7 100644 --- a/pkg/ccl/changefeedccl/sink_webhook_test.go +++ b/pkg/ccl/changefeedccl/sink_webhook_test.go @@ -122,7 +122,7 @@ func testSendAndReceiveRows(t *testing.T, sinkSrc Sink, sinkDest *cdctest.MockWe opts, err := getGenericWebhookSinkOptions().GetEncodingOptions() require.NoError(t, err) - enc, err := makeJSONEncoder(opts, changefeedbase.Targets{}) + enc, err := makeJSONEncoder(opts) require.NoError(t, err) // test a resolved timestamp entry diff --git a/pkg/ccl/changefeedccl/testfeed_test.go b/pkg/ccl/changefeedccl/testfeed_test.go index d6fbe9db9c4c..1d50d913c828 100644 --- a/pkg/ccl/changefeedccl/testfeed_test.go +++ b/pkg/ccl/changefeedccl/testfeed_test.go @@ -896,6 +896,16 @@ func (f *cloudFeedFactory) Feed( if createStmt.SinkURI != nil { return nil, errors.Errorf(`unexpected uri provided: "INTO %s"`, tree.AsString(createStmt.SinkURI)) } + + if createStmt.Select != nil { + createStmt.Options = append(createStmt.Options, + // Normally, cloud storage requires key_in_value; but if we use bare envelope, + // this option is not required. However, we need it to make this + // test feed work -- so, set it. + tree.KVOption{Key: changefeedbase.OptKeyInValue}, + ) + } + feedDir := strconv.Itoa(f.feedIdx) f.feedIdx++ sinkURI := `experimental-nodelocal://0/` + feedDir @@ -921,6 +931,7 @@ func (f *cloudFeedFactory) Feed( ss: ss, seenTrackerMap: make(map[string]struct{}), dir: feedDir, + isBare: createStmt.Select != nil, } if err := f.startFeedJob(c.jobFeed, createStmt.String(), args...); err != nil { return nil, err @@ -941,8 +952,9 @@ type cloudFeedEntry struct { type cloudFeed struct { *jobFeed seenTrackerMap - ss *sinkSynchronizer - dir string + ss *sinkSynchronizer + dir string + isBare bool resolved string rows []cloudFeedEntry @@ -977,23 +989,55 @@ func reformatJSON(j interface{}) ([]byte, error) { return buf.Bytes(), nil } -// extractKeyFromJSONValue extracts the `WITH key_in_value` key from a `WITH -// format=json, envelope=wrapped` value. -func extractKeyFromJSONValue(wrapped []byte) (key []byte, value []byte, _ error) { +func extractFieldFromJSONValue( + fieldName string, isBare bool, wrapped []byte, +) (field gojson.RawMessage, value []byte, err error) { parsed := make(map[string]gojson.RawMessage) + if err := gojson.Unmarshal(wrapped, &parsed); err != nil { return nil, nil, err } - keyParsed := parsed[`key`] - delete(parsed, `key`) - var err error - if key, err = reformatJSON(keyParsed); err != nil { - return nil, nil, err + if isBare { + meta := make(map[string]gojson.RawMessage) + if metaVal, haveMeta := parsed[jsonMetaSentinel]; haveMeta { + if err := gojson.Unmarshal(metaVal, &meta); err != nil { + return nil, nil, err + } + field = meta[fieldName] + delete(meta, fieldName) + if len(meta) == 0 { + delete(parsed, jsonMetaSentinel) + } else { + if metaVal, err = reformatJSON(meta); err != nil { + return nil, nil, err + } + parsed[jsonMetaSentinel] = metaVal + } + } + } else { + field = parsed[fieldName] + delete(parsed, fieldName) } + if value, err = reformatJSON(parsed); err != nil { return nil, nil, err } + return field, value, nil +} + +// extractKeyFromJSONValue extracts the `WITH key_in_value` key from a `WITH +// format=json, envelope=wrapped` value. +func extractKeyFromJSONValue(isBare bool, wrapped []byte) (key []byte, value []byte, err error) { + var keyParsed gojson.RawMessage + keyParsed, value, err = extractFieldFromJSONValue("key", isBare, wrapped) + if err != nil { + return nil, nil, err + } + + if key, err = reformatJSON(keyParsed); err != nil { + return nil, nil, err + } return key, value, nil } @@ -1027,7 +1071,7 @@ func (c *cloudFeed) Next() (*cdctest.TestFeedMessage, error) { // // TODO(dan): Leave the key in the value if the TestFeed user // specifically requested it. - if m.Key, m.Value, err = extractKeyFromJSONValue(m.Value); err != nil { + if m.Key, m.Value, err = extractKeyFromJSONValue(c.isBare, m.Value); err != nil { return nil, err } if isNew := c.markSeen(m); !isNew { @@ -1534,7 +1578,12 @@ func (f *webhookFeedFactory) Feed(create string, args ...interface{}) (cdctest.T // required value createStmt.Options = append(createStmt.Options, tree.KVOption{Key: changefeedbase.OptTopicInValue}) - + if createStmt.Select != nil { + // Normally, webhook requires key_in_value; but if we use bare envelope, + // this option is not required. However, we need it to make this + // test feed work -- so, set it. + createStmt.Options = append(createStmt.Options, tree.KVOption{Key: changefeedbase.OptKeyInValue}) + } var sinkDest *cdctest.MockWebhookSink cert, _, err := cdctest.NewCACertBase64Encoded() @@ -1580,6 +1629,7 @@ func (f *webhookFeedFactory) Feed(create string, args ...interface{}) (cdctest.T jobFeed: newJobFeed(f.jobsTableConn(), wrapSink), seenTrackerMap: make(map[string]struct{}), ss: ss, + isBare: createStmt.Select != nil, mockSink: sinkDest, } if err := f.startFeedJob(c.jobFeed, createStmt.String(), args...); err != nil { @@ -1597,6 +1647,7 @@ type webhookFeed struct { *jobFeed seenTrackerMap ss *sinkSynchronizer + isBare bool mockSink *cdctest.MockWebhookSink } @@ -1619,17 +1670,15 @@ func isResolvedTimestamp(message []byte) (bool, error) { // extractTopicFromJSONValue extracts the `WITH topic_in_value` topic from a `WITH // format=json, envelope=wrapped` value. -func extractTopicFromJSONValue(wrapped []byte) (topic string, value []byte, _ error) { - parsedValue := make(map[string]gojson.RawMessage) - if err := gojson.Unmarshal(wrapped, &parsedValue); err != nil { - return "", nil, err - } - if err := gojson.Unmarshal(parsedValue[`topic`], &topic); err != nil { +func extractTopicFromJSONValue( + isBare bool, wrapped []byte, +) (topic string, value []byte, err error) { + var topicRaw gojson.RawMessage + topicRaw, value, err = extractFieldFromJSONValue("topic", isBare, wrapped) + if err != nil { return "", nil, err } - delete(parsedValue, `topic`) - var err error - if value, err = reformatJSON(parsedValue); err != nil { + if err := gojson.Unmarshal(topicRaw, &topic); err != nil { return "", nil, err } return topic, value, nil @@ -1688,10 +1737,10 @@ func (f *webhookFeed) Next() (*cdctest.TestFeedMessage, error) { if err != nil { return nil, err } - if m.Key, m.Value, err = extractKeyFromJSONValue(wrappedValue); err != nil { + if m.Key, m.Value, err = extractKeyFromJSONValue(f.isBare, wrappedValue); err != nil { return nil, err } - if m.Topic, m.Value, err = extractTopicFromJSONValue(m.Value); err != nil { + if m.Topic, m.Value, err = extractTopicFromJSONValue(f.isBare, m.Value); err != nil { return nil, err } if isNew := f.markSeen(m); !isNew { diff --git a/pkg/util/json/BUILD.bazel b/pkg/util/json/BUILD.bazel index 4074795a5336..3fd8879a5700 100644 --- a/pkg/util/json/BUILD.bazel +++ b/pkg/util/json/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/sql/inverted", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", + "//pkg/util", "//pkg/util/encoding", "//pkg/util/syncutil", "//pkg/util/unique", diff --git a/pkg/util/json/json.go b/pkg/util/json/json.go index cf9927385e43..b283fdf897d7 100644 --- a/pkg/util/json/json.go +++ b/pkg/util/json/json.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/inverted" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" uniq "github.com/cockroachdb/cockroach/pkg/util/unique" "github.com/cockroachdb/errors" @@ -390,6 +391,61 @@ func (b *ObjectBuilder) Build() JSON { return jsonObject(sorter.pairs) } +// FixedKeysObjectBuilder is a JSON object builder that builds +// an object for the specified fixed set of unique keys. +// This object can be reused to build multiple instances of JSON object. +type FixedKeysObjectBuilder struct { + pairs []jsonKeyValuePair + keyOrd map[string]int + updated util.FastIntSet +} + +// NewFixedKeysObjectBuilder creates JSON object builder for the specified +// set of fixed, unique keys. +func NewFixedKeysObjectBuilder(keys []string) (*FixedKeysObjectBuilder, error) { + sort.Strings(keys) + pairs := make([]jsonKeyValuePair, len(keys)) + keyOrd := make(map[string]int, len(keys)) + + for i, k := range keys { + if _, exists := keyOrd[k]; exists { + return nil, errors.AssertionFailedf("expected unique keys, found %v", keys) + } + pairs[i] = jsonKeyValuePair{k: jsonString(keys[i])} + keyOrd[k] = i + } + + return &FixedKeysObjectBuilder{ + pairs: pairs, + keyOrd: keyOrd, + }, nil +} + +// Set sets the value for the specified key. +// All previously configured keys must be set before calling Build. +func (b *FixedKeysObjectBuilder) Set(k string, v JSON) error { + ord, ok := b.keyOrd[k] + if !ok { + return errors.AssertionFailedf("unknown key %s", k) + } + + b.pairs[ord].v = v + b.updated.Add(ord) + return nil +} + +// Build builds JSON object. +func (b *FixedKeysObjectBuilder) Build() (JSON, error) { + if b.updated.Len() != len(b.pairs) { + return nil, errors.AssertionFailedf( + "expected all %d keys to be updated, %d updated", + len(b.pairs), b.updated.Len()) + } + b.updated = util.FastIntSet{} + // Must copy b.pairs in case builder is reused. + return jsonObject(append([]jsonKeyValuePair(nil), b.pairs...)), nil +} + // pairSorter sorts and uniqueifies JSON pairs. In order to keep // the last one for pairs with the same key while sort.Sort is // not stable, pairSorter uses []int orders to maintain order and diff --git a/pkg/util/json/json_test.go b/pkg/util/json/json_test.go index bf58df437f52..b2adeb3d5549 100644 --- a/pkg/util/json/json_test.go +++ b/pkg/util/json/json_test.go @@ -503,6 +503,17 @@ func TestNewObjectBuilderWithCounter(t *testing.T) { } func TestBuildJSONObject(t *testing.T) { + checkJSONObjectsEqual := func(t *testing.T, expected, found JSON) { + t.Helper() + c, err := found.Compare(expected) + if err != nil { + t.Fatal(err) + } + if c != 0 { + t.Fatalf("expected %v to equal %v", found, expectError) + } + } + testCases := []struct { input []string }{ @@ -525,17 +536,52 @@ func TestBuildJSONObject(t *testing.T) { t.Fatal(err) } result := b.Build() - c, err := result.Compare(expectedResult) - if err != nil { - t.Fatal(err) - } - if c != 0 { - t.Fatalf("expected %v to equal %v", result, expectedResult) - } + checkJSONObjectsEqual(t, expectedResult, result) + + t.Run("fixedKeys", func(t *testing.T) { + uniqueKeys := func() (keys []string) { + for k := range m { + keys = append(keys, k) + } + return keys + }() + + fkb, err := NewFixedKeysObjectBuilder(uniqueKeys) + if err != nil { + t.Fatal(err) + } + for i := 0; i < 5; i++ { + for k, v := range m { + if err := fkb.Set(k, v.(JSON)); err != nil { + t.Fatal(err) + } + } + result, err := fkb.Build() + if err != nil { + t.Fatal(err) + } + checkJSONObjectsEqual(t, expectedResult, result) + } + }) }) } } +func TestBuildFixedKeysJSONObjectErrors(t *testing.T) { + t.Run("require_unique_keys", func(t *testing.T) { + _, err := NewFixedKeysObjectBuilder([]string{"a", "b", "c", "a", "d"}) + require.Error(t, err) + }) + t.Run("requires_all_keys_set", func(t *testing.T) { + b, err := NewFixedKeysObjectBuilder([]string{"a", "b", "c"}) + require.NoError(t, err) + require.NoError(t, b.Set("a", jsonNull{})) + require.NoError(t, b.Set("b", jsonNull{})) + _, err = b.Build() + require.Error(t, err) + }) +} + func jsonTestShorthand(s string) JSON { j, err := ParseJSON(s) if err != nil { From 0b8671f30f5fe947d83a09bd5af88699b9292bcb Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Mon, 22 Aug 2022 18:39:18 -0400 Subject: [PATCH 05/13] span: make span.Frontier thread safe Make `span.Frontier` thread safe by default. Release note: None --- pkg/util/span/BUILD.bazel | 1 + pkg/util/span/frontier.go | 22 ++++++++++++++++++++-- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/pkg/util/span/BUILD.bazel b/pkg/util/span/BUILD.bazel index 1d616a8fe4e2..819fac7615a2 100644 --- a/pkg/util/span/BUILD.bazel +++ b/pkg/util/span/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/roachpb", "//pkg/util/hlc", "//pkg/util/interval", + "//pkg/util/syncutil", ], ) diff --git a/pkg/util/span/frontier.go b/pkg/util/span/frontier.go index 7dc38531d2fa..7d2b7b2b9241 100644 --- a/pkg/util/span/frontier.go +++ b/pkg/util/span/frontier.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) // frontierEntry represents a timestamped span. It is used as the nodes in both @@ -91,6 +92,7 @@ func (h *frontierHeap) Pop() interface{} { // Frontier tracks the minimum timestamp of a set of spans. type Frontier struct { + syncutil.Mutex // tree contains `*frontierEntry` items for the entire current tracked // span set. Any tracked spans that have never been `Forward`ed will have a // zero timestamp. If any entries needed to be split along a tracking @@ -153,6 +155,12 @@ func (f *Frontier) AddSpansAt(startAt hlc.Timestamp, spans ...roachpb.Span) erro // Frontier returns the minimum timestamp being tracked. func (f *Frontier) Frontier() hlc.Timestamp { + f.Lock() + defer f.Unlock() + return f.frontierLocked() +} + +func (f *Frontier) frontierLocked() hlc.Timestamp { if f.minHeap.Len() == 0 { return hlc.Timestamp{} } @@ -161,6 +169,8 @@ func (f *Frontier) Frontier() hlc.Timestamp { // PeekFrontierSpan returns one of the spans at the Frontier. func (f *Frontier) PeekFrontierSpan() roachpb.Span { + f.Lock() + defer f.Unlock() if f.minHeap.Len() == 0 { return roachpb.Span{} } @@ -176,11 +186,13 @@ func (f *Frontier) PeekFrontierSpan() roachpb.Span { // set boundary). Similarly, an entry created by a previous Forward may be // partially overlapped and have to be split into two entries. func (f *Frontier) Forward(span roachpb.Span, ts hlc.Timestamp) (bool, error) { - prevFrontier := f.Frontier() + f.Lock() + defer f.Unlock() + prevFrontier := f.frontierLocked() if err := f.insert(span, ts); err != nil { return false, err } - return prevFrontier.Less(f.Frontier()), nil + return prevFrontier.Less(f.frontierLocked()), nil } // extendRangeToTheLeft extends the range to the left of the range, provided those @@ -390,6 +402,8 @@ type Operation func(roachpb.Span, hlc.Timestamp) (done OpResult) // Entries invokes the given callback with the current timestamp for each // component span in the tracked span set. func (f *Frontier) Entries(fn Operation) { + f.Lock() + defer f.Unlock() f.tree.Do(func(i interval.Interface) bool { spe := i.(*frontierEntry) return fn(spe.span, spe.ts).asBool() @@ -415,6 +429,8 @@ func (f *Frontier) Entries(fn Operation) { // Note: neither [a-b) nor [m, q) will be emitted since they fall outside the spans // tracked by this frontier. func (f *Frontier) SpanEntries(span roachpb.Span, op Operation) { + f.Lock() + defer f.Unlock() todoRange := span.AsRange() f.tree.DoMatching(func(i interval.Interface) bool { @@ -440,6 +456,8 @@ func (f *Frontier) SpanEntries(span roachpb.Span, op Operation) { // String implements Stringer. func (f *Frontier) String() string { + f.Lock() + defer f.Unlock() var buf strings.Builder f.tree.Do(func(i interval.Interface) bool { if buf.Len() != 0 { From 2a3cfd835b4a4daf6f7b9f54e902dd7575aec020 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Tue, 23 Aug 2022 08:58:44 -0400 Subject: [PATCH 06/13] changefeedccl: parallelize event consumption Previously, KV events were consumed and processed by changefeed aggregator using a single, synchronous Go routine. This PR makes it possible to run up to `changefeed.event_consumer_workers` consumers to consume events concurrently. The cluster setting `changefeed.event_consumer_worker_queue_size` is added to help control the number of concurrent events we can have in flight. Specifying `changefeed.event_consumer_workers=0` keeps existing single threaded implementation. Release note (enterprise change): This change adds the cluster setting `changefeed.event_consumer_workers` which allows changefeeds to process events concurrently. --- .../settings/settings-for-tenants.txt | 2 + docs/generated/settings/settings.html | 2 + pkg/ccl/changefeedccl/BUILD.bazel | 1 + .../changefeedccl/changefeed_processors.go | 60 ++-- .../changefeedccl/changefeedbase/BUILD.bazel | 1 + .../changefeedccl/changefeedbase/settings.go | 21 ++ pkg/ccl/changefeedccl/encoder_test.go | 9 + pkg/ccl/changefeedccl/event_processing.go | 325 +++++++++++++++++- .../changefeedccl/event_processing_test.go | 79 +++++ pkg/ccl/changefeedccl/metrics.go | 64 ++-- pkg/ccl/changefeedccl/sink.go | 45 +++ pkg/ts/catalog/chart_catalog.go | 18 + 12 files changed, 569 insertions(+), 58 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 9f11b0215de1..a77318cad8ac 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -10,6 +10,8 @@ bulkio.backup.file_size byte size 128 MiB target size for individual data files bulkio.backup.read_timeout duration 5m0s amount of time after which a read attempt is considered timed out, which causes the backup to fail bulkio.backup.read_with_priority_after duration 1m0s amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads bulkio.stream_ingestion.minimum_flush_interval duration 5s the minimum timestamp between flushes; flushes may still occur if internal buffers fill up +changefeed.event_consumer_worker_queue_size integer 16 if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of eventswhich a worker can buffer +changefeed.event_consumer_workers integer 8 the number of workers to use when processing events; 0 or 1 disables changefeed.node_throttle_config string specifies node level throttling configuration for all changefeeeds changefeed.schema_feed.read_with_priority_after duration 1m0s retry with high priority if we were not able to read descriptors for too long; 0 disables cloudstorage.azure.concurrent_upload_buffers integer 1 controls the number of concurrent buffers that will be used by the Azure client when uploading chunks.Each buffer can buffer up to cloudstorage.write_chunk.size of memory during an upload diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 0ddb27c443f9..392941a74a35 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -16,6 +16,8 @@ bulkio.backup.read_timeoutduration5m0samount of time after which a read attempt is considered timed out, which causes the backup to fail bulkio.backup.read_with_priority_afterduration1m0samount of time since the read-as-of time above which a BACKUP should use priority when retrying reads bulkio.stream_ingestion.minimum_flush_intervalduration5sthe minimum timestamp between flushes; flushes may still occur if internal buffers fill up +changefeed.event_consumer_worker_queue_sizeinteger16if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of eventswhich a worker can buffer +changefeed.event_consumer_workersinteger8the number of workers to use when processing events; 0 or 1 disables changefeed.node_throttle_configstringspecifies node level throttling configuration for all changefeeeds changefeed.schema_feed.read_with_priority_afterduration1m0sretry with high priority if we were not able to read descriptors for too long; 0 disables cloudstorage.azure.concurrent_upload_buffersinteger1controls the number of concurrent buffers that will be used by the Azure client when uploading chunks.Each buffer can buffer up to cloudstorage.write_chunk.size of memory during an upload diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index 66e3f4a84ae5..c3dd3c8c3e02 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -227,6 +227,7 @@ go_test( "//pkg/sql/parser", "//pkg/sql/randgen", "//pkg/sql/rowenc", + "//pkg/sql/rowenc/keyside", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/sem/volatility", diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index c25663ffcaeb..d447a7a6d523 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -60,8 +60,6 @@ type changeAggregator struct { kvFeedDoneCh chan struct{} kvFeedMemMon *mon.BytesMonitor - // encoder is the Encoder to use for key and value serialization. - encoder Encoder // sink is the Sink to write rows to. Resolved timestamps are never written // by changeAggregator. sink EventSink @@ -80,7 +78,7 @@ type changeAggregator struct { // eventProducer produces the next event from the kv feed. eventProducer kvevent.Reader // eventConsumer consumes the event. - eventConsumer *kvEventToRowConsumer + eventConsumer eventConsumer // lastFlush and flushFrequency keep track of the flush frequency. lastFlush time.Time @@ -93,7 +91,6 @@ type changeAggregator struct { metrics *Metrics sliMetrics *sliMetrics knobs TestingKnobs - topicNamer *TopicNamer } type timestampLowerBoundOracle interface { @@ -162,22 +159,6 @@ func newChangeAggregatorProcessor( opts := changefeedbase.MakeStatementOptions(ca.spec.Feed.Opts) - var err error - encodingOpts, err := opts.GetEncodingOptions() - if err != nil { - return nil, err - } - if ca.encoder, err = getEncoder(encodingOpts, AllTargets(ca.spec.Feed)); err != nil { - return nil, err - } - - if encodingOpts.TopicInValue { - ca.topicNamer, err = MakeTopicNamer(AllTargets(ca.spec.Feed)) - if err != nil { - return nil, err - } - } - // MinCheckpointFrequency controls how frequently the changeAggregator flushes the sink // and checkpoints the local frontier to changeFrontier. It is used as a rough // approximation of how latency-sensitive the changefeed user is. For a high latency @@ -225,7 +206,6 @@ func (ca *changeAggregator) Start(ctx context.Context) { feed := makeChangefeedConfigFromJobDetails(ca.spec.Feed) - endTime := feed.EndTime opts := feed.Opts if err != nil { @@ -295,7 +275,7 @@ func (ca *changeAggregator) Start(ctx context.Context) { kvFeedHighWater = ca.spec.Feed.StatementTime } - ca.eventProducer, err = ca.startKVFeed(ctx, spans, kvFeedHighWater, needsInitialScan, endTime) + ca.eventProducer, err = ca.startKVFeed(ctx, spans, kvFeedHighWater, needsInitialScan, feed) if err != nil { // Early abort in the case that there is an error creating the sink. ca.MoveToDraining(err) @@ -303,9 +283,9 @@ func (ca *changeAggregator) Start(ctx context.Context) { return } - ca.eventConsumer, err = newKVEventToRowConsumer( - ctx, ca.flowCtx.Cfg, ca.flowCtx.EvalCtx, ca.frontier.SpanFrontier(), kvFeedHighWater, - ca.sink, ca.encoder, feed, ca.spec.Select, ca.knobs, ca.topicNamer) + ca.eventConsumer, ca.sink, err = newEventConsumer( + ctx, ca.flowCtx, feed, ca.frontier.SpanFrontier(), kvFeedHighWater, + ca.sink, feed, ca.spec.Select, ca.knobs, ca.metrics, ca.isSinkless()) if err != nil { // Early abort in the case that there is an error setting up the consumption. @@ -320,7 +300,7 @@ func (ca *changeAggregator) startKVFeed( spans []roachpb.Span, initialHighWater hlc.Timestamp, needsInitialScan bool, - endTime hlc.Timestamp, + config ChangefeedConfig, ) (kvevent.Reader, error) { cfg := ca.flowCtx.Cfg buf := kvevent.NewThrottlingBuffer( @@ -329,7 +309,7 @@ func (ca *changeAggregator) startKVFeed( // KVFeed takes ownership of the kvevent.Writer portion of the buffer, while // we return the kvevent.Reader part to the caller. - kvfeedCfg, err := ca.makeKVFeedCfg(ctx, spans, buf, initialHighWater, needsInitialScan, endTime) + kvfeedCfg, err := ca.makeKVFeedCfg(ctx, config, spans, buf, initialHighWater, needsInitialScan) if err != nil { return nil, err } @@ -359,28 +339,27 @@ func (ca *changeAggregator) startKVFeed( func (ca *changeAggregator) makeKVFeedCfg( ctx context.Context, + config ChangefeedConfig, spans []roachpb.Span, buf kvevent.Writer, initialHighWater hlc.Timestamp, needsInitialScan bool, - endTime hlc.Timestamp, ) (kvfeed.Config, error) { - opts := changefeedbase.MakeStatementOptions(ca.spec.Feed.Opts) - schemaChange, err := opts.GetSchemaChangeHandlingOptions() + schemaChange, err := config.Opts.GetSchemaChangeHandlingOptions() if err != nil { return kvfeed.Config{}, err } - filters := opts.GetFilters() + filters := config.Opts.GetFilters() cfg := ca.flowCtx.Cfg - initialScanOnly := endTime.EqOrdering(initialHighWater) + initialScanOnly := config.EndTime.EqOrdering(initialHighWater) var sf schemafeed.SchemaFeed if schemaChange.Policy == changefeedbase.OptSchemaChangePolicyIgnore || initialScanOnly { sf = schemafeed.DoNothingSchemaFeed } else { sf = schemafeed.New(ctx, cfg, schemaChange.EventClass, AllTargets(ca.spec.Feed), - initialHighWater, &ca.metrics.SchemaFeedMetrics, opts.GetCanHandle()) + initialHighWater, &ca.metrics.SchemaFeedMetrics, config.Opts.GetCanHandle()) } return kvfeed.Config{ @@ -399,7 +378,7 @@ func (ca *changeAggregator) makeKVFeedCfg( OnBackfillRangeCallback: ca.sliMetrics.getBackfillRangeCallback(), MM: ca.kvFeedMemMon, InitialHighWater: initialHighWater, - EndTime: endTime, + EndTime: config.EndTime, WithDiff: filters.WithDiff, NeedsInitialScan: needsInitialScan, SchemaChangeEvents: schemaChange.EventClass, @@ -467,12 +446,17 @@ func (ca *changeAggregator) close() { if ca.kvFeedDoneCh != nil { <-ca.kvFeedDoneCh } + if ca.eventConsumer != nil { + if err := ca.eventConsumer.Close(); err != nil { + log.Warningf(ca.Ctx, "error closing event consumer: %s", err) + } + } + if ca.sink != nil { if err := ca.sink.Close(); err != nil { log.Warningf(ca.Ctx, `error closing sink. goroutines may have leaked: %v`, err) } } - ca.memAcc.Close(ca.Ctx) if ca.kvFeedMemMon != nil { ca.kvFeedMemMon.Stop(ca.Ctx) @@ -584,7 +568,7 @@ func (ca *changeAggregator) noteResolvedSpan(resolved jobspb.ResolvedSpan) error // flushFrontier flushes sink and emits resolved timestamp if needed. func (ca *changeAggregator) flushFrontier() error { - // Make sure to flush the sink before forwarding resolved spans, + // Make sure to the sink before forwarding resolved spans, // otherwise, we could lose buffered messages and violate the // at-least-once guarantee. This is also true for checkpointing the // resolved spans in the job progress. @@ -639,6 +623,10 @@ func (ca *changeAggregator) ConsumerClosed() { ca.close() } +func (ca *changeAggregator) isSinkless() bool { + return ca.spec.JobID == 0 +} + const ( emitAllResolved = 0 emitNoResolved = -1 diff --git a/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel b/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel index c5173f5e92e1..4bb125e722a1 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel +++ b/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/sql", "//pkg/sql/catalog/descpb", "//pkg/sql/flowinfra", + "//pkg/util", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/ccl/changefeedccl/changefeedbase/settings.go b/pkg/ccl/changefeedccl/changefeedbase/settings.go index fb9ae3a8bba2..a66f1a136989 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/settings.go +++ b/pkg/ccl/changefeedccl/changefeedbase/settings.go @@ -13,6 +13,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/errors" ) @@ -231,3 +232,23 @@ var UseMuxRangeFeed = settings.RegisterBoolSetting( "if true, changefeed uses multiplexing rangefeed RPC", false, ) + +// EventConsumerWorkers specifies the maximum number of workers to use when +// processing events. +var EventConsumerWorkers = settings.RegisterIntSetting( + settings.TenantWritable, + "changefeed.event_consumer_workers", + "the number of workers to use when processing events; 0 or 1 disables", + int64(util.ConstantWithMetamorphicTestRange("changefeed.consumer_max_workers", 8, 0, 32)), + settings.NonNegativeInt, +).WithPublic() + +// EventConsumerWorkerQueueSize specifies the maximum number of events a worker buffer. +var EventConsumerWorkerQueueSize = settings.RegisterIntSetting( + settings.TenantWritable, + "changefeed.event_consumer_worker_queue_size", + "if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events"+ + "which a worker can buffer", + int64(util.ConstantWithMetamorphicTestRange("changefeed.event_consumer_worker_queue_size", 16, 0, 16)), + settings.NonNegativeInt, +).WithPublic() diff --git a/pkg/ccl/changefeedccl/encoder_test.go b/pkg/ccl/changefeedccl/encoder_test.go index 16f29baf169c..983a1b47a731 100644 --- a/pkg/ccl/changefeedccl/encoder_test.go +++ b/pkg/ccl/changefeedccl/encoder_test.go @@ -600,6 +600,15 @@ func TestAvroSchemaNaming(t *testing.T) { testFn := func(t *testing.T, s TestServer, f cdctest.TestFeedFactory) { sqlDB := sqlutils.MakeSQLRunner(s.DB) + + // The expected results depend on caching in the avro encoder. + // With multiple workers, there are multiple encoders which each + // maintain their own caches. Depending on the number of + // workers, the results below may change, so disable parallel workers + // here for simplicity. + changefeedbase.EventConsumerWorkers.Override( + context.Background(), &s.Server.ClusterSettings().SV, 0) + sqlDB.Exec(t, `CREATE DATABASE movr`) sqlDB.Exec(t, `CREATE TABLE movr.drivers (id INT PRIMARY KEY, name STRING)`) sqlDB.Exec(t, diff --git a/pkg/ccl/changefeedccl/event_processing.go b/pkg/ccl/changefeedccl/event_processing.go index 8114edf1df13..74b3ba6129ed 100644 --- a/pkg/ccl/changefeedccl/event_processing.go +++ b/pkg/ccl/changefeedccl/event_processing.go @@ -10,18 +10,24 @@ package changefeedccl import ( "context" + "hash" + "hash/crc32" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdceval" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" + "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/span" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) @@ -32,8 +38,16 @@ type eventContext struct { topic string } +type eventConsumer interface { + ConsumeEvent(ctx context.Context, ev kvevent.Event) error + Close() error + Flush(ctx context.Context) error +} + +type frontier interface{ Frontier() hlc.Timestamp } + type kvEventToRowConsumer struct { - frontier *span.Frontier + frontier encoder Encoder scratch bufalloc.ByteAllocator sink EventSink @@ -48,11 +62,87 @@ type kvEventToRowConsumer struct { topicNamer *TopicNamer } +func newEventConsumer( + ctx context.Context, + flowCtx *execinfra.FlowCtx, + feed ChangefeedConfig, + spanFrontier *span.Frontier, + cursor hlc.Timestamp, + sink EventSink, + details ChangefeedConfig, + expr execinfrapb.Expression, + knobs TestingKnobs, + metrics *Metrics, + isSinkless bool, +) (eventConsumer, EventSink, error) { + cfg := flowCtx.Cfg + evalCtx := flowCtx.EvalCtx + + makeConsumer := func(s EventSink, frontier frontier) (eventConsumer, error) { + var err error + encodingOpts, err := feed.Opts.GetEncodingOptions() + if err != nil { + return nil, err + } + encoder, err := getEncoder(encodingOpts, feed.Targets) + if err != nil { + return nil, err + } + + var topicNamer *TopicNamer + if encodingOpts.TopicInValue { + topicNamer, err = MakeTopicNamer(feed.Targets) + if err != nil { + return nil, err + } + } + + return newKVEventToRowConsumer(ctx, cfg, evalCtx, frontier, cursor, s, + encoder, details, expr, knobs, topicNamer) + } + + // TODO (jayshrivastava) enable parallel consumers for sinkless changefeeds + numWorkers := changefeedbase.EventConsumerWorkers.Get(&cfg.Settings.SV) + if numWorkers <= 1 || isSinkless { + c, err := makeConsumer(sink, spanFrontier) + if err != nil { + return nil, nil, err + } + return c, sink, err + } + + c := ¶llelEventConsumer{ + g: ctxgroup.WithContext(ctx), + hasher: makeHasher(), + metrics: metrics, + termCh: make(chan struct{}), + flushCh: make(chan struct{}, 1), + doneCh: make(chan struct{}), + numWorkers: numWorkers, + workerCh: make([]chan kvevent.Event, numWorkers), + workerChSize: changefeedbase.EventConsumerWorkerQueueSize.Get(&cfg.Settings.SV), + spanFrontier: spanFrontier, + } + ss := &safeSink{wrapped: sink, beforeFlush: c.Flush} + c.makeConsumer = func() (eventConsumer, error) { + return makeConsumer(ss, c) + } + + if err := c.startWorkers(); err != nil { + return nil, nil, err + } + return c, ss, nil +} + +func makeHasher() hash.Hash32 { + return crc32.New(crc32.MakeTable(crc32.IEEE)) +} + func newKVEventToRowConsumer( ctx context.Context, cfg *execinfra.ServerConfig, evalCtx *eval.Context, - frontier *span.Frontier, + frontier frontier, cursor hlc.Timestamp, sink EventSink, encoder Encoder, @@ -240,3 +330,234 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even } return nil } + +// Close is a noop for the kvEventToRowConsumer because it +// has no goroutines in flight. +func (c *kvEventToRowConsumer) Close() error { + return nil +} + +// Flush is a noop for the kvEventToRowConsumer because it does not buffer any events. +func (c *kvEventToRowConsumer) Flush(ctx context.Context) error { + return nil +} + +type parallelEventConsumer struct { + // g is a group used to manage worker goroutines. + g ctxgroup.Group + + // hasher is used to shard keys into worker queues. + hasher hash.Hash32 + + metrics *Metrics + + // doneCh is used to shut down all workers when + // parallelEventConsumer.Close() is called. + doneCh chan struct{} + + // makeConsumer creates a single-threaded consumer + // which encodes and emits events. + makeConsumer func() (eventConsumer, error) + + // numWorkers is the number of worker threads. + numWorkers int64 + // workerCh stores the event buffer for each worker. + // It is a fixed size array with length numWorkers. + workerCh []chan kvevent.Event + // workerChSize is the maximum number of events a worker can buffer. + workerChSize int64 + + // spanFrontier stores the frontier for the aggregator + // that spawned this event consumer. + spanFrontier *span.Frontier + + // termErr and termCh are used to save the first error that occurs + // in any worker and signal all workers to stop. + // + // inFlight, waiting, and flushCh are used to flush the sink. + // + // flushFrontier tracks the local frontier. It is set to the + // spanFrontier upon flushing. This guarantees that workers are + // not buffering events which have timestamps lower than the frontier. + mu struct { + syncutil.Mutex + termErr error + + inFlight int + waiting bool + flushFrontier hlc.Timestamp + } + termCh chan struct{} + flushCh chan struct{} +} + +var _ eventConsumer = (*parallelEventConsumer)(nil) + +func (c *parallelEventConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Event) error { + startTime := timeutil.Now().UnixNano() + defer func() { + time := timeutil.Now().UnixNano() + c.metrics.ParallelConsumerConsumeNanos.Inc(time - startTime) + }() + + bucket := c.getBucketForEvent(ev) + + select { + case <-ctx.Done(): + return ctx.Err() + case <-c.termCh: + c.mu.Lock() + defer c.mu.Unlock() + return c.mu.termErr + case c.workerCh[bucket] <- ev: + c.incInFlight() + return nil + } +} + +// getBucketForEvent returns a deterministic value between [0, +// parallelEventConsumer.numWorkers). +// +// This is used to ensure events of the same key get sent to the same worker. +// Events of the same key are sent to the same worker so per-key ordering is +// maintained. +func (c *parallelEventConsumer) getBucketForEvent(ev kvevent.Event) int64 { + c.hasher.Reset() + c.hasher.Write(ev.KV().Key) + return int64(c.hasher.Sum32()) % c.numWorkers +} + +func (c *parallelEventConsumer) startWorkers() error { + // Create the consumers in a separate loop so that returning + // in case of an error is simple and does not require + // shutting down goroutines. + consumers := make([]eventConsumer, c.numWorkers) + for i := int64(0); i < c.numWorkers; i++ { + consumer, err := c.makeConsumer() + if err != nil { + return err + } + consumers[i] = consumer + } + + for i := int64(0); i < c.numWorkers; i++ { + c.workerCh[i] = make(chan kvevent.Event, c.workerChSize) + + // c.g.GoCtx requires a func(context.Context), so + // a closure is used to pass the consumer and id to + // the worker. + id := i + consumer := consumers[i] + workerClosure := func(ctx2 context.Context) error { + return c.workerLoop(ctx2, consumer, id) + } + c.g.GoCtx(workerClosure) + } + return nil +} + +func (c *parallelEventConsumer) workerLoop( + ctx context.Context, consumer eventConsumer, id int64, +) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-c.doneCh: + return nil + case <-c.termCh: + c.mu.Lock() + defer c.mu.Unlock() + return c.mu.termErr + case e := <-c.workerCh[id]: + if err := consumer.ConsumeEvent(ctx, e); err != nil { + return c.setWorkerError(err) + } + c.decInFlight() + } + } +} + +func (c *parallelEventConsumer) incInFlight() { + c.mu.Lock() + c.mu.inFlight++ + c.mu.Unlock() + c.metrics.ParallelConsumerInFlightEvents.Inc(1) +} + +func (c *parallelEventConsumer) decInFlight() { + c.mu.Lock() + c.mu.inFlight-- + c.metrics.ParallelConsumerInFlightEvents.Dec(1) + notifyFlush := c.mu.waiting && c.mu.inFlight == 0 + c.mu.Unlock() + + // If someone is waiting on a flush, signal to them + // that there are no more events. + if notifyFlush { + c.flushCh <- struct{}{} + } +} + +func (c *parallelEventConsumer) setWorkerError(err error) error { + c.mu.Lock() + defer c.mu.Unlock() + if c.mu.termErr == nil { + c.mu.termErr = err + close(c.termCh) + } + + return err +} + +// Flush flushes the consumer by blocking until all events are consumed, +// or until there is an error. +func (c *parallelEventConsumer) Flush(ctx context.Context) error { + startTime := timeutil.Now().UnixNano() + defer func() { + time := timeutil.Now().UnixNano() + c.metrics.ParallelConsumerFlushNanos.Inc(time - startTime) + }() + + needFlush := func() bool { + c.mu.Lock() + defer c.mu.Unlock() + if c.mu.inFlight > 0 { + c.mu.waiting = true + } + return c.mu.waiting + } + + if !needFlush() { + return nil + } + + select { + case <-ctx.Done(): + return ctx.Err() + case <-c.termCh: + c.mu.Lock() + defer c.mu.Unlock() + return c.mu.termErr + case <-c.flushCh: + c.mu.Lock() + c.mu.waiting = false + c.mu.flushFrontier = c.spanFrontier.Frontier() + c.mu.Unlock() + return nil + } +} + +func (c *parallelEventConsumer) Frontier() hlc.Timestamp { + c.mu.Lock() + defer c.mu.Unlock() + return c.mu.flushFrontier +} + +func (c *parallelEventConsumer) Close() error { + // Signal the done channel and wait for all workers to finish. + // If an error occurred, at least one worker will return an error, so + // it will be returned by c.g.Wait(). + close(c.doneCh) + return c.g.Wait() +} diff --git a/pkg/ccl/changefeedccl/event_processing_test.go b/pkg/ccl/changefeedccl/event_processing_test.go index b61ca883029b..63b66f3bcee7 100644 --- a/pkg/ccl/changefeedccl/event_processing_test.go +++ b/pkg/ccl/changefeedccl/event_processing_test.go @@ -9,12 +9,21 @@ package changefeedccl import ( + "math/rand" "testing" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" + "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -166,3 +175,73 @@ func TestTopicForEvent(t *testing.T) { }) } } + +// TestShardingByKey tests that the sharding function is deterministic and +// maps keys within a range of [0, numWorkers). +func TestShardingByKey(t *testing.T) { + defer leaktest.AfterTest(t)() + + rng, _ := randutil.NewTestRand() + p := parallelEventConsumer{numWorkers: 16, hasher: makeHasher()} + + poolSize := 10000 + eventPool := make([]kvevent.Event, poolSize) + for i := 0; i < poolSize; i++ { + eventPool[i] = makeKVEventKeyOnly(rng, 128) + } + + seen := map[string]int64{} + for i := 0; i < poolSize; i++ { + ev := eventPool[i] + + b := p.getBucketForEvent(ev) + key := getKeyFromKVEvent(ev) + + assert.True(t, 0 <= b && b < 16) + + if bucket, ok := seen[key]; ok { + assert.Equal(t, bucket, b) + } + seen[key] = b + } +} + +func BenchmarkShardingByKey(b *testing.B) { + rng, _ := randutil.NewTestRand() + p := parallelEventConsumer{numWorkers: 32, hasher: makeHasher()} + + poolSize := 1000 + eventPool := make([]kvevent.Event, poolSize) + for i := 0; i < poolSize; i++ { + eventPool[i] = makeKVEventKeyOnly(rng, 2<<31-1) + } + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + p.getBucketForEvent(eventPool[rng.Intn(len(eventPool))]) + } +} + +func makeKVEventKeyOnly(rng *rand.Rand, upper int) kvevent.Event { + testTableID := 42 + + key, err := keyside.Encode( + keys.SystemSQLCodec.TablePrefix(uint32(testTableID)), + tree.NewDInt(tree.DInt(rng.Intn(upper))), + encoding.Ascending, + ) + if err != nil { + panic(err) + } + + return kvevent.MakeKVEvent(&roachpb.RangeFeedEvent{ + Val: &roachpb.RangeFeedValue{ + Key: key, + }, + }) +} + +func getKeyFromKVEvent(ev kvevent.Event) string { + return ev.KV().Key.String() +} diff --git a/pkg/ccl/changefeedccl/metrics.go b/pkg/ccl/changefeedccl/metrics.go index 08f60004c34e..cd625b0f60fd 100644 --- a/pkg/ccl/changefeedccl/metrics.go +++ b/pkg/ccl/changefeedccl/metrics.go @@ -359,6 +359,24 @@ var ( Measurement: "Replans", Unit: metric.Unit_COUNT, } + metaChangefeedEventConsumerFlushNanos = metric.Metadata{ + Name: "changefeed.nprocs_flush_nanos", + Help: "Total time spent idle waiting for the parallel consumer to flush", + Measurement: "Nanoseconds", + Unit: metric.Unit_NANOSECONDS, + } + metaChangefeedEventConsumerConsumeNanos = metric.Metadata{ + Name: "changefeed.nprocs_consume_event_nanos", + Help: "Total time spent waiting to add an event to the parallel consumer", + Measurement: "Nanoseconds", + Unit: metric.Unit_NANOSECONDS, + } + metaChangefeedEventConsumerInFlightEvents = metric.Metadata{ + Name: "changefeed.nprocs_in_flight_count", + Help: "Number of buffered events in the parallel consumer", + Measurement: "Count of Events", + Unit: metric.Unit_COUNT, + } ) func newAggregateMetrics(histogramWindow time.Duration) *AggMetrics { @@ -548,16 +566,19 @@ func (a *AggMetrics) getOrCreateScope(scope string) (*sliMetrics, error) { // Metrics are for production monitoring of changefeeds. type Metrics struct { - AggMetrics *AggMetrics - KVFeedMetrics kvevent.Metrics - SchemaFeedMetrics schemafeed.Metrics - Failures *metric.Counter - ResolvedMessages *metric.Counter - QueueTimeNanos *metric.Counter - CheckpointHistNanos *metric.Histogram - FrontierUpdates *metric.Counter - ThrottleMetrics cdcutils.Metrics - ReplanCount *metric.Counter + AggMetrics *AggMetrics + KVFeedMetrics kvevent.Metrics + SchemaFeedMetrics schemafeed.Metrics + Failures *metric.Counter + ResolvedMessages *metric.Counter + QueueTimeNanos *metric.Counter + CheckpointHistNanos *metric.Histogram + FrontierUpdates *metric.Counter + ThrottleMetrics cdcutils.Metrics + ReplanCount *metric.Counter + ParallelConsumerFlushNanos *metric.Counter + ParallelConsumerConsumeNanos *metric.Counter + ParallelConsumerInFlightEvents *metric.Gauge mu struct { syncutil.Mutex @@ -578,16 +599,19 @@ func (m *Metrics) getSLIMetrics(scope string) (*sliMetrics, error) { // MakeMetrics makes the metrics for changefeed monitoring. func MakeMetrics(histogramWindow time.Duration) metric.Struct { m := &Metrics{ - AggMetrics: newAggregateMetrics(histogramWindow), - KVFeedMetrics: kvevent.MakeMetrics(histogramWindow), - SchemaFeedMetrics: schemafeed.MakeMetrics(histogramWindow), - ResolvedMessages: metric.NewCounter(metaChangefeedForwardedResolvedMessages), - Failures: metric.NewCounter(metaChangefeedFailures), - QueueTimeNanos: metric.NewCounter(metaEventQueueTime), - CheckpointHistNanos: metric.NewHistogram(metaChangefeedCheckpointHistNanos, histogramWindow, metric.IOLatencyBuckets), - FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates), - ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow), - ReplanCount: metric.NewCounter(metaChangefeedReplanCount), + AggMetrics: newAggregateMetrics(histogramWindow), + KVFeedMetrics: kvevent.MakeMetrics(histogramWindow), + SchemaFeedMetrics: schemafeed.MakeMetrics(histogramWindow), + ResolvedMessages: metric.NewCounter(metaChangefeedForwardedResolvedMessages), + Failures: metric.NewCounter(metaChangefeedFailures), + QueueTimeNanos: metric.NewCounter(metaEventQueueTime), + CheckpointHistNanos: metric.NewHistogram(metaChangefeedCheckpointHistNanos, histogramWindow, metric.IOLatencyBuckets), + FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates), + ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow), + ReplanCount: metric.NewCounter(metaChangefeedReplanCount), + ParallelConsumerFlushNanos: metric.NewCounter(metaChangefeedEventConsumerFlushNanos), + ParallelConsumerConsumeNanos: metric.NewCounter(metaChangefeedEventConsumerConsumeNanos), + ParallelConsumerInFlightEvents: metric.NewGauge(metaChangefeedEventConsumerInFlightEvents), } m.mu.resolved = make(map[int]hlc.Timestamp) diff --git a/pkg/ccl/changefeedccl/sink.go b/pkg/ccl/changefeedccl/sink.go index 758daaa5eeab..6b9d367d8a91 100644 --- a/pkg/ccl/changefeedccl/sink.go +++ b/pkg/ccl/changefeedccl/sink.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) @@ -536,3 +537,47 @@ func (n *nullSink) Close() error { func (n *nullSink) Dial() error { return nil } + +// safeSink wraps an EventSink in a mutex so it's methods are +// thread safe. It also has a beforeFlush hook which is called +// at the beginning of safeSink.Flush(). +type safeSink struct { + syncutil.Mutex + beforeFlush func(ctx context.Context) error + wrapped EventSink +} + +var _ EventSink = (*safeSink)(nil) + +func (s *safeSink) Dial() error { + s.Lock() + defer s.Unlock() + return s.wrapped.Dial() +} + +func (s *safeSink) Close() error { + s.Lock() + defer s.Unlock() + return s.wrapped.Close() +} + +func (s *safeSink) EmitRow( + ctx context.Context, + topic TopicDescriptor, + key, value []byte, + updated, mvcc hlc.Timestamp, + alloc kvevent.Alloc, +) error { + s.Lock() + defer s.Unlock() + return s.wrapped.EmitRow(ctx, topic, key, value, updated, mvcc, alloc) +} + +func (s *safeSink) Flush(ctx context.Context) error { + if err := s.beforeFlush(ctx); err != nil { + return err + } + s.Lock() + defer s.Unlock() + return s.wrapped.Flush(ctx) +} diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 31e16b1fa60b..795e04d5a274 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -1447,6 +1447,24 @@ var charts = []sectionDescription{ "changefeed.replan_count", }, }, + { + Title: "Nprocs Consume Event Nanos", + Metrics: []string{ + "changefeed.nprocs_consume_event_nanos", + }, + }, + { + Title: "Nprocs Flush Nanos", + Metrics: []string{ + "changefeed.nprocs_flush_nanos", + }, + }, + { + Title: "Nprocs In Flight Count", + Metrics: []string{ + "changefeed.nprocs_in_flight_count", + }, + }, { Title: "Flushed Bytes", Metrics: []string{ From aaea3ee2f4687f9f87c064ef15a548ee1f31578d Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Wed, 21 Sep 2022 16:21:35 -0400 Subject: [PATCH 07/13] changefeedccl: Do not block on file size based flushes Prior to this change, cloud storage sink trigger file sized based flush whenever new row would would push the file size beyond configured threshold. This had the effect of singificantly reducing the throughput whenever such event occured -- no additional events could be added to cloud storage sink, while the previus flush was active. This is not necessary. Cloud storage sink can trigger file based flushes asynchronously. The only requirement is that if a real, non file based, flush arrives, or if we need to emit resolved timestamps, then we must wait for all of the active flush requests to complete. In addition, because every event added to cloud sink has associate allocation, which is released when file is written out, performing flushes asynchronously is safe with respect to memory usage and accounting. Release note (enterprise change): Changefeeds, using cloud storage sink, now have better throughput. Release justification: performance fix --- pkg/ccl/changefeedccl/sink_cloudstorage.go | 105 +++++++-- .../changefeedccl/sink_cloudstorage_test.go | 199 ++++++++++-------- 2 files changed, 198 insertions(+), 106 deletions(-) diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage.go b/pkg/ccl/changefeedccl/sink_cloudstorage.go index c4b91317353d..4e4670a5d5c7 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage.go @@ -19,6 +19,7 @@ import ( "net/url" "path/filepath" "strings" + "sync" "sync/atomic" "time" @@ -27,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -307,6 +309,11 @@ type cloudStorageSink struct { dataFilePartition string prevFilename string metrics metricsRecorder + + asyncFlushActive bool + flushCtx context.Context + flushGroup sync.WaitGroup + flushErr atomic.Value } const sinkCompressionGzip = "gzip" @@ -367,8 +374,11 @@ func makeCloudStorageSink( partitionFormat: defaultPartitionFormat, timestampOracle: timestampOracle, // TODO(dan,ajwerner): Use the jobs framework's session ID once that's available. - jobSessionID: sessID, - topicNamer: tn, + jobSessionID: sessID, + topicNamer: tn, + asyncFlushActive: enableAsyncFlush.Get(&settings.SV), + // TODO (yevgeniy): Consider adding ctx to Dial method instead. + flushCtx: ctx, } if partitionFormat := u.consumeParam(changefeedbase.SinkParamPartitionFormat); partitionFormat != "" { @@ -506,6 +516,13 @@ func (s *cloudStorageSink) EmitResolvedTimestamp( if err != nil { return err } + + // Wait for previously issued async flush requests to complete + // before we write resolved time stamp file. + if err := s.waitAsyncFlush(); err != nil { + return errors.Wrapf(err, "while emitting resolved timestamp") + } + // Don't need to copy payload because we never buffer it anywhere. part := resolved.GoTime().Format(s.partitionFormat) @@ -575,29 +592,44 @@ func (s *cloudStorageSink) Flush(ctx context.Context) error { // for an overview of the naming convention and proof of correctness. s.dataFileTs = cloudStorageFormatTime(s.timestampOracle.inclusiveLowerBoundTS()) s.dataFilePartition = s.timestampOracle.inclusiveLowerBoundTS().GoTime().Format(s.partitionFormat) - return nil + return s.waitAsyncFlush() } -// file should not be used after flushing. -func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSinkFile) error { - defer file.alloc.Release(ctx) +// enableAsyncFlush controls async flushing behavior for this sink. +var enableAsyncFlush = settings.RegisterBoolSetting( + settings.TenantWritable, + "changefeed.cloudstorage.async_flush.enabled", + "enable async flushing", + true, +) - if file.rawSize == 0 { - // This method shouldn't be called with an empty file, but be defensive - // about not writing empty files anyway. - return nil +// waitAsyncFlush waits until all async flushes complete. +func (s *cloudStorageSink) waitAsyncFlush() error { + s.flushGroup.Wait() + if v := s.flushErr.Load(); v != nil { + return v.(error) } + return nil +} - if file.codec != nil { - if err := file.codec.Close(); err != nil { +// flushFile flushes file to the cloud storage. +// file should not be used after flushing. +func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSinkFile) error { + asyncFlushEnabled := enableAsyncFlush.Get(&s.settings.SV) + if s.asyncFlushActive && !asyncFlushEnabled { + // Async flush behavior was turned off -- drain any active flush requests + // before flushing this file. + if err := s.waitAsyncFlush(); err != nil { return err } } + s.asyncFlushActive = asyncFlushEnabled // We use this monotonically increasing fileID to ensure correct ordering // among files emitted at the same timestamp during the same job session. fileID := s.fileID s.fileID++ + // Pad file ID to maintain lexical ordering among files from the same sink. // Note that we use `-` here to delimit the filename because we want // `%d.RESOLVED` files to lexicographically succeed data files that have the @@ -609,11 +641,52 @@ func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSink "precedes a file emitted before: %s", filename, s.prevFilename) } s.prevFilename = filename - compressedBytes := file.buf.Len() - if err := cloud.WriteFile(ctx, s.es, filepath.Join(s.dataFilePartition, filename), bytes.NewReader(file.buf.Bytes())); err != nil { + dest := filepath.Join(s.dataFilePartition, filename) + + if !asyncFlushEnabled { + return file.flushToStorage(ctx, s.es, dest, s.metrics) + } + + s.flushGroup.Add(1) + go func() { + defer s.flushGroup.Done() + // NB: must use s.flushCtx; ctx may be short lived (i.e. cancelled). + if err := file.flushToStorage(s.flushCtx, s.es, dest, s.metrics); err != nil { + log.Errorf(ctx, "error flushing file to storage: %s", err) + // We must use the same type for error we store in flushErr. + s.flushErr.CompareAndSwap(nil, &flushError{error: err}) + } + }() + return nil +} + +type flushError struct { + error +} + +// flushToStorage writes out file into external storage into 'dest'. +func (f *cloudStorageSinkFile) flushToStorage( + ctx context.Context, es cloud.ExternalStorage, dest string, m metricsRecorder, +) error { + defer f.alloc.Release(ctx) + + if f.rawSize == 0 { + // This method shouldn't be called with an empty file, but be defensive + // about not writing empty files anyway. + return nil + } + + if f.codec != nil { + if err := f.codec.Close(); err != nil { + return err + } + } + + compressedBytes := f.buf.Len() + if err := cloud.WriteFile(ctx, es, dest, bytes.NewReader(f.buf.Bytes())); err != nil { return err } - s.metrics.recordEmittedBatch(file.created, file.numMessages, file.oldestMVCC, file.rawSize, compressedBytes) + m.recordEmittedBatch(f.created, f.numMessages, f.oldestMVCC, f.rawSize, compressedBytes) return nil } @@ -621,7 +694,7 @@ func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSink // Close implements the Sink interface. func (s *cloudStorageSink) Close() error { s.files = nil - return s.es.Close() + return errors.CombineErrors(s.waitAsyncFlush(), s.es.Close()) } // Dial implements the Sink interface. diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go index 9fe5b318fd30..e925476068ca 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go @@ -69,7 +69,7 @@ func TestCloudStorageSink(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - dir, dirCleanupFn := testutils.TempDir(t) + externalIODir, dirCleanupFn := testutils.TempDir(t) defer dirCleanupFn() gzipDecompress := func(t *testing.T, compressed []byte) []byte { @@ -88,8 +88,12 @@ func TestCloudStorageSink(t *testing.T) { return decompressed } - listLeafDirectories := func(root string) []string { - absRoot := filepath.Join(dir, root) + testDir := func(t *testing.T) string { + return strings.ReplaceAll(t.Name(), "/", ";") + } + + listLeafDirectories := func(t *testing.T) []string { + absRoot := filepath.Join(externalIODir, testDir(t)) var folders []string @@ -126,7 +130,7 @@ func TestCloudStorageSink(t *testing.T) { // slurpDir returns the contents of every file under root (relative to the // temp dir created above), sorted by the name of the file. - slurpDir := func(t *testing.T, root string) []string { + slurpDir := func(t *testing.T) []string { var files []string walkFn := func(path string, info os.FileInfo, err error) error { if err != nil { @@ -145,7 +149,7 @@ func TestCloudStorageSink(t *testing.T) { files = append(files, string(file)) return nil } - absRoot := filepath.Join(dir, root) + absRoot := filepath.Join(externalIODir, testDir(t)) require.NoError(t, os.MkdirAll(absRoot, 0755)) require.NoError(t, filepath.Walk(absRoot, walkFn)) return files @@ -154,7 +158,7 @@ func TestCloudStorageSink(t *testing.T) { const unlimitedFileSize int64 = math.MaxInt64 var noKey []byte settings := cluster.MakeTestingClusterSettings() - settings.ExternalIODir = dir + settings.ExternalIODir = externalIODir opts := changefeedbase.EncodingOptions{ Format: changefeedbase.OptFormatJSON, Envelope: changefeedbase.OptEnvelopeWrapped, @@ -180,25 +184,33 @@ func TestCloudStorageSink(t *testing.T) { user := username.RootUserName() - sinkURI := func(dir string, maxFileSize int64) sinkURL { - uri := `nodelocal://0/` + dir + sinkURI := func(t *testing.T, maxFileSize int64) sinkURL { + u, err := url.Parse(fmt.Sprintf("nodelocal://0/%s", testDir(t))) + require.NoError(t, err) + sink := sinkURL{URL: u} if maxFileSize != unlimitedFileSize { - uri += fmt.Sprintf("?%s=%d", changefeedbase.SinkParamFileSize, maxFileSize) + sink.addParam(changefeedbase.SinkParamFileSize, strconv.FormatInt(maxFileSize, 10)) } - u, err := url.Parse(uri) - require.NoError(t, err) - return sinkURL{URL: u} + return sink } - t.Run(`golden`, func(t *testing.T) { + testWithAndWithoutAsyncFlushing := func(t *testing.T, name string, testFn func(*testing.T)) { + t.Helper() + testutils.RunTrueAndFalse(t, name+"/asyncFlush", func(t *testing.T, enable bool) { + enableAsyncFlush.Override(context.Background(), &settings.SV, enable) + testFn(t) + }) + } + + testWithAndWithoutAsyncFlushing(t, `golden`, func(t *testing.T) { t1 := makeTopic(`t1`) testSpan := roachpb.Span{Key: []byte("a"), EndKey: []byte("b")} sf, err := span.MakeFrontier(testSpan) require.NoError(t, err) timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} - sinkDir := `golden` + s, err := makeCloudStorageSink( - ctx, sinkURI(sinkDir, unlimitedFileSize), 1, settings, + ctx, sinkURI(t, unlimitedFileSize), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) @@ -210,11 +222,11 @@ func TestCloudStorageSink(t *testing.T) { require.Equal(t, []string{ "v1\n", - }, slurpDir(t, sinkDir)) + }, slurpDir(t)) require.NoError(t, s.EmitResolvedTimestamp(ctx, e, ts(5))) resolvedFile, err := os.ReadFile(filepath.Join( - dir, sinkDir, `1970-01-01`, `197001010000000000000050000000000.RESOLVED`)) + externalIODir, testDir(t), `1970-01-01`, `197001010000000000000050000000000.RESOLVED`)) require.NoError(t, err) require.Equal(t, `{"resolved":"5.0000000000"}`, string(resolvedFile)) }) @@ -225,7 +237,14 @@ func TestCloudStorageSink(t *testing.T) { return forwarded } - t.Run(`single-node`, func(t *testing.T) { + stringOrDefault := func(s, ifEmpty string) string { + if len(s) == 0 { + return ifEmpty + } + return s + } + + testWithAndWithoutAsyncFlushing(t, `single-node`, func(t *testing.T) { before := opts.Compression // Compression codecs include buffering that interferes with other tests, // e.g. the bucketing test that configures very small flush sizes. @@ -234,7 +253,7 @@ func TestCloudStorageSink(t *testing.T) { }() for _, compression := range []string{"", "gzip"} { opts.Compression = compression - t.Run("compress="+compression, func(t *testing.T) { + t.Run("compress="+stringOrDefault(compression, "none"), func(t *testing.T) { t1 := makeTopic(`t1`) t2 := makeTopic(`t2`) @@ -242,9 +261,8 @@ func TestCloudStorageSink(t *testing.T) { sf, err := span.MakeFrontier(testSpan) require.NoError(t, err) timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} - dir := `single-node` + compression s, err := makeCloudStorageSink( - ctx, sinkURI(dir, unlimitedFileSize), 1, settings, + ctx, sinkURI(t, unlimitedFileSize), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) @@ -253,7 +271,7 @@ func TestCloudStorageSink(t *testing.T) { // Empty flush emits no files. require.NoError(t, s.Flush(ctx)) - require.Equal(t, []string(nil), slurpDir(t, dir)) + require.Equal(t, []string(nil), slurpDir(t)) // Emitting rows and flushing should write them out in one file per table. Note // the ordering among these two files is non deterministic as either of them could @@ -268,19 +286,19 @@ func TestCloudStorageSink(t *testing.T) { "v1\nv2\n", "w1\n", } - actual := slurpDir(t, dir) + actual := slurpDir(t) sort.Strings(actual) require.Equal(t, expected, actual) // Flushing with no new emits writes nothing new. require.NoError(t, s.Flush(ctx)) - actual = slurpDir(t, dir) + actual = slurpDir(t) sort.Strings(actual) require.Equal(t, expected, actual) // Without a flush, nothing new shows up. require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(`v3`), ts(3), ts(3), zeroAlloc)) - actual = slurpDir(t, dir) + actual = slurpDir(t) sort.Strings(actual) require.Equal(t, expected, actual) @@ -290,7 +308,7 @@ func TestCloudStorageSink(t *testing.T) { require.NoError(t, s.Flush(ctx)) require.Equal(t, []string{ "v3\n", - }, slurpDir(t, dir)[2:]) + }, slurpDir(t)[2:]) // Data from different versions of a table is put in different files, so that we // can guarantee that all rows in any given file have the same schema. @@ -306,7 +324,7 @@ func TestCloudStorageSink(t *testing.T) { "v4\n", "v5\n", } - actual = slurpDir(t, dir) + actual = slurpDir(t) actual = actual[len(actual)-2:] sort.Strings(actual) require.Equal(t, expected, actual) @@ -314,22 +332,20 @@ func TestCloudStorageSink(t *testing.T) { } }) - t.Run(`multi-node`, func(t *testing.T) { + testWithAndWithoutAsyncFlushing(t, `multi-node`, func(t *testing.T) { t1 := makeTopic(`t1`) - testSpan := roachpb.Span{Key: []byte("a"), EndKey: []byte("b")} sf, err := span.MakeFrontier(testSpan) require.NoError(t, err) timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} - dir := `multi-node` s1, err := makeCloudStorageSink( - ctx, sinkURI(dir, unlimitedFileSize), 1, + ctx, sinkURI(t, unlimitedFileSize), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) defer func() { require.NoError(t, s1.Close()) }() s2, err := makeCloudStorageSink( - ctx, sinkURI(dir, unlimitedFileSize), 2, + ctx, sinkURI(t, unlimitedFileSize), 2, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) defer func() { require.NoError(t, s2.Close()) }() @@ -353,19 +369,19 @@ func TestCloudStorageSink(t *testing.T) { require.Equal(t, []string{ "v1\n", "w1\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) // If a node restarts then the entire distsql flow has to restart. If // this happens before checkpointing, some data is written again but // this is unavoidable. s1R, err := makeCloudStorageSink( - ctx, sinkURI(dir, unbuffered), 1, + ctx, sinkURI(t, unbuffered), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) defer func() { require.NoError(t, s1R.Close()) }() s2R, err := makeCloudStorageSink( - ctx, sinkURI(dir, unbuffered), 2, + ctx, sinkURI(t, unbuffered), 2, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) @@ -390,7 +406,7 @@ func TestCloudStorageSink(t *testing.T) { "v1\n", "w1\n", "w1\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) }) // The jobs system can't always clean up perfectly after itself and so there @@ -400,15 +416,14 @@ func TestCloudStorageSink(t *testing.T) { // // This test is also sufficient for verifying the behavior of a multi-node // changefeed using this sink. Ditto job restarts. - t.Run(`zombie`, func(t *testing.T) { + testWithAndWithoutAsyncFlushing(t, `zombie`, func(t *testing.T) { t1 := makeTopic(`t1`) testSpan := roachpb.Span{Key: []byte("a"), EndKey: []byte("b")} sf, err := span.MakeFrontier(testSpan) require.NoError(t, err) timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} - dir := `zombie` s1, err := makeCloudStorageSink( - ctx, sinkURI(dir, unlimitedFileSize), 1, + ctx, sinkURI(t, unlimitedFileSize), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) @@ -416,7 +431,7 @@ func TestCloudStorageSink(t *testing.T) { s1.(*cloudStorageSink).sinkID = 7 // Force a deterministic sinkID. s1.(*cloudStorageSink).jobSessionID = "a" // Force deterministic job session ID. s2, err := makeCloudStorageSink( - ctx, sinkURI(dir, unlimitedFileSize), 1, + ctx, sinkURI(t, unlimitedFileSize), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) @@ -441,19 +456,21 @@ func TestCloudStorageSink(t *testing.T) { "v1\nv2\n", "v3\n", "v1\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) }) - t.Run(`bucketing`, func(t *testing.T) { + waitAsyncFlush := func(s Sink) error { + return s.(*cloudStorageSink).waitAsyncFlush() + } + testWithAndWithoutAsyncFlushing(t, `bucketing`, func(t *testing.T) { t1 := makeTopic(`t1`) testSpan := roachpb.Span{Key: []byte("a"), EndKey: []byte("b")} sf, err := span.MakeFrontier(testSpan) require.NoError(t, err) timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} - dir := `bucketing` const targetMaxFileSize = 6 s, err := makeCloudStorageSink( - ctx, sinkURI(dir, targetMaxFileSize), 1, + ctx, sinkURI(t, targetMaxFileSize), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) @@ -465,16 +482,17 @@ func TestCloudStorageSink(t *testing.T) { for i := int64(1); i <= 5; i++ { require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(fmt.Sprintf(`v%d`, i)), ts(i), ts(i), zeroAlloc)) } + require.NoError(t, waitAsyncFlush(s)) require.Equal(t, []string{ "v1\nv2\nv3\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) // Flush then writes the rest. require.NoError(t, s.Flush(ctx)) require.Equal(t, []string{ "v1\nv2\nv3\n", "v4\nv5\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) // Forward the SpanFrontier here and trigger an empty flush to update // the sink's `inclusiveLowerBoundTs` @@ -487,11 +505,12 @@ func TestCloudStorageSink(t *testing.T) { for i := int64(6); i < 10; i++ { require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(fmt.Sprintf(`v%d`, i)), ts(i), ts(i), zeroAlloc)) } + require.NoError(t, waitAsyncFlush(s)) require.Equal(t, []string{ "v1\nv2\nv3\n", "v4\nv5\n", "v6\nv7\nv8\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) // Resolved timestamps are periodically written. This happens // asynchronously from a different node and can be given an earlier @@ -507,7 +526,7 @@ func TestCloudStorageSink(t *testing.T) { "v4\nv5\n", `{"resolved":"5.0000000000"}`, "v6\nv7\nv8\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) // Flush then writes the rest. Since we use the time of the EmitRow // or EmitResolvedTimestamp calls to order files, the resolved timestamp @@ -520,7 +539,7 @@ func TestCloudStorageSink(t *testing.T) { `{"resolved":"5.0000000000"}`, "v6\nv7\nv8\n", "v9\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) // A resolved timestamp emitted with ts > 5 should follow everything // emitted thus far. @@ -532,10 +551,10 @@ func TestCloudStorageSink(t *testing.T) { "v6\nv7\nv8\n", "v9\n", `{"resolved":"6.0000000000"}`, - }, slurpDir(t, dir)) + }, slurpDir(t)) }) - t.Run(`partition-formatting`, func(t *testing.T) { + testWithAndWithoutAsyncFlushing(t, `partition-formatting`, func(t *testing.T) { t1 := makeTopic(`t1`) testSpan := roachpb.Span{Key: []byte("a"), EndKey: []byte("b")} const targetMaxFileSize = 6 @@ -550,7 +569,7 @@ func TestCloudStorageSink(t *testing.T) { time.Date(2000, time.January, 2, 6, 1, 1, 0, time.UTC), } - for i, tc := range []struct { + for _, tc := range []struct { format string expectedFolders []string }{ @@ -582,51 +601,50 @@ func TestCloudStorageSink(t *testing.T) { }, }, } { - t.Run(tc.format, func(t *testing.T) { - sf, err := span.MakeFrontier(testSpan) - require.NoError(t, err) - timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} + testWithAndWithoutAsyncFlushing(t, stringOrDefault(tc.format, "default"), + func(t *testing.T) { + sf, err := span.MakeFrontier(testSpan) + require.NoError(t, err) + timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} - dir := fmt.Sprintf(`partition-formatting-%d`, i) + sinkURIWithParam := sinkURI(t, targetMaxFileSize) + sinkURIWithParam.addParam(changefeedbase.SinkParamPartitionFormat, tc.format) + t.Logf("format=%s sinkgWithParam: %s", tc.format, sinkURIWithParam.String()) + s, err := makeCloudStorageSink( + ctx, sinkURIWithParam, 1, + settings, opts, timestampOracle, externalStorageFromURI, user, nil, + ) - sinkURIWithParam := sinkURI(dir, targetMaxFileSize) - sinkURIWithParam.addParam(changefeedbase.SinkParamPartitionFormat, tc.format) - s, err := makeCloudStorageSink( - ctx, sinkURIWithParam, 1, - settings, opts, timestampOracle, externalStorageFromURI, user, nil, - ) - - require.NoError(t, err) - defer func() { require.NoError(t, s.Close()) }() - s.(*cloudStorageSink).sinkID = 7 // Force a deterministic sinkID. + require.NoError(t, err) + defer func() { require.NoError(t, s.Close()) }() + s.(*cloudStorageSink).sinkID = 7 // Force a deterministic sinkID. - for i, timestamp := range timestamps { - hlcTime := ts(timestamp.UnixNano()) + for i, timestamp := range timestamps { + hlcTime := ts(timestamp.UnixNano()) - // Move the frontier and flush to update the dataFilePartition value - _, err = sf.Forward(testSpan, hlcTime) - require.NoError(t, err) - require.NoError(t, s.Flush(ctx)) + // Move the frontier and flush to update the dataFilePartition value + _, err = sf.Forward(testSpan, hlcTime) + require.NoError(t, err) + require.NoError(t, s.Flush(ctx)) - require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(fmt.Sprintf(`v%d`, i)), hlcTime, hlcTime, zeroAlloc)) - } + require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(fmt.Sprintf(`v%d`, i)), hlcTime, hlcTime, zeroAlloc)) + } - require.NoError(t, s.Flush(ctx)) // Flush the last file - require.ElementsMatch(t, tc.expectedFolders, listLeafDirectories(dir)) - require.Equal(t, []string{"v0\n", "v1\n", "v2\n", "v3\n", "v4\n"}, slurpDir(t, dir)) - }) + require.NoError(t, s.Flush(ctx)) // Flush the last file + require.ElementsMatch(t, tc.expectedFolders, listLeafDirectories(t)) + require.Equal(t, []string{"v0\n", "v1\n", "v2\n", "v3\n", "v4\n"}, slurpDir(t)) + }) } }) - t.Run(`file-ordering`, func(t *testing.T) { + testWithAndWithoutAsyncFlushing(t, `file-ordering`, func(t *testing.T) { t1 := makeTopic(`t1`) testSpan := roachpb.Span{Key: []byte("a"), EndKey: []byte("b")} sf, err := span.MakeFrontier(testSpan) require.NoError(t, err) timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} - dir := `file-ordering` s, err := makeCloudStorageSink( - ctx, sinkURI(dir, unlimitedFileSize), 1, + ctx, sinkURI(t, unlimitedFileSize), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil, ) require.NoError(t, err) @@ -663,7 +681,7 @@ func TestCloudStorageSink(t *testing.T) { `{"resolved":"3.0000000000"}`, "e3next\n", `{"resolved":"4.0000000000"}`, - }, slurpDir(t, dir)) + }, slurpDir(t)) // Test that files with timestamp lower than the least resolved timestamp // as of file creation time are ignored. @@ -675,19 +693,18 @@ func TestCloudStorageSink(t *testing.T) { `{"resolved":"3.0000000000"}`, "e3next\n", `{"resolved":"4.0000000000"}`, - }, slurpDir(t, dir)) + }, slurpDir(t)) }) - t.Run(`ordering-among-schema-versions`, func(t *testing.T) { + testWithAndWithoutAsyncFlushing(t, `ordering-among-schema-versions`, func(t *testing.T) { t1 := makeTopic(`t1`) testSpan := roachpb.Span{Key: []byte("a"), EndKey: []byte("b")} sf, err := span.MakeFrontier(testSpan) require.NoError(t, err) timestampOracle := &changeAggregatorLowerBoundOracle{sf: sf} - dir := `ordering-among-schema-versions` var targetMaxFileSize int64 = 10 s, err := makeCloudStorageSink( - ctx, sinkURI(dir, targetMaxFileSize), 1, settings, + ctx, sinkURI(t, targetMaxFileSize), 1, settings, opts, timestampOracle, externalStorageFromURI, user, nil) require.NoError(t, err) defer func() { require.NoError(t, s.Close()) }() @@ -699,20 +716,22 @@ func TestCloudStorageSink(t *testing.T) { // for the first file but not the second one. t1.Version = 0 require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(`trigger-flush-v1`), ts(1), ts(1), zeroAlloc)) + require.NoError(t, waitAsyncFlush(s)) require.Equal(t, []string{ "v1\ntrigger-flush-v1\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) // Now make the file with the newer schema exceed its file size threshold and ensure // that the file with the older schema is flushed (and ordered) before. require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(`v2`), ts(1), ts(1), zeroAlloc)) t1.Version = 1 require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(`trigger-flush-v3`), ts(1), ts(1), zeroAlloc)) + require.NoError(t, waitAsyncFlush(s)) require.Equal(t, []string{ "v1\ntrigger-flush-v1\n", "v2\n", "v3\ntrigger-flush-v3\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) // Calling `Flush()` on the sink should emit files in the order of their schema IDs. require.NoError(t, s.EmitRow(ctx, t1, noKey, []byte(`w1`), ts(1), ts(1), zeroAlloc)) @@ -725,6 +744,6 @@ func TestCloudStorageSink(t *testing.T) { "v3\ntrigger-flush-v3\n", "x1\n", "w1\n", - }, slurpDir(t, dir)) + }, slurpDir(t)) }) } From 5ad219fe87c350b446d1f5e6028f79158af71142 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Fri, 23 Sep 2022 15:41:46 -0400 Subject: [PATCH 08/13] changefeedccl: Expand the set of available compression algorithms Expand the set of supported compression algorithms in changefeed. A faster implementation of gzip algorithm is avaible, and is used by default. The gzip algorithm implementation can be reverted to Go standard gzip implementation via the `changefeed.fast_gzip.enabled` setting. In addition, add support for compression files with zstd. Release notes (enterprise change): Changefeed can emit files compressed with zstd algorithm -- which provides good compression, and is much faster than gzip. In addition, a new, faster implementation of gzip is used by default. --- .../settings/settings-for-tenants.txt | 1 + docs/generated/settings/settings.html | 1 + pkg/ccl/changefeedccl/BUILD.bazel | 2 + .../changefeedccl/changefeedbase/options.go | 2 +- pkg/ccl/changefeedccl/compression.go | 70 +++++++++++++++++++ pkg/ccl/changefeedccl/sink_cloudstorage.go | 36 +++++----- 6 files changed, 95 insertions(+), 17 deletions(-) create mode 100644 pkg/ccl/changefeedccl/compression.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index a77318cad8ac..9f312711b9ad 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -12,6 +12,7 @@ bulkio.backup.read_with_priority_after duration 1m0s amount of time since the re bulkio.stream_ingestion.minimum_flush_interval duration 5s the minimum timestamp between flushes; flushes may still occur if internal buffers fill up changefeed.event_consumer_worker_queue_size integer 16 if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of eventswhich a worker can buffer changefeed.event_consumer_workers integer 8 the number of workers to use when processing events; 0 or 1 disables +changefeed.fast_gzip.enabled boolean true use fast gzip implementation changefeed.node_throttle_config string specifies node level throttling configuration for all changefeeeds changefeed.schema_feed.read_with_priority_after duration 1m0s retry with high priority if we were not able to read descriptors for too long; 0 disables cloudstorage.azure.concurrent_upload_buffers integer 1 controls the number of concurrent buffers that will be used by the Azure client when uploading chunks.Each buffer can buffer up to cloudstorage.write_chunk.size of memory during an upload diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 392941a74a35..ed69d45aa37d 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -18,6 +18,7 @@ bulkio.stream_ingestion.minimum_flush_intervalduration5sthe minimum timestamp between flushes; flushes may still occur if internal buffers fill up changefeed.event_consumer_worker_queue_sizeinteger16if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of eventswhich a worker can buffer changefeed.event_consumer_workersinteger8the number of workers to use when processing events; 0 or 1 disables +changefeed.fast_gzip.enabledbooleantrueuse fast gzip implementation changefeed.node_throttle_configstringspecifies node level throttling configuration for all changefeeeds changefeed.schema_feed.read_with_priority_afterduration1m0sretry with high priority if we were not able to read descriptors for too long; 0 disables cloudstorage.azure.concurrent_upload_buffersinteger1controls the number of concurrent buffers that will be used by the Azure client when uploading chunks.Each buffer can buffer up to cloudstorage.write_chunk.size of memory during an upload diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index c3dd3c8c3e02..3859f1d7ddb2 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "changefeed_dist.go", "changefeed_processors.go", "changefeed_stmt.go", + "compression.go", "doc.go", "encoder.go", "encoder_avro.go", @@ -96,6 +97,7 @@ go_library( "//pkg/sql/sessiondatapb", "//pkg/sql/sqlutil", "//pkg/sql/types", + "//pkg/util", "//pkg/util/bitarray", "//pkg/util/bufalloc", "//pkg/util/cache", diff --git a/pkg/ccl/changefeedccl/changefeedbase/options.go b/pkg/ccl/changefeedccl/changefeedbase/options.go index ec6c80cdbf24..d132778fb90c 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/options.go +++ b/pkg/ccl/changefeedccl/changefeedbase/options.go @@ -297,7 +297,7 @@ var ChangefeedOptionExpectValues = map[string]OptionPermittedValues{ OptUpdatedTimestamps: flagOption, OptMVCCTimestamps: flagOption, OptDiff: flagOption, - OptCompression: enum("gzip"), + OptCompression: enum("gzip", "zstd"), OptSchemaChangeEvents: enum("column_changes", "default"), OptSchemaChangePolicy: enum("backfill", "nobackfill", "stop", "ignore"), OptSplitColumnFamilies: flagOption, diff --git a/pkg/ccl/changefeedccl/compression.go b/pkg/ccl/changefeedccl/compression.go new file mode 100644 index 000000000000..c4d744a1445b --- /dev/null +++ b/pkg/ccl/changefeedccl/compression.go @@ -0,0 +1,70 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package changefeedccl + +import ( + stdgzip "compress/gzip" + "io" + "strings" + + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/errors" + "github.com/klauspost/compress/zstd" + "github.com/klauspost/pgzip" +) + +var useFastGzip = settings.RegisterBoolSetting( + settings.TenantWritable, + "changefeed.fast_gzip.enabled", + "use fast gzip implementation", + util.ConstantWithMetamorphicTestBool( + "changefeed.fast_gzip.enabled", true, + ), +).WithPublic() + +type compressionAlgo string + +const sinkCompressionGzip compressionAlgo = "gzip" +const sinkCompressionZstd compressionAlgo = "zstd" + +func (a compressionAlgo) enabled() bool { + return a != "" +} + +// newCompressionCodec returns compression codec for the specified algorithm, +// which writes compressed data to the destination. +// TODO(yevgeniy): Support compression configuration (level, speed, etc). +// TODO(yevgeniy): Add telemetry. +func newCompressionCodec( + algo compressionAlgo, sv *settings.Values, dest io.Writer, +) (io.WriteCloser, error) { + switch algo { + case sinkCompressionGzip: + if useFastGzip.Get(sv) { + return pgzip.NewWriterLevel(dest, pgzip.DefaultCompression) + } + return stdgzip.NewWriterLevel(dest, stdgzip.DefaultCompression) + case sinkCompressionZstd: + return zstd.NewWriter(dest, zstd.WithEncoderLevel(zstd.SpeedFastest)) + default: + return nil, errors.AssertionFailedf("unsupported compression algorithm %q", algo) + } +} + +// compressionFromString returns compression algorithm type along with file extension. +func compressionFromString(algo string) (_ compressionAlgo, ext string, _ error) { + if strings.EqualFold(algo, string(sinkCompressionGzip)) { + return sinkCompressionGzip, ".gz", nil + } + if strings.EqualFold(algo, string(sinkCompressionZstd)) { + return sinkCompressionZstd, ".zst", nil + } + return "", "", errors.AssertionFailedf("unsupported compression algorithm %q", algo) +} diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage.go b/pkg/ccl/changefeedccl/sink_cloudstorage.go index 4e4670a5d5c7..fbc6262255f7 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage.go @@ -10,7 +10,6 @@ package changefeedccl import ( "bytes" - "compress/gzip" "context" "crypto/rand" "encoding/hex" @@ -291,7 +290,7 @@ type cloudStorageSink struct { ext string rowDelimiter []byte - compression string + compression compressionAlgo es cloud.ExternalStorage @@ -316,8 +315,6 @@ type cloudStorageSink struct { flushErr atomic.Value } -const sinkCompressionGzip = "gzip" - var cloudStorageSinkIDAtomic int64 // Files that are emitted can be partitioned by their earliest event time, @@ -423,12 +420,12 @@ func makeCloudStorageSink( } if codec := encodingOpts.Compression; codec != "" { - if strings.EqualFold(codec, "gzip") { - s.compression = sinkCompressionGzip - s.ext = s.ext + ".gz" - } else { - return nil, errors.Errorf(`unsupported compression codec %q`, codec) + algo, ext, err := compressionFromString(codec) + if err != nil { + return nil, err } + s.compression = algo + s.ext = s.ext + ext } // We make the external storage with a nil IOAccountingInterceptor since we @@ -446,7 +443,7 @@ func makeCloudStorageSink( func (s *cloudStorageSink) getOrCreateFile( topic TopicDescriptor, eventMVCC hlc.Timestamp, -) *cloudStorageSinkFile { +) (*cloudStorageSinkFile, error) { name, _ := s.topicNamer.Name(topic) key := cloudStorageSinkKey{name, int64(topic.GetVersion())} if item := s.files.Get(key); item != nil { @@ -454,19 +451,23 @@ func (s *cloudStorageSink) getOrCreateFile( if eventMVCC.Less(f.oldestMVCC) { f.oldestMVCC = eventMVCC } - return f + return f, nil } f := &cloudStorageSinkFile{ created: timeutil.Now(), cloudStorageSinkKey: key, oldestMVCC: eventMVCC, } - switch s.compression { - case sinkCompressionGzip: - f.codec = gzip.NewWriter(&f.buf) + + if s.compression.enabled() { + codec, err := newCompressionCodec(s.compression, &s.settings.SV, &f.buf) + if err != nil { + return nil, err + } + f.codec = codec } s.files.ReplaceOrInsert(f) - return f + return f, nil } // EmitRow implements the Sink interface. @@ -482,7 +483,10 @@ func (s *cloudStorageSink) EmitRow( } s.metrics.recordMessageSize(int64(len(key) + len(value))) - file := s.getOrCreateFile(topic, mvcc) + file, err := s.getOrCreateFile(topic, mvcc) + if err != nil { + return err + } file.alloc.Merge(&alloc) if _, err := file.Write(value); err != nil { From dd091491c35cc44da87fd4bfd503a938fc963ee0 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Sat, 24 Sep 2022 19:28:41 -0400 Subject: [PATCH 09/13] changefeedccl: Uniformly distribute work during export. By default, changefeed distributes the work to nodes based on which nodes are the lease holder for the ranges. This makes sense since running rangefeed against local node is more efficient. In a cluster where ranges are almost uniformly assigned to each node, running changefeed export is efficient: all nodes are busy, until they are done. KV server is responsible for making sure that the ranges are more or less uniformly distributed across the cluster; however, this determination is based on the set of all ranges in the cluster, and not based on a particular table. As a result, it is possible to have a table that does not uniform distribution of its ranges across all the nodes. When this happens, the changefeed export would take long time due to the long tail: as each node completes its set of assigned ranges, it idles until changefeed completes. This PR introduces a change (controlled via `changefeed.balance_range_distribution.enable` setting) where the changefeed try to produce a more balanced assignment, where each node is responsible for roughly 1/Nth of the work for the cluster of N nodes. Release note (enterprise change): Changefeed exports are up to 25% faster due to uniform work assignment. --- .../settings/settings-for-tenants.txt | 1 + docs/generated/settings/settings.html | 1 + pkg/ccl/changefeedccl/BUILD.bazel | 1 + pkg/ccl/changefeedccl/changefeed_dist.go | 134 ++++++++++++++++-- pkg/ccl/changefeedccl/changefeed_test.go | 81 +++++++++++ pkg/ccl/changefeedccl/kvfeed/scanner.go | 5 +- 6 files changed, 211 insertions(+), 12 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 9f312711b9ad..102b89e9da19 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -10,6 +10,7 @@ bulkio.backup.file_size byte size 128 MiB target size for individual data files bulkio.backup.read_timeout duration 5m0s amount of time after which a read attempt is considered timed out, which causes the backup to fail bulkio.backup.read_with_priority_after duration 1m0s amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads bulkio.stream_ingestion.minimum_flush_interval duration 5s the minimum timestamp between flushes; flushes may still occur if internal buffers fill up +changefeed.balance_range_distribution.enable boolean false if enabled, the ranges are balanced equally among all nodes changefeed.event_consumer_worker_queue_size integer 16 if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of eventswhich a worker can buffer changefeed.event_consumer_workers integer 8 the number of workers to use when processing events; 0 or 1 disables changefeed.fast_gzip.enabled boolean true use fast gzip implementation diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index ed69d45aa37d..7c694271908a 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -16,6 +16,7 @@ bulkio.backup.read_timeoutduration5m0samount of time after which a read attempt is considered timed out, which causes the backup to fail bulkio.backup.read_with_priority_afterduration1m0samount of time since the read-as-of time above which a BACKUP should use priority when retrying reads bulkio.stream_ingestion.minimum_flush_intervalduration5sthe minimum timestamp between flushes; flushes may still occur if internal buffers fill up +changefeed.balance_range_distribution.enablebooleanfalseif enabled, the ranges are balanced equally among all nodes changefeed.event_consumer_worker_queue_sizeinteger16if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of eventswhich a worker can buffer changefeed.event_consumer_workersinteger8the number of workers to use when processing events; 0 or 1 disables changefeed.fast_gzip.enabledbooleantrueuse fast gzip implementation diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index 3859f1d7ddb2..86e691fdce26 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -59,6 +59,7 @@ go_library( "//pkg/jobs/jobsprotectedts", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvserver", "//pkg/kv/kvserver/closedts", "//pkg/kv/kvserver/protectedts", diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go index 91210a4e10a9..3952e4729121 100644 --- a/pkg/ccl/changefeedccl/changefeed_dist.go +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -10,12 +10,15 @@ package changefeedccl import ( "context" + "sort" "time" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdceval" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" + "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql" @@ -29,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" @@ -307,6 +311,14 @@ func startDistChangefeed( return err } +var enableBalancedRangeDistribution = settings.RegisterBoolSetting( + settings.TenantWritable, + "changefeed.balance_range_distribution.enable", + "if enabled, the ranges are balanced equally among all nodes", + util.ConstantWithMetamorphicTestBool( + "changefeed.balance_range_distribution.enable", false), +).WithPublic() + func makePlan( execCtx sql.JobExecContext, jobID jobspb.JobID, @@ -316,24 +328,41 @@ func makePlan( trackedSpans []roachpb.Span, selectClause string, ) func(context.Context, *sql.DistSQLPlanner) (*sql.PhysicalPlan, *sql.PlanningCtx, error) { - return func(ctx context.Context, dsp *sql.DistSQLPlanner) (*sql.PhysicalPlan, *sql.PlanningCtx, error) { var blankTxn *kv.Txn + distMode := sql.DistributionTypeAlways + if details.SinkURI == `` { + // Sinkless feeds get one ChangeAggregator on this node. + distMode = sql.DistributionTypeNone + } + planCtx := dsp.NewPlanningCtx(ctx, execCtx.ExtendedEvalContext(), nil /* planner */, blankTxn, - sql.DistributionTypeAlways) + sql.DistributionType(distMode)) + spanPartitions, err := dsp.PartitionSpans(ctx, planCtx, trackedSpans) + if err != nil { + return nil, nil, err + } - var spanPartitions []sql.SpanPartition - if details.SinkURI == `` { - // Sinkless feeds get one ChangeAggregator on the gateway. - spanPartitions = []sql.SpanPartition{{SQLInstanceID: dsp.GatewayID(), Spans: trackedSpans}} - } else { - // All other feeds get a ChangeAggregator local on the leaseholder. - var err error - spanPartitions, err = dsp.PartitionSpans(ctx, planCtx, trackedSpans) + sv := &execCtx.ExecCfg().Settings.SV + if enableBalancedRangeDistribution.Get(sv) { + scanType, err := changefeedbase.MakeStatementOptions(details.Opts).GetInitialScanType() if err != nil { return nil, nil, err } + + // Currently, balanced range distribution supported only in export mode. + // TODO(yevgeniy): Consider lifting this restriction. + if scanType == changefeedbase.OnlyInitialScan { + sender := execCtx.ExecCfg().DB.NonTransactionalSender() + distSender := sender.(*kv.CrossRangeTxnWrapperSender).Wrapped().(*kvcoord.DistSender) + + spanPartitions, err = rebalanceSpanPartitions( + ctx, &distResolver{distSender}, rebalanceThreshold.Get(sv), spanPartitions) + if err != nil { + return nil, nil, err + } + } } // Use the same checkpoint for all aggregators; each aggregator will only look at @@ -442,3 +471,88 @@ func (w *changefeedResultWriter) SetError(err error) { func (w *changefeedResultWriter) Err() error { return w.err } + +var rebalanceThreshold = settings.RegisterFloatSetting( + settings.TenantWritable, + "changefeed.balance_range_distribution.sensitivity", + "rebalance if the number of ranges on a node exceeds the average by this fraction", + 0.05, + settings.PositiveFloat, +) + +type rangeResolver interface { + getRangesForSpans(ctx context.Context, spans []roachpb.Span) ([]roachpb.Span, error) +} + +type distResolver struct { + *kvcoord.DistSender +} + +func (r *distResolver) getRangesForSpans( + ctx context.Context, spans []roachpb.Span, +) ([]roachpb.Span, error) { + return kvfeed.AllRangeSpans(ctx, r.DistSender, spans) +} + +func rebalanceSpanPartitions( + ctx context.Context, r rangeResolver, sensitivity float64, p []sql.SpanPartition, +) ([]sql.SpanPartition, error) { + if len(p) <= 1 { + return p, nil + } + + // Explode set of spans into set of ranges. + // TODO(yevgeniy): This might not be great if the tables are huge. + numRanges := 0 + for i := range p { + spans, err := r.getRangesForSpans(ctx, p[i].Spans) + if err != nil { + return nil, err + } + p[i].Spans = spans + numRanges += len(spans) + } + + // Sort descending based on the number of ranges. + sort.Slice(p, func(i, j int) bool { + return len(p[i].Spans) > len(p[j].Spans) + }) + + targetRanges := int((1 + sensitivity) * float64(numRanges) / float64(len(p))) + + for i, j := 0, len(p)-1; i < j && len(p[i].Spans) > targetRanges && len(p[j].Spans) < targetRanges; { + from, to := i, j + + // Figure out how many ranges we can move. + numToMove := len(p[from].Spans) - targetRanges + canMove := targetRanges - len(p[to].Spans) + if numToMove <= canMove { + i++ + } + if canMove <= numToMove { + numToMove = canMove + j-- + } + if numToMove == 0 { + break + } + + // Move numToMove spans from 'from' to 'to'. + idx := len(p[from].Spans) - numToMove + p[to].Spans = append(p[to].Spans, p[from].Spans[idx:]...) + p[from].Spans = p[from].Spans[:idx] + } + + // Collapse ranges into nice set of contiguous spans. + for i := range p { + var g roachpb.SpanGroup + g.Add(p[i].Spans...) + p[i].Spans = g.Slice() + } + + // Finally, re-sort based on the node id. + sort.Slice(p, func(i, j int) bool { + return p[i].SQLInstanceID < p[j].SQLInstanceID + }) + return p, nil +} diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 1b2a063f36bd..1dc1252d04f4 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -7425,3 +7425,84 @@ func TestChangefeedKafkaMessageTooLarge(t *testing.T) { cdcTest(t, testFn, feedTestForceSink(`kafka`)) } + +type echoResolver struct { + result []roachpb.Spans + pos int +} + +func (r *echoResolver) getRangesForSpans( + _ context.Context, _ []roachpb.Span, +) (spans []roachpb.Span, _ error) { + spans = r.result[r.pos] + r.pos++ + return spans, nil +} + +func TestPartitionSpans(t *testing.T) { + defer leaktest.AfterTest(t)() + + partitions := func(p ...sql.SpanPartition) []sql.SpanPartition { + return p + } + mkPart := func(n base.SQLInstanceID, spans ...roachpb.Span) sql.SpanPartition { + return sql.SpanPartition{SQLInstanceID: n, Spans: spans} + } + mkSpan := func(start, end string) roachpb.Span { + return roachpb.Span{Key: []byte(start), EndKey: []byte(end)} + } + spans := func(s ...roachpb.Span) roachpb.Spans { + return s + } + const sensitivity = 0.01 + + for i, tc := range []struct { + input []sql.SpanPartition + resolve []roachpb.Spans + expect []sql.SpanPartition + }{ + { + input: partitions( + mkPart(1, mkSpan("a", "j")), + mkPart(2, mkSpan("j", "q")), + mkPart(3, mkSpan("q", "z")), + ), + // 6 total ranges, 2 per node. + resolve: []roachpb.Spans{ + spans(mkSpan("a", "c"), mkSpan("c", "e"), mkSpan("e", "j")), + spans(mkSpan("j", "q")), + spans(mkSpan("q", "y"), mkSpan("y", "z")), + }, + expect: partitions( + mkPart(1, mkSpan("a", "e")), + mkPart(2, mkSpan("e", "q")), + mkPart(3, mkSpan("q", "z")), + ), + }, + { + input: partitions( + mkPart(1, mkSpan("a", "c"), mkSpan("e", "p"), mkSpan("r", "z")), + mkPart(2), + mkPart(3, mkSpan("c", "e"), mkSpan("p", "r")), + ), + // 5 total ranges -- on 2 nodes; target should be 1 per node. + resolve: []roachpb.Spans{ + spans(mkSpan("a", "c"), mkSpan("e", "p"), mkSpan("r", "z")), + spans(), + spans(mkSpan("c", "e"), mkSpan("p", "r")), + }, + expect: partitions( + mkPart(1, mkSpan("a", "c"), mkSpan("e", "p")), + mkPart(2, mkSpan("r", "z")), + mkPart(3, mkSpan("c", "e"), mkSpan("p", "r")), + ), + }, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + sp, err := rebalanceSpanPartitions(context.Background(), + &echoResolver{result: tc.resolve}, sensitivity, tc.input) + require.NoError(t, err) + require.Equal(t, tc.expect, sp) + }) + } +} diff --git a/pkg/ccl/changefeedccl/kvfeed/scanner.go b/pkg/ccl/changefeedccl/kvfeed/scanner.go index 2c1f2195b7ed..d53f92336dcb 100644 --- a/pkg/ccl/changefeedccl/kvfeed/scanner.go +++ b/pkg/ccl/changefeedccl/kvfeed/scanner.go @@ -194,7 +194,7 @@ func (p *scanRequestScanner) exportSpan( func getSpansToProcess( ctx context.Context, ds *kvcoord.DistSender, targetSpans []roachpb.Span, ) ([]roachpb.Span, error) { - ranges, err := allRangeSpans(ctx, ds, targetSpans) + ranges, err := AllRangeSpans(ctx, ds, targetSpans) if err != nil { return nil, err } @@ -261,7 +261,8 @@ func slurpScanResponse( return nil } -func allRangeSpans( +// AllRangeSpans returns the list of all ranges that for the specified list of spans. +func AllRangeSpans( ctx context.Context, ds *kvcoord.DistSender, spans []roachpb.Span, ) ([]roachpb.Span, error) { From aecb0a83be69bf04776bc72f818da430531cd5fe Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Tue, 27 Sep 2022 10:53:22 -0400 Subject: [PATCH 10/13] changefeedccl: Fix array encoding avro bug. Fix latent array avro encoding bug where previously allocated memo array might contain 'nil' element, while the code assumed that the element must always be a map. Release note: none. --- pkg/ccl/changefeedccl/avro.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/ccl/changefeedccl/avro.go b/pkg/ccl/changefeedccl/avro.go index 23059ff791e5..e6a81150daeb 100644 --- a/pkg/ccl/changefeedccl/avro.go +++ b/pkg/ccl/changefeedccl/avro.go @@ -607,6 +607,7 @@ func typeToAvroSchema(typ *types.T) (*avroSchemaField, error) { }, func(d tree.Datum, memo interface{}) (interface{}, error) { datumArr := d.(*tree.DArray) + var avroArr []interface{} if memo != nil { avroArr = memo.([]interface{}) @@ -616,14 +617,13 @@ func typeToAvroSchema(typ *types.T) (*avroSchemaField, error) { } else { avroArr = make([]interface{}, 0, datumArr.Len()) } - for i, elt := range datumArr.Array { var encoded interface{} if elt == tree.DNull { encoded = nil } else { var encErr error - if i < len(avroArr) { + if i < len(avroArr) && avroArr[i] != nil { encoded, encErr = itemSchema.encodeDatum(elt, avroArr[i].(map[string]interface{})[itemUnionKey]) } else { encoded, encErr = itemSchema.encodeDatum(elt, nil) From d8f371204566361bfd9e0794bd9f75d108b3bb92 Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Mon, 10 Oct 2022 10:48:08 -0400 Subject: [PATCH 11/13] changefeedccl: use numcpu >> 2 workers for event consumers Previously, a default value of 8 was used for the kvevent parallel consumer. The reason for this was that we observed performance improvements in a 15 node 32 VCPU cluster when we increased this parameter to 8. After 8, the improvements were much smaller. The issue with a default of 8 is that that on smaller machines, 8 workers can be too much overhead, especially since the work is CPU intensive. This change updates the default to be runtime.NumCPU() >> 2 workers, which aligns with using 8 workers on 32 VCPU machines. Fixes https://github.com/cockroachdb/cockroach/issues/89589 Epic: none Release note: None --- docs/generated/settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/ccl/changefeedccl/changefeedbase/settings.go | 6 +++--- pkg/ccl/changefeedccl/encoder_test.go | 2 +- pkg/ccl/changefeedccl/event_processing.go | 16 ++++++++++++++++ 5 files changed, 22 insertions(+), 6 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 102b89e9da19..8de6a8f52915 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -12,7 +12,7 @@ bulkio.backup.read_with_priority_after duration 1m0s amount of time since the re bulkio.stream_ingestion.minimum_flush_interval duration 5s the minimum timestamp between flushes; flushes may still occur if internal buffers fill up changefeed.balance_range_distribution.enable boolean false if enabled, the ranges are balanced equally among all nodes changefeed.event_consumer_worker_queue_size integer 16 if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of eventswhich a worker can buffer -changefeed.event_consumer_workers integer 8 the number of workers to use when processing events; 0 or 1 disables +changefeed.event_consumer_workers integer 0 the number of workers to use when processing events: <0 disables, 0 assigns a reasonable default, >0 assigns the setting value changefeed.fast_gzip.enabled boolean true use fast gzip implementation changefeed.node_throttle_config string specifies node level throttling configuration for all changefeeeds changefeed.schema_feed.read_with_priority_after duration 1m0s retry with high priority if we were not able to read descriptors for too long; 0 disables diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 7c694271908a..2d21f8281bfe 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -18,7 +18,7 @@ bulkio.stream_ingestion.minimum_flush_intervalduration5sthe minimum timestamp between flushes; flushes may still occur if internal buffers fill up changefeed.balance_range_distribution.enablebooleanfalseif enabled, the ranges are balanced equally among all nodes changefeed.event_consumer_worker_queue_sizeinteger16if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of eventswhich a worker can buffer -changefeed.event_consumer_workersinteger8the number of workers to use when processing events; 0 or 1 disables +changefeed.event_consumer_workersinteger0the number of workers to use when processing events: <0 disables, 0 assigns a reasonable default, >0 assigns the setting value changefeed.fast_gzip.enabledbooleantrueuse fast gzip implementation changefeed.node_throttle_configstringspecifies node level throttling configuration for all changefeeeds changefeed.schema_feed.read_with_priority_afterduration1m0sretry with high priority if we were not able to read descriptors for too long; 0 disables diff --git a/pkg/ccl/changefeedccl/changefeedbase/settings.go b/pkg/ccl/changefeedccl/changefeedbase/settings.go index a66f1a136989..5169dc1aadce 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/settings.go +++ b/pkg/ccl/changefeedccl/changefeedbase/settings.go @@ -238,9 +238,9 @@ var UseMuxRangeFeed = settings.RegisterBoolSetting( var EventConsumerWorkers = settings.RegisterIntSetting( settings.TenantWritable, "changefeed.event_consumer_workers", - "the number of workers to use when processing events; 0 or 1 disables", - int64(util.ConstantWithMetamorphicTestRange("changefeed.consumer_max_workers", 8, 0, 32)), - settings.NonNegativeInt, + "the number of workers to use when processing events: <0 disables, "+ + "0 assigns a reasonable default, >0 assigns the setting value", + 0, ).WithPublic() // EventConsumerWorkerQueueSize specifies the maximum number of events a worker buffer. diff --git a/pkg/ccl/changefeedccl/encoder_test.go b/pkg/ccl/changefeedccl/encoder_test.go index 983a1b47a731..eaf680bd63f7 100644 --- a/pkg/ccl/changefeedccl/encoder_test.go +++ b/pkg/ccl/changefeedccl/encoder_test.go @@ -607,7 +607,7 @@ func TestAvroSchemaNaming(t *testing.T) { // workers, the results below may change, so disable parallel workers // here for simplicity. changefeedbase.EventConsumerWorkers.Override( - context.Background(), &s.Server.ClusterSettings().SV, 0) + context.Background(), &s.Server.ClusterSettings().SV, -1) sqlDB.Exec(t, `CREATE DATABASE movr`) sqlDB.Exec(t, `CREATE TABLE movr.drivers (id INT PRIMARY KEY, name STRING)`) diff --git a/pkg/ccl/changefeedccl/event_processing.go b/pkg/ccl/changefeedccl/event_processing.go index 74b3ba6129ed..082ee60002f6 100644 --- a/pkg/ccl/changefeedccl/event_processing.go +++ b/pkg/ccl/changefeedccl/event_processing.go @@ -12,6 +12,7 @@ import ( "context" "hash" "hash/crc32" + "runtime" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdceval" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" @@ -103,6 +104,10 @@ func newEventConsumer( // TODO (jayshrivastava) enable parallel consumers for sinkless changefeeds numWorkers := changefeedbase.EventConsumerWorkers.Get(&cfg.Settings.SV) + if numWorkers == 0 { + // Pick a reasonable default. + numWorkers = defaultNumWorkers() + } if numWorkers <= 1 || isSinkless { c, err := makeConsumer(sink, spanFrontier) if err != nil { @@ -134,6 +139,17 @@ func newEventConsumer( return c, ss, nil } +func defaultNumWorkers() int64 { + idealNumber := runtime.GOMAXPROCS(0) >> 2 + if idealNumber < 1 { + return 1 + } + if idealNumber > 8 { + return 8 + } + return int64(idealNumber) +} + func makeHasher() hash.Hash32 { return crc32.New(crc32.MakeTable(crc32.IEEE)) } From bc90f53240258c249feaee0cd25387d5e71d4304 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Thu, 29 Sep 2022 07:43:34 -0400 Subject: [PATCH 12/13] changefeedccl: Release overhead allocation To account for the overhead during the encoding stage, we allocate more memory resources than needed (`changefeed.event_memory_multiplier`). Those resources are released when they are amitted by the sink. Some sinks, such as file based sinks, batch many such events (to generate files of target size). This batching, when combined with compression, could result in a situation where maximum of allowed resources, `changefeed.memory.per_changefeed_limit` are all batched, making it impossible to ingest additional events without forcing a sink flush. The only way to avoid premature forced flush, is to increase the memory limit for changefeed -- but doing so is not without the costs (i.e. more events batched, more pressure on Go GC, etc). This PR adjust event resources to match the final size of the data that will be emitted into the sink -- that is, we release the overhead back into the pool, once the event processing is done. Release note: none --- pkg/ccl/changefeedccl/event_processing.go | 9 ++++-- pkg/ccl/changefeedccl/kvevent/BUILD.bazel | 1 + pkg/ccl/changefeedccl/kvevent/alloc.go | 11 +++++++ pkg/ccl/changefeedccl/kvevent/alloc_test.go | 33 +++++++++++++++++---- 4 files changed, 46 insertions(+), 8 deletions(-) diff --git a/pkg/ccl/changefeedccl/event_processing.go b/pkg/ccl/changefeedccl/event_processing.go index 082ee60002f6..7af53ebfccb2 100644 --- a/pkg/ccl/changefeedccl/event_processing.go +++ b/pkg/ccl/changefeedccl/event_processing.go @@ -335,9 +335,14 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even return err } } + + // Since we're done processing/converting this event, and will not use much more + // than len(key)+len(bytes) worth of resources, adjust allocation to match. + a := ev.DetachAlloc() + a.AdjustBytesToTarget(ctx, int64(len(keyCopy)+len(valueCopy))) + if err := c.sink.EmitRow( - ctx, topic, - keyCopy, valueCopy, schemaTimestamp, mvccTimestamp, ev.DetachAlloc(), + ctx, topic, keyCopy, valueCopy, schemaTimestamp, mvccTimestamp, a, ); err != nil { return err } diff --git a/pkg/ccl/changefeedccl/kvevent/BUILD.bazel b/pkg/ccl/changefeedccl/kvevent/BUILD.bazel index 2d9c2ce3a3ac..35052ab74d33 100644 --- a/pkg/ccl/changefeedccl/kvevent/BUILD.bazel +++ b/pkg/ccl/changefeedccl/kvevent/BUILD.bazel @@ -63,6 +63,7 @@ go_test( "//pkg/util/quotapool", "//pkg/util/randutil", "//pkg/util/syncutil", + "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", ], diff --git a/pkg/ccl/changefeedccl/kvevent/alloc.go b/pkg/ccl/changefeedccl/kvevent/alloc.go index ba63a8b4d7b9..cf3566fb4630 100644 --- a/pkg/ccl/changefeedccl/kvevent/alloc.go +++ b/pkg/ccl/changefeedccl/kvevent/alloc.go @@ -46,6 +46,17 @@ func (a *Alloc) Release(ctx context.Context) { a.clear() } +// AdjustBytesToTarget adjust byte allocation to the specified target. +// Target bytes cannot be adjusted to the higher level than the current allocation. +func (a *Alloc) AdjustBytesToTarget(ctx context.Context, targetBytes int64) { + if a.isZero() || targetBytes <= 0 || targetBytes >= a.bytes { + return + } + toRelease := a.bytes - targetBytes + a.bytes = targetBytes + a.ap.Release(ctx, toRelease, 0) +} + // Bytes returns the size of this alloc in bytes. func (a *Alloc) Bytes() int64 { return a.bytes diff --git a/pkg/ccl/changefeedccl/kvevent/alloc_test.go b/pkg/ccl/changefeedccl/kvevent/alloc_test.go index 3d4d50294a96..9c899e403a24 100644 --- a/pkg/ccl/changefeedccl/kvevent/alloc_test.go +++ b/pkg/ccl/changefeedccl/kvevent/alloc_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -36,7 +37,7 @@ func TestAllocMergeRandomized(t *testing.T) { // Allocate N allocs from the P pools. poolPerm := rand.Perm(P) for i := range allocs { - allocs[i] = pools[poolPerm[i%P]].alloc() + allocs[i] = pools[poolPerm[i%P]].alloc(1) } // Randomly merge the allocs together. @@ -82,6 +83,26 @@ func TestAllocMergeRandomized(t *testing.T) { } } +func TestAllocAdjust(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + p := &testAllocPool{} + a := p.alloc(10) + require.EqualValues(t, 10, a.Bytes()) + require.EqualValues(t, 1, a.Events()) + a.AdjustBytesToTarget(ctx, 6) + require.EqualValues(t, 6, a.Bytes()) + a.AdjustBytesToTarget(ctx, 7) // no-op + require.EqualValues(t, 6, a.Bytes()) + a.AdjustBytesToTarget(ctx, -5) // no-op + require.EqualValues(t, 6, a.Bytes()) + a.AdjustBytesToTarget(ctx, 1) + require.EqualValues(t, 1, a.Bytes()) + a.Release(ctx) +} + type testAllocPool struct { syncutil.Mutex n int64 @@ -91,17 +112,17 @@ type testAllocPool struct { func (ap *testAllocPool) Release(ctx context.Context, bytes, entries int64) { ap.Lock() defer ap.Unlock() - if ap.n == 0 { - panic("can't release zero resources") + if ap.n < bytes { + panic(errors.AssertionFailedf("can't release %d bytes from zero resources", bytes)) } ap.n -= bytes } -func (ap *testAllocPool) alloc() Alloc { +func (ap *testAllocPool) alloc(bytes int64) Alloc { ap.Lock() defer ap.Unlock() - ap.n++ - return TestingMakeAlloc(1, ap) + ap.n += bytes + return TestingMakeAlloc(bytes, ap) } func (ap *testAllocPool) getN() int { From 3cf556547feae7c437a9786c0f3198f013be6581 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Fri, 14 Oct 2022 16:39:58 -0400 Subject: [PATCH 13/13] changefeedccl: Ensure correct file ordering. When async flushing enabled, the following sequence of events is possible (even if very unlikely): * k@t1 is emitted, causing async flush to write file f1 * k@t2 is emitted, causing async flush to write file f2 * f2 is written out before f1. In this unlikely scenario -- and the reason why it's unlikely is that we have to generate megabytes of data to cause a flush, unless, file_size parameter was pathologically small -- if a client were to read the contents of the directory right after step 2, and then read directory after step 3, the client will first observe k@t2, and then observe k@t1 -- which is a violation of ordering guarantees. This PR fixes this issue by adopting a queue so that if there is a pending flush in flight, the next flush is queued behind. It is possible that this simple approach may result in throughput decrease. This situation will occur if the underlying storage (s3 or gcp) cannot keep up with writing out data before the next file comes in. However, at this point, such situation is unlikely for two reasons: one, the rest of changefeed machinery must be able to generate one or more files worth of data before the previous flush completes -- and this task in of itself is not trivial, and two, changefeed must have enough memory allocated to it so that pushback mechanism does not trigger. The above assumption was validated in reasonably sized test -- i.e. the non-zero queue depth was never observed. Nonetheless, this PR also adds a log message which may be helpful to detect the situation when the sink might not keep up with the incoming data rate. A more complex solution -- for example, allow unlimited inflight requests during backfill -- may be revisited later, if the above assumption proven incorrect. Fixes #89683 Release note: None. --- pkg/ccl/changefeedccl/sink_cloudstorage.go | 142 ++++++++++++++---- .../changefeedccl/sink_cloudstorage_test.go | 4 +- 2 files changed, 115 insertions(+), 31 deletions(-) diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage.go b/pkg/ccl/changefeedccl/sink_cloudstorage.go index fbc6262255f7..2176bcc13d21 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage.go @@ -18,7 +18,6 @@ import ( "net/url" "path/filepath" "strings" - "sync" "sync/atomic" "time" @@ -31,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -310,9 +310,16 @@ type cloudStorageSink struct { metrics metricsRecorder asyncFlushActive bool - flushCtx context.Context - flushGroup sync.WaitGroup - flushErr atomic.Value + flushGroup ctxgroup.Group + asyncFlushCh chan flushRequest // channel for submitting flush requests. + asyncFlushTermCh chan struct{} // channel closed by async flusher to indicate an error + asyncFlushErr error // set by async flusher, prior to closing asyncFlushTermCh +} + +type flushRequest struct { + file *cloudStorageSinkFile + dest string + flush chan struct{} } var cloudStorageSinkIDAtomic int64 @@ -328,6 +335,16 @@ var partitionDateFormats = map[string]string{ } var defaultPartitionFormat = partitionDateFormats["daily"] +// flushQueueDepth puts a limit on how many flush requests +// may be outstanding, before we block. +// In reality, we will block much sooner than this limit due +// to blocking buffer memory limits (in its default configuration); +// We just want this setting to be sufficiently large, but not +// so large as to have extremely large flush queues. +// The default of 256, with the default file size of 16MB, gives us +// a queue of 2.5GB of outstanding flush data. +const flushQueueDepth = 256 + func makeCloudStorageSink( ctx context.Context, u sinkURL, @@ -375,8 +392,11 @@ func makeCloudStorageSink( topicNamer: tn, asyncFlushActive: enableAsyncFlush.Get(&settings.SV), // TODO (yevgeniy): Consider adding ctx to Dial method instead. - flushCtx: ctx, + flushGroup: ctxgroup.WithContext(ctx), + asyncFlushCh: make(chan flushRequest, flushQueueDepth), + asyncFlushTermCh: make(chan struct{}), } + s.flushGroup.GoCtx(s.asyncFlusher) if partitionFormat := u.consumeParam(changefeedbase.SinkParamPartitionFormat); partitionFormat != "" { dateFormat, ok := partitionDateFormats[partitionFormat] @@ -388,8 +408,7 @@ func makeCloudStorageSink( } if s.timestampOracle != nil { - s.dataFileTs = cloudStorageFormatTime(s.timestampOracle.inclusiveLowerBoundTS()) - s.dataFilePartition = s.timestampOracle.inclusiveLowerBoundTS().GoTime().Format(s.partitionFormat) + s.setDataFileTimestamp() } switch encodingOpts.Format { @@ -523,7 +542,7 @@ func (s *cloudStorageSink) EmitResolvedTimestamp( // Wait for previously issued async flush requests to complete // before we write resolved time stamp file. - if err := s.waitAsyncFlush(); err != nil { + if err := s.waitAsyncFlush(ctx); err != nil { return errors.Wrapf(err, "while emitting resolved timestamp") } @@ -590,13 +609,17 @@ func (s *cloudStorageSink) Flush(ctx context.Context) error { return err } s.files.Clear(true /* addNodesToFreeList */) + s.setDataFileTimestamp() + return s.waitAsyncFlush(ctx) +} +func (s *cloudStorageSink) setDataFileTimestamp() { // Record the least resolved timestamp being tracked in the frontier as of this point, // to use for naming files until the next `Flush()`. See comment on cloudStorageSink // for an overview of the naming convention and proof of correctness. - s.dataFileTs = cloudStorageFormatTime(s.timestampOracle.inclusiveLowerBoundTS()) - s.dataFilePartition = s.timestampOracle.inclusiveLowerBoundTS().GoTime().Format(s.partitionFormat) - return s.waitAsyncFlush() + ts := s.timestampOracle.inclusiveLowerBoundTS() + s.dataFileTs = cloudStorageFormatTime(ts) + s.dataFilePartition = ts.GoTime().Format(s.partitionFormat) } // enableAsyncFlush controls async flushing behavior for this sink. @@ -608,14 +631,28 @@ var enableAsyncFlush = settings.RegisterBoolSetting( ) // waitAsyncFlush waits until all async flushes complete. -func (s *cloudStorageSink) waitAsyncFlush() error { - s.flushGroup.Wait() - if v := s.flushErr.Load(); v != nil { - return v.(error) +func (s *cloudStorageSink) waitAsyncFlush(ctx context.Context) error { + done := make(chan struct{}) + select { + case <-ctx.Done(): + return ctx.Err() + case <-s.asyncFlushTermCh: + return s.asyncFlushErr + case s.asyncFlushCh <- flushRequest{flush: done}: + } + + select { + case <-ctx.Done(): + return ctx.Err() + case <-s.asyncFlushTermCh: + return s.asyncFlushErr + case <-done: + return nil } - return nil } +var logQueueDepth = log.Every(30 * time.Second) + // flushFile flushes file to the cloud storage. // file should not be used after flushing. func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSinkFile) error { @@ -623,7 +660,7 @@ func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSink if s.asyncFlushActive && !asyncFlushEnabled { // Async flush behavior was turned off -- drain any active flush requests // before flushing this file. - if err := s.waitAsyncFlush(); err != nil { + if err := s.waitAsyncFlush(ctx); err != nil { return err } } @@ -651,21 +688,63 @@ func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSink return file.flushToStorage(ctx, s.es, dest, s.metrics) } - s.flushGroup.Add(1) - go func() { - defer s.flushGroup.Done() - // NB: must use s.flushCtx; ctx may be short lived (i.e. cancelled). - if err := file.flushToStorage(s.flushCtx, s.es, dest, s.metrics); err != nil { - log.Errorf(ctx, "error flushing file to storage: %s", err) - // We must use the same type for error we store in flushErr. - s.flushErr.CompareAndSwap(nil, &flushError{error: err}) + // Try to submit flush request, but produce warning message + // if we can't. + select { + case <-ctx.Done(): + return ctx.Err() + case <-s.asyncFlushTermCh: + return s.asyncFlushErr + case s.asyncFlushCh <- flushRequest{file: file, dest: dest}: + return nil + default: + if logQueueDepth.ShouldLog() { + log.Infof(ctx, "changefeed flush queue is full; ~%d bytes to flush", + flushQueueDepth*s.targetMaxFileSize) } - }() - return nil + } + + // Queue was full, block until it's not. + select { + case <-ctx.Done(): + return ctx.Err() + case <-s.asyncFlushTermCh: + return s.asyncFlushErr + case s.asyncFlushCh <- flushRequest{file: file, dest: dest}: + return nil + } } -type flushError struct { - error +func (s *cloudStorageSink) asyncFlusher(ctx context.Context) error { + defer close(s.asyncFlushTermCh) + + for { + select { + case <-ctx.Done(): + return ctx.Err() + case req, ok := <-s.asyncFlushCh: + if !ok { + return nil // we're done + } + + // handle flush request. + if req.flush != nil { + close(req.flush) + continue + } + + // flush file to storage. + flushDone := s.metrics.recordFlushRequestCallback() + err := req.file.flushToStorage(ctx, s.es, req.dest, s.metrics) + flushDone() + + if err != nil { + log.Errorf(ctx, "error flushing file to storage: %s", err) + s.asyncFlushErr = err + return err + } + } + } } // flushToStorage writes out file into external storage into 'dest'. @@ -698,7 +777,10 @@ func (f *cloudStorageSinkFile) flushToStorage( // Close implements the Sink interface. func (s *cloudStorageSink) Close() error { s.files = nil - return errors.CombineErrors(s.waitAsyncFlush(), s.es.Close()) + err := s.waitAsyncFlush(context.Background()) + close(s.asyncFlushCh) // signal flusher to exit. + err = errors.CombineErrors(err, s.flushGroup.Wait()) + return errors.CombineErrors(err, s.es.Close()) } // Dial implements the Sink interface. diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go index e925476068ca..f545dc03853d 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go @@ -197,7 +197,9 @@ func TestCloudStorageSink(t *testing.T) { testWithAndWithoutAsyncFlushing := func(t *testing.T, name string, testFn func(*testing.T)) { t.Helper() testutils.RunTrueAndFalse(t, name+"/asyncFlush", func(t *testing.T, enable bool) { + old := enableAsyncFlush.Get(&settings.SV) enableAsyncFlush.Override(context.Background(), &settings.SV, enable) + defer enableAsyncFlush.Override(context.Background(), &settings.SV, old) testFn(t) }) } @@ -460,7 +462,7 @@ func TestCloudStorageSink(t *testing.T) { }) waitAsyncFlush := func(s Sink) error { - return s.(*cloudStorageSink).waitAsyncFlush() + return s.(*cloudStorageSink).waitAsyncFlush(context.Background()) } testWithAndWithoutAsyncFlushing(t, `bucketing`, func(t *testing.T) { t1 := makeTopic(`t1`)