From 318eaf1749322d34a585bb500389dc985992b015 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 2 Jan 2025 15:44:59 -0500 Subject: [PATCH 1/8] opt/exec: split plan gist factory and decoder The plan gist factory is no longer used to decode plan gists. The `planGistDecoder` has been introduced to fulfill this role. Release note: None --- pkg/sql/opt/exec/explain/plan_gist_factory.go | 129 +++++++++--------- .../optgen/cmd/optgen/exec_plan_gist_gen.go | 8 +- .../optgen/cmd/optgen/testdata/execplangist | 24 ++-- 3 files changed, 82 insertions(+), 79 deletions(-) diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index 9bb20987035a..ddd1e3600673 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -89,21 +89,13 @@ func (fp PlanGist) Hash() uint64 { type PlanGistFactory struct { wrappedFactory exec.Factory - // buffer is used for reading and writing (i.e. decoding and encoding). - // Data that is written to the buffer is also added to the hash. The - // exception is when we're dealing with ids where we will write the id to the - // buffer and the "string" to the hash. This allows the hash to be id agnostic - // (ie hash's will be stable across plans from different databases with - // different DDL history). + // buffer accumulates the encoded gist. Data that is written to the buffer + // is also added to the hash. The exception is when we're dealing with ids + // where we will write the id to the buffer and the "string" to the hash. + // This allows the hash to be id agnostic (ie hash's will be stable across + // plans from different databases with different DDL history). buffer bytes.Buffer hash util.FNV64 - - nodeStack []*Node - - // catalog is used to resolve table and index ids that are stored in the gist. - // catalog can be nil when decoding gists via decode_external_plan_gist in which - // case we don't attempt to resolve tables or indexes. - catalog cat.Catalog } var _ exec.Factory = &PlanGistFactory{} @@ -155,6 +147,17 @@ func (f *PlanGistFactory) PlanGist() PlanGist { hash: f.hash.Sum()} } +// planGistDecoder is used to decode a plan gist into a logical plan. +type planGistDecoder struct { + buffer bytes.Buffer + nodeStack []*Node + + // catalog is used to resolve table and index ids that are stored in the gist. + // catalog can be nil when decoding gists via decode_external_plan_gist in which + // case we don't attempt to resolve tables or indexes. + catalog cat.Catalog +} + // DecodePlanGistToRows converts a gist to a logical plan and returns the rows. func DecodePlanGistToRows( ctx context.Context, evalCtx *eval.Context, gist string, catalog cat.Catalog, @@ -189,39 +192,38 @@ func DecodePlanGistToRows( } // DecodePlanGistToPlan constructs an explain.Node tree from a gist. -func DecodePlanGistToPlan(s string, cat cat.Catalog) (plan *Plan, retErr error) { - f := NewPlanGistFactory(exec.StubFactory{}) - f.catalog = cat - bytes, err := base64.StdEncoding.DecodeString(s) +func DecodePlanGistToPlan(s string, cat cat.Catalog) (plan *Plan, err error) { + d := planGistDecoder{ + catalog: cat, + } + b, err := base64.StdEncoding.DecodeString(s) if err != nil { return nil, err } - // Clear out buffer which will have version in it from NewPlanGistFactory. - f.buffer.Reset() - f.buffer.Write(bytes) + d.buffer.Write(b) plan = &Plan{} - ver := f.decodeInt() + ver := d.decodeInt() if ver != gistVersion { return nil, errors.Errorf("unsupported gist version %d (expected %d)", ver, gistVersion) } for { - op := f.decodeOp() + op := d.decodeOp() if op == unknownOp { break } switch op { case errorIfRowsOp: - plan.Checks = append(plan.Checks, f.popChild()) + plan.Checks = append(plan.Checks, d.popChild()) } } - plan.Root = f.popChild() + plan.Root = d.popChild() - for _, n := range f.nodeStack { + for _, n := range d.nodeStack { subquery := exec.Subquery{ Root: n, } @@ -231,27 +233,27 @@ func DecodePlanGistToPlan(s string, cat cat.Catalog) (plan *Plan, retErr error) return plan, nil } -func (f *PlanGistFactory) decodeOp() execOperator { - val, err := f.buffer.ReadByte() +func (d *planGistDecoder) decodeOp() execOperator { + val, err := d.buffer.ReadByte() if err != nil || val == 0 { return unknownOp } - n, err := f.decodeOperatorBody(execOperator(val)) + n, err := d.decodeOperatorBody(execOperator(val)) if err != nil { panic(err) } - f.nodeStack = append(f.nodeStack, n) + d.nodeStack = append(d.nodeStack, n) return n.op } -func (f *PlanGistFactory) popChild() *Node { - l := len(f.nodeStack) +func (d *planGistDecoder) popChild() *Node { + l := len(d.nodeStack) if l == 0 { return nil } - n := f.nodeStack[l-1] - f.nodeStack = f.nodeStack[:l-1] + n := d.nodeStack[l-1] + d.nodeStack = d.nodeStack[:l-1] return n } @@ -266,8 +268,8 @@ func (f *PlanGistFactory) encodeInt(i int) { f.writeAndHash(buf[:n]) } -func (f *PlanGistFactory) decodeInt() int { - val, err := binary.ReadVarint(&f.buffer) +func (d *planGistDecoder) decodeInt() int { + val, err := binary.ReadVarint(&d.buffer) if err != nil { panic(err) } @@ -291,16 +293,17 @@ func (f *PlanGistFactory) encodeID(id cat.StableID) { f.encodeInt(int(id)) } -func (f *PlanGistFactory) decodeID() cat.StableID { - return cat.StableID(f.decodeInt()) +func (d *planGistDecoder) decodeID() cat.StableID { + return cat.StableID(d.decodeInt()) } -func (f *PlanGistFactory) decodeTable() cat.Table { - if f.catalog == nil { +func (d *planGistDecoder) decodeTable() cat.Table { + if d.catalog == nil { return &unknownTable{} } - id := f.decodeID() - ds, _, err := f.catalog.ResolveDataSourceByID(f.Ctx(), cat.Flags{}, id) + id := d.decodeID() + // TODO(mgartner): Do not use the background context. + ds, _, err := d.catalog.ResolveDataSourceByID(context.Background(), cat.Flags{}, id) if err == nil { return ds.(cat.Table) } @@ -310,8 +313,8 @@ func (f *PlanGistFactory) decodeTable() cat.Table { panic(err) } -func (f *PlanGistFactory) decodeIndex(tbl cat.Table) cat.Index { - id := f.decodeID() +func (d *planGistDecoder) decodeIndex(tbl cat.Table) cat.Index { + id := d.decodeID() for i, n := 0, tbl.IndexCount(); i < n; i++ { if tbl.Index(i).ID() == id { return tbl.Index(i) @@ -321,8 +324,8 @@ func (f *PlanGistFactory) decodeIndex(tbl cat.Table) cat.Index { } // TODO: implement this and figure out how to test... -func (f *PlanGistFactory) decodeSchema() cat.Schema { - id := f.decodeID() +func (d *planGistDecoder) decodeSchema() cat.Schema { + id := d.decodeID() _ = id return nil } @@ -331,8 +334,8 @@ func (f *PlanGistFactory) encodeNodeColumnOrdinals(vals []exec.NodeColumnOrdinal f.encodeInt(len(vals)) } -func (f *PlanGistFactory) decodeNodeColumnOrdinals() []exec.NodeColumnOrdinal { - l := f.decodeInt() +func (d *planGistDecoder) decodeNodeColumnOrdinals() []exec.NodeColumnOrdinal { + l := d.decodeInt() if l < 0 { return nil } @@ -344,8 +347,8 @@ func (f *PlanGistFactory) encodeResultColumns(vals colinfo.ResultColumns) { f.encodeInt(len(vals)) } -func (f *PlanGistFactory) decodeResultColumns() colinfo.ResultColumns { - numCols := f.decodeInt() +func (d *planGistDecoder) decodeResultColumns() colinfo.ResultColumns { + numCols := d.decodeInt() if numCols < 0 { return nil } @@ -357,16 +360,16 @@ func (f *PlanGistFactory) encodeByte(b byte) { f.hash.Add(uint64(b)) } -func (f *PlanGistFactory) decodeByte() byte { - val, err := f.buffer.ReadByte() +func (d *planGistDecoder) decodeByte() byte { + val, err := d.buffer.ReadByte() if err != nil { panic(err) } return val } -func (f *PlanGistFactory) decodeJoinType() descpb.JoinType { - val := f.decodeByte() +func (d *planGistDecoder) decodeJoinType() descpb.JoinType { + val := d.decodeByte() return descpb.JoinType(val) } @@ -378,8 +381,8 @@ func (f *PlanGistFactory) encodeBool(b bool) { } } -func (f *PlanGistFactory) decodeBool() bool { - val := f.decodeByte() +func (d *planGistDecoder) decodeBool() bool { + val := d.decodeByte() return val != 0 } @@ -395,7 +398,7 @@ func (f *PlanGistFactory) encodeFastIntSet(s intsets.Fast) { func (f *PlanGistFactory) encodeColumnOrdering(cols colinfo.ColumnOrdering) { } -func (f *PlanGistFactory) decodeColumnOrdering() colinfo.ColumnOrdering { +func (d *planGistDecoder) decodeColumnOrdering() colinfo.ColumnOrdering { return nil } @@ -427,15 +430,15 @@ func (f *PlanGistFactory) encodeScanParams(params exec.ScanParams) { } } -func (f *PlanGistFactory) decodeScanParams() exec.ScanParams { +func (d *planGistDecoder) decodeScanParams() exec.ScanParams { neededCols := intsets.Fast{} - err := neededCols.Decode(&f.buffer) + err := neededCols.Decode(&d.buffer) if err != nil { panic(err) } var idxConstraint *constraint.Constraint - l := f.decodeInt() + l := d.decodeInt() if l > 0 { idxConstraint = new(constraint.Constraint) idxConstraint.Spans.Alloc(l) @@ -444,12 +447,12 @@ func (f *PlanGistFactory) decodeScanParams() exec.ScanParams { } var invertedConstraint inverted.Spans - l = f.decodeInt() + l = d.decodeInt() if l > 0 { invertedConstraint = make([]inverted.Span, l) } - hardLimit := f.decodeInt() + hardLimit := d.decodeInt() // Since we no longer record the limit value and its just a bool tell the emit code // to just print "limit", instead the misleading "limit: 1". @@ -469,8 +472,8 @@ func (f *PlanGistFactory) encodeRows(rows [][]tree.TypedExpr) { f.encodeInt(len(rows)) } -func (f *PlanGistFactory) decodeRows() [][]tree.TypedExpr { - numRows := f.decodeInt() +func (d *planGistDecoder) decodeRows() [][]tree.TypedExpr { + numRows := d.decodeInt() if numRows < 0 { return nil } diff --git a/pkg/sql/opt/optgen/cmd/optgen/exec_plan_gist_gen.go b/pkg/sql/opt/optgen/cmd/optgen/exec_plan_gist_gen.go index a9773346f9d6..5164359bcd1a 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/exec_plan_gist_gen.go +++ b/pkg/sql/opt/optgen/cmd/optgen/exec_plan_gist_gen.go @@ -132,7 +132,7 @@ func (g *execPlanGistGen) genPlanGistFactory() { func (g *execPlanGistGen) genPlanGistDecoder() { g.w.write("\n") - g.w.nest("func (f *PlanGistFactory) decodeOperatorBody(op execOperator) (*Node, error) {\n") + g.w.nest("func (d *planGistDecoder) decodeOperatorBody(op execOperator) (*Node, error) {\n") g.w.writeIndent("var _n *Node\n") g.w.writeIndent("var reqOrdering exec.OutputOrdering\n") g.w.writeIndent("var err error\n") @@ -144,7 +144,7 @@ func (g *execPlanGistGen) genPlanGistDecoder() { g.w.nestIndent("var args %sArgs\n", unTitle(string(define.Name))) // table is implicit if strings.HasPrefix(string(define.Name), "AlterTable") { - g.w.writeIndent("tbl := f.decodeTable()\n") + g.w.writeIndent("tbl := d.decodeTable()\n") } for f, field := range define.Fields { if omitted(string(define.Name), string(field.Name)) { @@ -184,7 +184,7 @@ func (g *execPlanGistGen) genPlanGistDecoder() { } if len(decoder) > 0 { - g.w.writeIndent("args.%s = f.%s(%s)\n", argName, decoder, decoderArg) + g.w.writeIndent("args.%s = d.%s(%s)\n", argName, decoder, decoderArg) } if len(store) > 0 { g.w.writeIndent("%s = args.%s\n", store, argName) @@ -198,7 +198,7 @@ func (g *execPlanGistGen) genPlanGistDecoder() { for i := len(childrenNames) - 1; i >= 0; i-- { childrenNames[i] = "args." + childrenNames[i] - g.w.writeIndent("%s = f.popChild()\n", childrenNames[i]) + g.w.writeIndent("%s = d.popChild()\n", childrenNames[i]) } g.w.writeIndent("_n, err = newNode(op, &args, reqOrdering, %s)\n", strings.Join(childrenNames, ",")) diff --git a/pkg/sql/opt/optgen/cmd/optgen/testdata/execplangist b/pkg/sql/opt/optgen/cmd/optgen/testdata/execplangist index 063fdc5d1059..c07ef76c20cd 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/testdata/execplangist +++ b/pkg/sql/opt/optgen/cmd/optgen/testdata/execplangist @@ -105,7 +105,7 @@ func (f *PlanGistFactory) ConstructHashJoin( return node, err } -func (f *PlanGistFactory) decodeOperatorBody(op execOperator) (*Node, error) { +func (d *planGistDecoder) decodeOperatorBody(op execOperator) (*Node, error) { var _n *Node var reqOrdering exec.OutputOrdering var err error @@ -113,24 +113,24 @@ func (f *PlanGistFactory) decodeOperatorBody(op execOperator) (*Node, error) { switch op { case scanOp: var args scanArgs - args.Table = f.decodeTable() + args.Table = d.decodeTable() tbl = args.Table - args.Index = f.decodeIndex(tbl) - args.Params = f.decodeScanParams() + args.Index = d.decodeIndex(tbl) + args.Params = d.decodeScanParams() _n, err = newNode(op, &args, reqOrdering) case filterOp: var args filterArgs - args.Input = f.popChild() + args.Input = d.popChild() _n, err = newNode(op, &args, reqOrdering, args.Input) case hashJoinOp: var args hashJoinArgs - args.JoinType = f.decodeJoinType() - args.LeftEqCols = f.decodeNodeColumnOrdinals() - args.RightEqCols = f.decodeNodeColumnOrdinals() - args.LeftEqColsAreKey = f.decodeBool() - args.RightEqColsAreKey = f.decodeBool() - args.Right = f.popChild() - args.Left = f.popChild() + args.JoinType = d.decodeJoinType() + args.LeftEqCols = d.decodeNodeColumnOrdinals() + args.RightEqCols = d.decodeNodeColumnOrdinals() + args.LeftEqColsAreKey = d.decodeBool() + args.RightEqColsAreKey = d.decodeBool() + args.Right = d.popChild() + args.Left = d.popChild() _n, err = newNode(op, &args, reqOrdering, args.Left, args.Right) default: return nil, errors.Newf("invalid op: %d", op) From 5864ce4fae120db40d890f515193a115e7147459 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 2 Jan 2025 16:00:26 -0500 Subject: [PATCH 2/8] opt/exec: use `bytes.Reader` in `planGistDecoder` `planGistDecoder` now uses a `bytes.Reader` instead of a `bytes.Buffer`. Writing bytes to a `bytes.Buffer` is done with the `Write` method which copies the bytes into the buffer, allocating more space if necessary. A `bytes.Reader` can read over an existing slice of bytes with the `Reset` method without copying or allocating. Release note: None --- pkg/sql/opt/exec/explain/plan_gist_factory.go | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index ddd1e3600673..b173d1c351f5 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -149,7 +149,7 @@ func (f *PlanGistFactory) PlanGist() PlanGist { // planGistDecoder is used to decode a plan gist into a logical plan. type planGistDecoder struct { - buffer bytes.Buffer + buf bytes.Reader nodeStack []*Node // catalog is used to resolve table and index ids that are stored in the gist. @@ -193,16 +193,15 @@ func DecodePlanGistToRows( // DecodePlanGistToPlan constructs an explain.Node tree from a gist. func DecodePlanGistToPlan(s string, cat cat.Catalog) (plan *Plan, err error) { - d := planGistDecoder{ - catalog: cat, - } - b, err := base64.StdEncoding.DecodeString(s) if err != nil { return nil, err } - d.buffer.Write(b) + var d planGistDecoder + d.buf.Reset(b) + d.catalog = cat + plan = &Plan{} ver := d.decodeInt() @@ -234,7 +233,7 @@ func DecodePlanGistToPlan(s string, cat cat.Catalog) (plan *Plan, err error) { } func (d *planGistDecoder) decodeOp() execOperator { - val, err := d.buffer.ReadByte() + val, err := d.buf.ReadByte() if err != nil || val == 0 { return unknownOp } @@ -269,7 +268,7 @@ func (f *PlanGistFactory) encodeInt(i int) { } func (d *planGistDecoder) decodeInt() int { - val, err := binary.ReadVarint(&d.buffer) + val, err := binary.ReadVarint(&d.buf) if err != nil { panic(err) } @@ -361,7 +360,7 @@ func (f *PlanGistFactory) encodeByte(b byte) { } func (d *planGistDecoder) decodeByte() byte { - val, err := d.buffer.ReadByte() + val, err := d.buf.ReadByte() if err != nil { panic(err) } @@ -432,7 +431,7 @@ func (f *PlanGistFactory) encodeScanParams(params exec.ScanParams) { func (d *planGistDecoder) decodeScanParams() exec.ScanParams { neededCols := intsets.Fast{} - err := neededCols.Decode(&d.buffer) + err := neededCols.Decode(&d.buf) if err != nil { panic(err) } From f64b485ebeeb676a7fa17dc06eb968f8724bb525 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 2 Jan 2025 17:10:12 -0500 Subject: [PATCH 3/8] util/base64: add `Encoder` The `base64.Encoder` utility has been added for encoding byte slice streams into a base64 string. It is adapted from the standard library's base64 encoder returned by `base64.NewEncoder` with some simplifications and an API that helps eliminate unnecessary allocations. Release note: None --- pkg/BUILD.bazel | 3 ++ pkg/util/base64/BUILD.bazel | 15 ++++++ pkg/util/base64/base64.go | 84 ++++++++++++++++++++++++++++++++++ pkg/util/base64/base64_test.go | 55 ++++++++++++++++++++++ 4 files changed, 157 insertions(+) create mode 100644 pkg/util/base64/BUILD.bazel create mode 100644 pkg/util/base64/base64.go create mode 100644 pkg/util/base64/base64_test.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 733105b607ad..82060dcc7824 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -678,6 +678,7 @@ ALL_TESTS = [ "//pkg/util/admission/admissionpb:admissionpb_test", "//pkg/util/admission:admission_test", "//pkg/util/assetbundle:assetbundle_test", + "//pkg/util/base64:base64_test", "//pkg/util/binfetcher:binfetcher_test", "//pkg/util/bitarray:bitarray_test", "//pkg/util/bitmap:bitmap_test", @@ -2432,6 +2433,8 @@ GO_TARGETS = [ "//pkg/util/assetbundle/gen:gen_lib", "//pkg/util/assetbundle:assetbundle", "//pkg/util/assetbundle:assetbundle_test", + "//pkg/util/base64:base64", + "//pkg/util/base64:base64_test", "//pkg/util/binfetcher:binfetcher", "//pkg/util/binfetcher:binfetcher_test", "//pkg/util/bitarray:bitarray", diff --git a/pkg/util/base64/BUILD.bazel b/pkg/util/base64/BUILD.bazel new file mode 100644 index 000000000000..94aa14c24983 --- /dev/null +++ b/pkg/util/base64/BUILD.bazel @@ -0,0 +1,15 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "base64", + srcs = ["base64.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/util/base64", + visibility = ["//visibility:public"], +) + +go_test( + name = "base64_test", + srcs = ["base64_test.go"], + embed = [":base64"], + deps = ["//pkg/util/randutil"], +) diff --git a/pkg/util/base64/base64.go b/pkg/util/base64/base64.go new file mode 100644 index 000000000000..507977f204d4 --- /dev/null +++ b/pkg/util/base64/base64.go @@ -0,0 +1,84 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package base64 + +import ( + "encoding/base64" + "strings" +) + +// Encoder is a streaming encoder for base64 strings. It must be initialized +// with Init. +// +// The code has been adapted from the standard library encoder that is returned +// by base64.NewEncoder. The main differences are that Init allows for an +// Encoder value to be initialized directly and that it writes to an internal +// strings.Builder instead of an io.Writer. Both allow streaming encoding +// without extra allocations. Writes to a strings.Builder never fail, so error +// handling has been removed for simplification. +type Encoder struct { + enc *base64.Encoding + sb strings.Builder + buf [3]byte // buffered data waiting to be encoded + nbuf int8 // number of bytes in buf + out [1024]byte // output buffer +} + +// Init initializes an Encoder with the given base64.Encoding. +func (e *Encoder) Init(enc *base64.Encoding) { + *e = Encoder{ + enc: enc, + } +} + +// Write encodes p in an internal buffer. +func (e *Encoder) Write(p []byte) { + // Leading fringe. + if e.nbuf > 0 { + var i int + for i = 0; i < len(p) && e.nbuf < 3; i++ { + e.buf[e.nbuf] = p[i] + e.nbuf++ + } + p = p[i:] + if e.nbuf < 3 { + return + } + e.enc.Encode(e.out[:], e.buf[:]) + _, _ = e.sb.Write(e.out[:4]) + e.nbuf = 0 + } + + // Large interior chunks. + for len(p) >= 3 { + nn := len(e.out) / 4 * 3 + if nn > len(p) { + nn = len(p) + nn -= nn % 3 + } + e.enc.Encode(e.out[:], p[:nn]) + _, _ = e.sb.Write(e.out[0 : nn/3*4]) + p = p[nn:] + } + + // Trailing fringe. + copy(e.buf[:], p) + e.nbuf = int8(len(p)) +} + +// String flushes any pending output from the encoder and returns the encoded +// string. The Encoder is reset to its initial state so it can be reused. +func (e *Encoder) String() (s string) { + // If there's anything left in the buffer, flush it out. + if e.nbuf > 0 { + e.enc.Encode(e.out[:], e.buf[:e.nbuf]) + _, _ = e.sb.Write(e.out[:e.enc.EncodedLen(int(e.nbuf))]) + e.nbuf = 0 + } + s = e.sb.String() + e.Init(e.enc) + return s +} diff --git a/pkg/util/base64/base64_test.go b/pkg/util/base64/base64_test.go new file mode 100644 index 000000000000..f3541d4af707 --- /dev/null +++ b/pkg/util/base64/base64_test.go @@ -0,0 +1,55 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package base64 + +import ( + "bytes" + "encoding/base64" + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestEncoder(t *testing.T) { + const ( + iters = 1000 + maxLen = 1000 + ) + + rng, _ := randutil.NewTestRand() + + var enc Encoder + enc.Init(base64.StdEncoding) + + if s := enc.String(); s != "" { + t.Errorf("expected empty string, got %q", s) + } + + for i := 0; i < iters; i++ { + // Generate a random byte slice. + b := randutil.RandBytes(rng, rng.Intn(maxLen+1)) + + // Write b in randomly sized chunks. + rest := b + for len(rest) > 0 { + chunk := rng.Intn(len(rest)) + 1 + enc.Write(rest[:chunk]) + rest = rest[chunk:] + } + + // Decode the encoded string. + s := enc.String() + dec, err := base64.StdEncoding.DecodeString(s) + if err != nil { + t.Fatal(err) + } + + // Check that the decoded string equals the original byte slice. + if !bytes.Equal(b, dec) { + t.Errorf("failed round-trip encoding/decoding of %v", b) + } + } +} From 2984feac7f044ab9edd02ea94fe41388df176462 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 3 Jan 2025 11:37:44 -0500 Subject: [PATCH 4/8] util: break dependence on util/intsets The `util` package no longer imports `util/intsets`. This will allow `util/intsets` to import `util` in future commits without creating an import cycle. Release note: None --- pkg/util/BUILD.bazel | 1 - pkg/util/fast_int_map.go | 7 +++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/pkg/util/BUILD.bazel b/pkg/util/BUILD.bazel index 8ebf2f185285..fae8d3c09087 100644 --- a/pkg/util/BUILD.bazel +++ b/pkg/util/BUILD.bazel @@ -25,7 +25,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/util", visibility = ["//visibility:public"], deps = [ - "//pkg/util/intsets", "//pkg/util/netutil/addr", "//pkg/util/syncutil", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/util/fast_int_map.go b/pkg/util/fast_int_map.go index 08f9dc68f29a..a0d89f988413 100644 --- a/pkg/util/fast_int_map.go +++ b/pkg/util/fast_int_map.go @@ -8,10 +8,9 @@ package util import ( "bytes" "fmt" + "math" "math/bits" "sort" - - "github.com/cockroachdb/cockroach/pkg/util/intsets" ) // FastIntMap is a replacement for map[int]int which is more efficient when both @@ -131,7 +130,7 @@ func (m FastIntMap) MaxKey() (_ int, ok bool) { if len(m.large) == 0 { return 0, false } - max := intsets.MinInt + max := math.MinInt for k := range m.large { if max < k { max = k @@ -168,7 +167,7 @@ func (m FastIntMap) MaxValue() (_ int, ok bool) { if len(m.large) == 0 { return 0, false } - max := intsets.MinInt + max := math.MinInt for _, v := range m.large { if max < v { max = v From 64e701368c8fc4ce9b198f3654ca0ec1f0ccbcaa Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 2 Jan 2025 17:54:19 -0500 Subject: [PATCH 5/8] opt/exec: use `base64.Encoder` for plan gist encoding `base64.Encoder` is now used in `PlanGistFactory` to simplify the logic and reduce allocations. Release note: None --- pkg/sql/opt/exec/explain/BUILD.bazel | 1 + pkg/sql/opt/exec/explain/plan_gist_factory.go | 51 ++++++++++--------- 2 files changed, 29 insertions(+), 23 deletions(-) diff --git a/pkg/sql/opt/exec/explain/BUILD.bazel b/pkg/sql/opt/exec/explain/BUILD.bazel index 126108f0447a..a03d9535682f 100644 --- a/pkg/sql/opt/exec/explain/BUILD.bazel +++ b/pkg/sql/opt/exec/explain/BUILD.bazel @@ -36,6 +36,7 @@ go_library( "//pkg/sql/sessiondatapb", "//pkg/sql/types", "//pkg/util", + "//pkg/util/base64", "//pkg/util/errorutil", "//pkg/util/humanizeutil", "//pkg/util/intsets", diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index b173d1c351f5..599f332b62cf 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -8,7 +8,7 @@ package explain import ( "bytes" "context" - "encoding/base64" + b64 "encoding/base64" "encoding/binary" "github.com/cockroachdb/cockroach/pkg/geo/geopb" @@ -26,6 +26,7 @@ import ( "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/base64" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/intsets" "github.com/cockroachdb/errors" @@ -89,13 +90,15 @@ func (fp PlanGist) Hash() uint64 { type PlanGistFactory struct { wrappedFactory exec.Factory - // buffer accumulates the encoded gist. Data that is written to the buffer - // is also added to the hash. The exception is when we're dealing with ids - // where we will write the id to the buffer and the "string" to the hash. - // This allows the hash to be id agnostic (ie hash's will be stable across - // plans from different databases with different DDL history). - buffer bytes.Buffer - hash util.FNV64 + // enc accumulates the encoded gist. Data that is written to enc is also + // added to hash. The exception is when we're dealing with ids where we will + // write the id to buf and the "string" to hash. This allows the hash to be + // id agnostic (ie hash's will be stable across plans from different + // databases with different DDL history). + enc base64.Encoder + // buf is a scratch bytes buffer used during encoding. + buf bytes.Buffer + hash util.FNV64 } var _ exec.Factory = &PlanGistFactory{} @@ -107,10 +110,9 @@ func (f *PlanGistFactory) Ctx() context.Context { // writeAndHash writes an arbitrary slice of bytes to the buffer and hashes each // byte. -func (f *PlanGistFactory) writeAndHash(data []byte) int { - i, _ := f.buffer.Write(data) +func (f *PlanGistFactory) writeAndHash(data []byte) { + f.enc.Write(data) f.updateHash(data) - return i } func (f *PlanGistFactory) updateHash(data []byte) { @@ -123,6 +125,7 @@ func (f *PlanGistFactory) updateHash(data []byte) { func NewPlanGistFactory(wrappedFactory exec.Factory) *PlanGistFactory { f := new(PlanGistFactory) f.wrappedFactory = wrappedFactory + f.enc.Init(b64.StdEncoding) f.hash.Init() f.encodeInt(gistVersion) return f @@ -141,10 +144,13 @@ func (f *PlanGistFactory) ConstructPlan( return plan, err } -// PlanGist returns a pointer to a PlanGist. +// PlanGist returns an encoded PlanGist. It should only be called once after +// ConstructPlan. func (f *PlanGistFactory) PlanGist() PlanGist { - return PlanGist{gist: base64.StdEncoding.EncodeToString(f.buffer.Bytes()), - hash: f.hash.Sum()} + return PlanGist{ + gist: f.enc.String(), + hash: f.hash.Sum(), + } } // planGistDecoder is used to decode a plan gist into a logical plan. @@ -193,7 +199,7 @@ func DecodePlanGistToRows( // DecodePlanGistToPlan constructs an explain.Node tree from a gist. func DecodePlanGistToPlan(s string, cat cat.Catalog) (plan *Plan, err error) { - b, err := base64.StdEncoding.DecodeString(s) + b, err := b64.StdEncoding.DecodeString(s) if err != nil { return nil, err } @@ -281,10 +287,7 @@ func (d *planGistDecoder) decodeInt() int { func (f *PlanGistFactory) encodeDataSource(id cat.StableID, name tree.Name) { var buf [binary.MaxVarintLen64]byte n := binary.PutVarint(buf[:], int64(id)) - _, err := f.buffer.Write(buf[:n]) - if err != nil { - panic(err) - } + f.enc.Write(buf[:n]) f.updateHash([]byte(string(name))) } @@ -355,7 +358,7 @@ func (d *planGistDecoder) decodeResultColumns() colinfo.ResultColumns { } func (f *PlanGistFactory) encodeByte(b byte) { - f.buffer.WriteByte(b) + f.enc.Write([]byte{b}) f.hash.Add(uint64(b)) } @@ -386,11 +389,13 @@ func (d *planGistDecoder) decodeBool() bool { } func (f *PlanGistFactory) encodeFastIntSet(s intsets.Fast) { - lenBefore := f.buffer.Len() - if err := s.Encode(&f.buffer); err != nil { + f.buf.Reset() + if err := s.Encode(&f.buf); err != nil { panic(err) } - f.updateHash(f.buffer.Bytes()[lenBefore:]) + b := f.buf.Bytes() + f.enc.Write(b) + f.updateHash(b) } // TODO: enable this or remove it... From ac13c1455460d52f600d61bb11835404976c25da Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 3 Jan 2025 13:36:22 -0500 Subject: [PATCH 6/8] opt/exec: encode plan gist intsets directly as base64 The `intsets.(*Fast).EncodeBase64` method has been added and it is now used to base64 encode intsets directly in plan gists. This avoids the intermediate step of encoding the intset to a temporary buffer before base64 encoding it. Release note: None --- pkg/sql/opt/exec/explain/plan_gist_factory.go | 10 +---- pkg/util/intsets/BUILD.bazel | 12 +++++- pkg/util/intsets/fast.go | 40 ++++++++++++++++--- pkg/util/intsets/fast_test.go | 19 +++++++++ pkg/util/intsets/fast_testonly.go | 35 ++++++++++++++-- 5 files changed, 96 insertions(+), 20 deletions(-) diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index 599f332b62cf..d79faec2adf4 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -95,9 +95,7 @@ type PlanGistFactory struct { // write the id to buf and the "string" to hash. This allows the hash to be // id agnostic (ie hash's will be stable across plans from different // databases with different DDL history). - enc base64.Encoder - // buf is a scratch bytes buffer used during encoding. - buf bytes.Buffer + enc base64.Encoder hash util.FNV64 } @@ -389,13 +387,9 @@ func (d *planGistDecoder) decodeBool() bool { } func (f *PlanGistFactory) encodeFastIntSet(s intsets.Fast) { - f.buf.Reset() - if err := s.Encode(&f.buf); err != nil { + if err := s.EncodeBase64(&f.enc, &f.hash); err != nil { panic(err) } - b := f.buf.Bytes() - f.enc.Write(b) - f.updateHash(b) } // TODO: enable this or remove it... diff --git a/pkg/util/intsets/BUILD.bazel b/pkg/util/intsets/BUILD.bazel index 37aff3f16da9..82696182c172 100644 --- a/pkg/util/intsets/BUILD.bazel +++ b/pkg/util/intsets/BUILD.bazel @@ -14,7 +14,11 @@ go_library( ], importpath = "github.com/cockroachdb/cockroach/pkg/util/intsets", visibility = ["//visibility:public"], - deps = ["@com_github_cockroachdb_errors//:errors"], + deps = [ + "//pkg/util", + "//pkg/util/base64", + "@com_github_cockroachdb_errors//:errors", + ], ) go_test( @@ -25,5 +29,9 @@ go_test( "sparse_test.go", ], embed = [":intsets"], - deps = ["//pkg/util/randutil"], + deps = [ + "//pkg/util", + "//pkg/util/base64", + "//pkg/util/randutil", + ], ) diff --git a/pkg/util/intsets/fast.go b/pkg/util/intsets/fast.go index cd3101058e16..35713c97e4f9 100644 --- a/pkg/util/intsets/fast.go +++ b/pkg/util/intsets/fast.go @@ -14,6 +14,8 @@ import ( "io" "math/bits" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/base64" "github.com/cockroachdb/errors" ) @@ -301,29 +303,55 @@ func (s Fast) SubsetOf(rhs Fast) bool { // // If the set has only elements in the range [0, 63], we encode a 0 followed by // a 64-bit bitmap. Otherwise, we encode a length followed by each element. +func (s *Fast) Encode(buf *bytes.Buffer) error { + return s.encodeImpl(buf, nil, nil) +} + +// EncodeBase64 is similar to Encode. It writes the encoded set to enc. It also +// adds each pre-base64-encoded byte to hash. +// +// Closures or interfaces could be used to merge both methods into one, but they +// are intentionally avoided to prevent extra allocations of temporary buffers +// used during encoding. // // WARNING: this is used by plan gists, so if this encoding changes, // explain.gistVersion needs to be bumped. -func (s *Fast) Encode(buf *bytes.Buffer) error { +func (s *Fast) EncodeBase64(enc *base64.Encoder, hash *util.FNV64) error { + return s.encodeImpl(nil, enc, hash) +} + +func (s *Fast) encodeImpl(buf *bytes.Buffer, enc *base64.Encoder, hash *util.FNV64) error { if s.large != nil && s.large.Min() < 0 { return errors.AssertionFailedf("Encode used with negative elements") } + write := func(b []byte) { + if buf != nil { + buf.Write(b) + } else { + enc.Write(b) + for i := range b { + hash.Add(uint64(b[i])) + } + } + } + // This slice should stay on stack. We only need enough bytes to encode a 0 // and then an arbitrary 64-bit integer. //gcassert:noescape tmp := make([]byte, binary.MaxVarintLen64+1) + var n int if s.small.hi == 0 && s.fitsInSmall() { - n := binary.PutUvarint(tmp, 0) + n = binary.PutUvarint(tmp, 0) n += binary.PutUvarint(tmp[n:], s.small.lo) - buf.Write(tmp[:n]) + write(tmp[:n]) } else { - n := binary.PutUvarint(tmp, uint64(s.Len())) - buf.Write(tmp[:n]) + n = binary.PutUvarint(tmp, uint64(s.Len())) + write(tmp[:n]) for i, ok := s.Next(0); ok; i, ok = s.Next(i + 1) { n := binary.PutUvarint(tmp, uint64(i)) - buf.Write(tmp[:n]) + write(tmp[:n]) } } return nil diff --git a/pkg/util/intsets/fast_test.go b/pkg/util/intsets/fast_test.go index 5e2237325919..95b57ac8a81f 100644 --- a/pkg/util/intsets/fast_test.go +++ b/pkg/util/intsets/fast_test.go @@ -7,10 +7,13 @@ package intsets import ( "bytes" + b64 "encoding/base64" "fmt" "reflect" "testing" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/base64" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) @@ -112,6 +115,22 @@ func TestFast(t *testing.T) { t.Fatalf("error during Decode: %v", err) } assertSame(s, s2) + // Verify that EncodeBase64 decodes to the result of + // Encoded. + var enc base64.Encoder + var hash util.FNV64 + enc.Init(b64.StdEncoding) + hash.Init() + if err := s.EncodeBase64(&enc, &hash); err != nil { + t.Fatalf("error during EncodeBase64: %v", err) + } + dec, err := b64.StdEncoding.DecodeString(enc.String()) + if err != nil { + t.Fatalf("error during base64 Decode: %v", err) + } + if encoded != string(dec) { + t.Fatalf("expected decoded base64, %q, to match encoding %q", string(dec), encoded) + } // Verify that decoding into a non-empty set still works. var s3 Fast s3.Add(minVal + rng.Intn(maxVal-minVal)) diff --git a/pkg/util/intsets/fast_testonly.go b/pkg/util/intsets/fast_testonly.go index 9c650142086c..5c822a99acc3 100644 --- a/pkg/util/intsets/fast_testonly.go +++ b/pkg/util/intsets/fast_testonly.go @@ -18,6 +18,8 @@ import ( "encoding/binary" "io" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/base64" "github.com/cockroachdb/errors" ) @@ -219,10 +221,24 @@ func (s Fast) SubsetOf(rhs Fast) bool { // // If the set has only elements in the range [0, 63], we encode a 0 followed by // a 64-bit bitmap. Otherwise, we encode a length followed by each element. +func (s *Fast) Encode(buf *bytes.Buffer) error { + return s.encodeImpl(buf, nil, nil) +} + +// EncodeBase64 is similar to Encode. It writes the encoded set to enc. It also +// adds each pre-base64-encoded byte to hash. +// +// Closures or interfaces could be used to merge both methods into one, but they +// are intentionally avoided to prevent extra allocations of temporary buffers +// used during encoding. // // WARNING: this is used by plan gists, so if this encoding changes, // explain.gistVersion needs to be bumped. -func (s *Fast) Encode(buf *bytes.Buffer) error { +func (s *Fast) EncodeBase64(enc *base64.Encoder, hash *util.FNV64) error { + return s.encodeImpl(nil, enc, hash) +} + +func (s *Fast) encodeImpl(buf *bytes.Buffer, enc *base64.Encoder, hash *util.FNV64) error { if s.s != nil && s.s.Min() < 0 { return errors.AssertionFailedf("Encode used with negative elements") } @@ -232,6 +248,17 @@ func (s *Fast) Encode(buf *bytes.Buffer) error { //gcassert:noescape tmp := make([]byte, binary.MaxVarintLen64+1) + write := func(b []byte) { + if buf != nil { + buf.Write(b) + } else { + enc.Write(b) + for i := range b { + hash.Add(uint64(b[i])) + } + } + } + max := MinInt s.ForEach(func(i int) { if i > max { @@ -246,13 +273,13 @@ func (s *Fast) Encode(buf *bytes.Buffer) error { bitmap |= (1 << uint64(i)) } n += binary.PutUvarint(tmp[n:], bitmap) - buf.Write(tmp[:n]) + write(tmp[:n]) } else { n := binary.PutUvarint(tmp, uint64(s.Len())) - buf.Write(tmp[:n]) + write(tmp[:n]) for i, ok := s.Next(0); ok; i, ok = s.Next(i + 1) { n := binary.PutUvarint(tmp, uint64(i)) - buf.Write(tmp[:n]) + write(tmp[:n]) } } return nil From f2be10825ea1c31e7f76efd8908748373b33c6c7 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 3 Jan 2025 14:01:23 -0500 Subject: [PATCH 7/8] sql: embed `explain.PlanGistFactory` in `optPlanningCtx` A reusable `explain.PlanGistFactory` is now embedded in `optPlanningCtx` to avoid re-allocation of the factory for each query. Release note: None --- pkg/sql/opt/exec/explain/plan_gist_factory.go | 23 ++++++++++++++++--- pkg/sql/plan_opt.go | 12 ++++++---- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index d79faec2adf4..1f76dce8174b 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -120,13 +120,30 @@ func (f *PlanGistFactory) updateHash(data []byte) { } // NewPlanGistFactory creates a new PlanGistFactory. -func NewPlanGistFactory(wrappedFactory exec.Factory) *PlanGistFactory { +func NewPlanGistFactory(factory exec.Factory) *PlanGistFactory { f := new(PlanGistFactory) - f.wrappedFactory = wrappedFactory + f.Init(factory) + return f +} + +// Init initializes a PlanGistFactory. +func (f *PlanGistFactory) Init(factory exec.Factory) { + *f = PlanGistFactory{ + wrappedFactory: factory, + } f.enc.Init(b64.StdEncoding) f.hash.Init() f.encodeInt(gistVersion) - return f +} + +// Initialized returns true if the PlanGistFactory has been initialized. +func (f *PlanGistFactory) Initialized() bool { + return f.wrappedFactory != nil +} + +// Reset resets the PlanGistFactory, clearing references to the wrapped factory. +func (f *PlanGistFactory) Reset() { + *f = PlanGistFactory{} } // ConstructPlan delegates to the wrapped factory. diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 62a3fbe428ca..d5ef93a425af 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -360,6 +360,8 @@ type optPlanningCtx struct { useCache bool flags planFlags + + gf explain.PlanGistFactory } // init performs one-time initialization of the planning context; reset() must @@ -882,10 +884,10 @@ func (opc *optPlanningCtx) runExecBuilder( allowAutoCommit bool, ) error { var result *planComponents - var gf *explain.PlanGistFactory if !opc.p.SessionData().DisablePlanGists { - gf = explain.NewPlanGistFactory(f) - f = gf + opc.gf.Init(f) + defer opc.gf.Reset() + f = &opc.gf } var bld *execbuilder.Builder if !planTop.instrumentation.ShouldBuildExplainPlan() { @@ -923,8 +925,8 @@ func (opc *optPlanningCtx) runExecBuilder( planTop.instrumentation.scanCounts = bld.ScanCounts planTop.instrumentation.indexesUsed = bld.IndexesUsed - if gf != nil { - planTop.instrumentation.planGist = gf.PlanGist() + if opc.gf.Initialized() { + planTop.instrumentation.planGist = opc.gf.PlanGist() } planTop.instrumentation.costEstimate = mem.RootExpr().(memo.RelExpr).Cost().C available := mem.RootExpr().(memo.RelExpr).Relational().Statistics().Available From 29999dcaf40ae48594b3d7531dc5a88319dcbe2d Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Wed, 8 Jan 2025 09:59:03 -0500 Subject: [PATCH 8/8] opt/exec: remove `explain.NewPlanGistFactory` `explain.NewPlanGistFactory` has been replaced by `(*PlanGistFactory).Init`. Release note: None --- pkg/sql/opt/bench/bench_test.go | 12 +++++++++--- pkg/sql/opt/exec/execbuilder/statement.go | 5 +++-- pkg/sql/opt/exec/explain/plan_gist_factory.go | 7 ------- pkg/sql/opt/exec/explain/plan_gist_test.go | 5 +++-- 4 files changed, 15 insertions(+), 14 deletions(-) diff --git a/pkg/sql/opt/bench/bench_test.go b/pkg/sql/opt/bench/bench_test.go index 430c86d1eae4..7386ee67bd8a 100644 --- a/pkg/sql/opt/bench/bench_test.go +++ b/pkg/sql/opt/bench/bench_test.go @@ -943,10 +943,12 @@ func (h *harness) runSimple(tb testing.TB, query benchQuery, phase Phase) { tb.Fatalf("invalid phase %s for Simple", phase) } + var gf explain.PlanGistFactory + gf.Init(exec.StubFactory{}) root := execMemo.RootExpr() eb := execbuilder.New( context.Background(), - explain.NewPlanGistFactory(exec.StubFactory{}), + &gf, &h.optimizer, execMemo, nil, /* catalog */ @@ -999,10 +1001,12 @@ func (h *harness) runPrepared(tb testing.TB, phase Phase) { tb.Fatalf("invalid phase %s for Prepared", phase) } + var gf explain.PlanGistFactory + gf.Init(exec.StubFactory{}) root := execMemo.RootExpr() eb := execbuilder.New( context.Background(), - explain.NewPlanGistFactory(exec.StubFactory{}), + &gf, &h.optimizer, execMemo, nil, /* catalog */ @@ -1777,9 +1781,11 @@ func BenchmarkExecBuild(b *testing.B) { b.Run(tc.query.name, func(b *testing.B) { for i := 0; i < b.N; i++ { + var gf explain.PlanGistFactory + gf.Init(exec.StubFactory{}) eb := execbuilder.New( context.Background(), - explain.NewPlanGistFactory(exec.StubFactory{}), + &gf, &h.optimizer, execMemo, nil, /* catalog */ diff --git a/pkg/sql/opt/exec/execbuilder/statement.go b/pkg/sql/opt/exec/execbuilder/statement.go index c7f25a9a993e..e6421ded061b 100644 --- a/pkg/sql/opt/exec/execbuilder/statement.go +++ b/pkg/sql/opt/exec/execbuilder/statement.go @@ -176,8 +176,9 @@ func (b *Builder) buildExplain( // annotates nodes with extra information when the factory is an // exec.ExplainFactory so it must be the outer factory and the gist // factory must be the inner factory. - gf := explain.NewPlanGistFactory(f) - ef := explain.NewFactory(gf, b.semaCtx, b.evalCtx) + var gf explain.PlanGistFactory + gf.Init(f) + ef := explain.NewFactory(&gf, b.semaCtx, b.evalCtx) explainBld := New( b.ctx, ef, b.optimizer, b.mem, b.catalog, explainExpr.Input, diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index 1f76dce8174b..714be17d44ce 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -119,13 +119,6 @@ func (f *PlanGistFactory) updateHash(data []byte) { } } -// NewPlanGistFactory creates a new PlanGistFactory. -func NewPlanGistFactory(factory exec.Factory) *PlanGistFactory { - f := new(PlanGistFactory) - f.Init(factory) - return f -} - // Init initializes a PlanGistFactory. func (f *PlanGistFactory) Init(factory exec.Factory) { *f = PlanGistFactory{ diff --git a/pkg/sql/opt/exec/explain/plan_gist_test.go b/pkg/sql/opt/exec/explain/plan_gist_test.go index 2e1f0777225d..0caad1583705 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_test.go +++ b/pkg/sql/opt/exec/explain/plan_gist_test.go @@ -23,7 +23,8 @@ import ( ) func makeGist(ot *opttester.OptTester, t *testing.T) explain.PlanGist { - f := explain.NewPlanGistFactory(exec.StubFactory{}) + var f explain.PlanGistFactory + f.Init(exec.StubFactory{}) expr, err := ot.Optimize() if err != nil { t.Error(err) @@ -32,7 +33,7 @@ func makeGist(ot *opttester.OptTester, t *testing.T) explain.PlanGist { if rel, ok := expr.(memo.RelExpr); ok { mem = rel.Memo() } - _, err = ot.ExecBuild(f, mem, expr) + _, err = ot.ExecBuild(&f, mem, expr) if err != nil { t.Error(err) }