Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
76533: span: refactor span splitting into Splitter r=RaduBerinde a=RaduBerinde

This change pulls out the span.Builder code related to splitting spans
along needed families into a new type, span.Splitter.

This disentanglement makes the logic easier to follow and makes the
higher level APIs cleaner (we just pass a Splitter around instead of
flags like `forDelete`).

Release note: None

76607: sql: add notice for custom var with the same name as a cluster setting r=otan a=rafiss

fixes cockroachdb#74375

No release note since custom session variables are new in 22.1.

Release note: None

Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
3 people committed Feb 16, 2022
3 parents 9f98cde + 32963dc + 03a8885 commit 1a8570f
Show file tree
Hide file tree
Showing 18 changed files with 416 additions and 309 deletions.
2 changes: 0 additions & 2 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -529,7 +529,6 @@ go_test(
"show_test.go",
"show_trace_replica_test.go",
"sort_test.go",
"span_builder_test.go",
"split_test.go",
"statement_mark_redaction_test.go",
"table_ref_test.go",
Expand Down Expand Up @@ -637,7 +636,6 @@ go_test(
"//pkg/sql/sessiondata",
"//pkg/sql/sessiondatapb",
"//pkg/sql/sessionphase",
"//pkg/sql/span",
"//pkg/sql/sqlliveness",
"//pkg/sql/sqlstats",
"//pkg/sql/sqltestutils",
Expand Down
12 changes: 8 additions & 4 deletions pkg/sql/colexec/colexecspan/span_assembler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ func TestSpanAssembler(t *testing.T) {
converter := colconv.NewAllVecToDatumConverter(len(typs))

builder := span.MakeBuilder(&evalCtx, keys.TODOSQLCodec, testTable, testTable.GetPrimaryIndex())
builder.SetNeededColumns(neededColumns)
splitter := span.MakeSplitter(testTable, testTable.GetPrimaryIndex(), neededColumns)

colBuilder := NewColSpanAssembler(
keys.TODOSQLCodec, testAllocator, testTable,
Expand Down Expand Up @@ -143,7 +143,7 @@ func TestSpanAssembler(t *testing.T) {
}
rows[i] = row
}
oracleSpans = append(oracleSpans, spanGeneratorOracle(t, builder, rows, len(typs))...)
oracleSpans = append(oracleSpans, spanGeneratorOracle(t, builder, splitter, rows, len(typs))...)
}

if len(oracleSpans) != len(testSpans) {
Expand All @@ -168,15 +168,19 @@ func TestSpanAssembler(t *testing.T) {
// spanGeneratorOracle extracts the logic from joinreader_span_generator.go that
// pertains to index joins.
func spanGeneratorOracle(
t *testing.T, spanBuilder *span.Builder, rows []rowenc.EncDatumRow, lookupCols int,
t *testing.T,
spanBuilder *span.Builder,
spanSplitter span.Splitter,
rows []rowenc.EncDatumRow,
lookupCols int,
) roachpb.Spans {
var spans roachpb.Spans
for _, inputRow := range rows {
generatedSpan, containsNull, err := spanBuilder.SpanFromEncDatums(inputRow, lookupCols)
if err != nil {
t.Fatal(err)
}
spans = spanBuilder.MaybeSplitSpanIntoSeparateFamilies(
spans = spanSplitter.MaybeSplitSpanIntoSeparateFamilies(
spans, generatedSpan, lookupCols, containsNull)
}
return spans
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/distsql_spec_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,7 +217,8 @@ func (e *distSQLSpecExecFactory) ConstructScan(
if params.InvertedConstraint != nil {
spans, err = sb.SpansFromInvertedSpans(params.InvertedConstraint, params.IndexConstraint, nil /* scratch */)
} else {
spans, err = sb.SpansFromConstraint(params.IndexConstraint, params.NeededCols, false /* forDelete */)
splitter := span.MakeSplitter(tabDesc, idx, params.NeededCols)
spans, err = sb.SpansFromConstraint(params.IndexConstraint, splitter)
}
if err != nil {
return nil, err
Expand Down
15 changes: 9 additions & 6 deletions pkg/sql/insert_fast_path.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/span"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -88,11 +89,12 @@ type insertFastPathFKSpanInfo struct {
type insertFastPathFKCheck struct {
exec.InsertFastPathFKCheck

tabDesc catalog.TableDescriptor
idx catalog.Index
keyPrefix []byte
colMap catalog.TableColMap
spanBuilder *span.Builder
tabDesc catalog.TableDescriptor
idx catalog.Index
keyPrefix []byte
colMap catalog.TableColMap
spanBuilder *span.Builder
spanSplitter span.Splitter
}

func (c *insertFastPathFKCheck) init(params runParams) error {
Expand All @@ -103,6 +105,7 @@ func (c *insertFastPathFKCheck) init(params runParams) error {
codec := params.ExecCfg().Codec
c.keyPrefix = rowenc.MakeIndexKeyPrefix(codec, c.tabDesc.GetID(), c.idx.GetID())
c.spanBuilder = span.MakeBuilder(params.EvalContext(), codec, c.tabDesc, c.idx)
c.spanSplitter = span.MakeSplitter(c.tabDesc, c.idx, util.FastIntSet{} /* neededColOrdinals */)

if len(c.InsertCols) > idx.numLaxKeyCols {
return errors.AssertionFailedf(
Expand All @@ -125,7 +128,7 @@ func (c *insertFastPathFKCheck) init(params runParams) error {
// generateSpan returns the span that we need to look up to confirm existence of
// the referenced row.
func (c *insertFastPathFKCheck) generateSpan(inputRow tree.Datums) (roachpb.Span, error) {
return row.FKCheckSpan(c.spanBuilder, inputRow, c.colMap, len(c.InsertCols))
return row.FKCheckSpan(c.spanBuilder, c.spanSplitter, inputRow, c.colMap, len(c.InsertCols))
}

// errorForRow returns an error indicating failure of this FK check for the
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/set
Original file line number Diff line number Diff line change
Expand Up @@ -753,6 +753,13 @@ SHOW tracing.custom
----
ijk

# Show a notice if a custom session variable has the same name as a cluster setting.
query T noticetrace
SET server.shutdown.drain_wait = '10s'
----
NOTICE: setting custom variable "server.shutdown.drain_wait"
HINT: did you mean SET CLUSTER SETTING?

# Test that RESET ALL changes custom options to empty strings.
statement ok
RESET ALL
Expand Down
7 changes: 3 additions & 4 deletions pkg/sql/opt_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,8 @@ func generateScanSpans(
if params.InvertedConstraint != nil {
return sb.SpansFromInvertedSpans(params.InvertedConstraint, params.IndexConstraint, nil /* scratch */)
}
return sb.SpansFromConstraint(params.IndexConstraint, params.NeededCols, false /* forDelete */)
splitter := span.MakeSplitter(tabDesc, index, params.NeededCols)
return sb.SpansFromConstraint(params.IndexConstraint, splitter)
}

func (ef *execFactory) constructVirtualScan(
Expand Down Expand Up @@ -1737,9 +1738,7 @@ func (ef *execFactory) ConstructDeleteRange(
return nil, err
}

// Setting the "forDelete" flag includes all column families in case where a
// single record is deleted.
spans, err := sb.SpansFromConstraint(indexConstraint, needed, true /* forDelete */)
spans, err := sb.SpansFromConstraint(indexConstraint, span.NoopSplitter())
if err != nil {
return nil, err
}
Expand Down
13 changes: 6 additions & 7 deletions pkg/sql/row/fk_spans.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,15 @@ import (
// FKCheckSpan returns a span that can be scanned to ascertain existence of a
// specific row in a given index.
func FKCheckSpan(
s *span.Builder, values []tree.Datum, colMap catalog.TableColMap, numCols int,
s *span.Builder,
splitter span.Splitter,
values []tree.Datum,
colMap catalog.TableColMap,
numCols int,
) (roachpb.Span, error) {
span, containsNull, err := s.SpanFromDatumRow(values, numCols, colMap)
if err != nil {
return roachpb.Span{}, err
}
// If it is safe to split this lookup into multiple families, generate a point lookup for
// family 0. Because we are just checking for existence, we only need family 0.
if s.CanSplitSpanIntoFamilySpans(1 /* numNeededFamilies */, numCols, containsNull) {
return s.SpanToPointSpan(span, 0), nil
}
return span, nil
return splitter.ExistenceCheckSpan(span, numCols, containsNull), nil
}
1 change: 0 additions & 1 deletion pkg/sql/rowexec/inverted_joiner.go
Original file line number Diff line number Diff line change
Expand Up @@ -338,7 +338,6 @@ func newInvertedJoiner(
}

ij.spanBuilder = span.MakeBuilder(flowCtx.EvalCtx, flowCtx.Codec(), ij.desc, ij.index)
ij.spanBuilder.SetNeededColumns(allIndexCols)

// Initialize memory monitors and row container for index rows.
ctx := flowCtx.EvalCtx.Ctx()
Expand Down
8 changes: 6 additions & 2 deletions pkg/sql/rowexec/joinreader.go
Original file line number Diff line number Diff line change
Expand Up @@ -507,7 +507,7 @@ func (jr *joinReader) initJoinReaderStrategy(
readerType joinReaderType,
) error {
spanBuilder := span.MakeBuilder(flowCtx.EvalCtx, flowCtx.Codec(), jr.desc, jr.index)
spanBuilder.SetNeededColumns(neededRightCols)
spanSplitter := span.MakeSplitter(jr.desc, jr.index, neededRightCols)

strategyMemAcc := jr.MemMonitor.MakeBoundAccount()
spanGeneratorMemAcc := jr.MemMonitor.MakeBoundAccount()
Expand All @@ -521,6 +521,7 @@ func (jr *joinReader) initJoinReaderStrategy(
}
generator = &defaultSpanGenerator{
spanBuilder: spanBuilder,
spanSplitter: spanSplitter,
keyToInputRowIndices: keyToInputRowIndices,
numKeyCols: numKeyCols,
lookupCols: jr.lookupCols,
Expand All @@ -546,6 +547,7 @@ func (jr *joinReader) initJoinReaderStrategy(
multiSpanGen := &multiSpanGenerator{}
if err := multiSpanGen.init(
spanBuilder,
spanSplitter,
numKeyCols,
len(jr.input.OutputTypes()),
&jr.lookupExpr,
Expand All @@ -558,11 +560,13 @@ func (jr *joinReader) initJoinReaderStrategy(
} else {
localityOptSpanGen := &localityOptimizedSpanGenerator{}
remoteSpanBuilder := span.MakeBuilder(flowCtx.EvalCtx, flowCtx.Codec(), jr.desc, jr.index)
remoteSpanBuilder.SetNeededColumns(neededRightCols)
remoteSpanSplitter := span.MakeSplitter(jr.desc, jr.index, neededRightCols)
remoteSpanGenMemAcc := jr.MemMonitor.MakeBoundAccount()
if err := localityOptSpanGen.init(
spanBuilder,
spanSplitter,
remoteSpanBuilder,
remoteSpanSplitter,
numKeyCols,
len(jr.input.OutputTypes()),
&jr.lookupExpr,
Expand Down
24 changes: 15 additions & 9 deletions pkg/sql/rowexec/joinreader_span_generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,10 @@ var _ joinReaderSpanGenerator = &multiSpanGenerator{}
var _ joinReaderSpanGenerator = &localityOptimizedSpanGenerator{}

type defaultSpanGenerator struct {
spanBuilder *span.Builder
numKeyCols int
lookupCols []uint32
spanBuilder *span.Builder
spanSplitter span.Splitter
numKeyCols int
lookupCols []uint32

indexKeyRow rowenc.EncDatumRow
// keyToInputRowIndices maps a lookup span key to the input row indices that
Expand Down Expand Up @@ -133,12 +134,12 @@ func (g *defaultSpanGenerator) generateSpans(
}
if g.keyToInputRowIndices == nil {
// Index join.
g.scratchSpans = g.spanBuilder.MaybeSplitSpanIntoSeparateFamilies(
g.scratchSpans = g.spanSplitter.MaybeSplitSpanIntoSeparateFamilies(
g.scratchSpans, generatedSpan, len(g.lookupCols), containsNull)
} else {
inputRowIndices := g.keyToInputRowIndices[string(generatedSpan.Key)]
if inputRowIndices == nil {
g.scratchSpans = g.spanBuilder.MaybeSplitSpanIntoSeparateFamilies(
g.scratchSpans = g.spanSplitter.MaybeSplitSpanIntoSeparateFamilies(
g.scratchSpans, generatedSpan, len(g.lookupCols), containsNull)
}
g.keyToInputRowIndices[string(generatedSpan.Key)] = append(inputRowIndices, i)
Expand Down Expand Up @@ -222,7 +223,8 @@ func (s spanRowIndices) memUsage() int64 {
// In this case, the multiSpanGenerator would generate two spans for each input
// row: [/'east'/<val_a> - /'east'/<val_a>] [/'west'/<val_a> - /'west'/<val_a>].
type multiSpanGenerator struct {
spanBuilder *span.Builder
spanBuilder *span.Builder
spanSplitter span.Splitter

// indexColInfos stores info about the values that each index column can
// take on in the spans produced by the multiSpanGenerator. See the comment
Expand Down Expand Up @@ -344,13 +346,15 @@ func (g *multiSpanGenerator) maxLookupCols() int {
// spans.
func (g *multiSpanGenerator) init(
spanBuilder *span.Builder,
spanSplitter span.Splitter,
numKeyCols int,
numInputCols int,
exprHelper *execinfrapb.ExprHelper,
tableOrdToIndexOrd util.FastIntMap,
memAcc *mon.BoundAccount,
) error {
g.spanBuilder = spanBuilder
g.spanSplitter = spanSplitter
g.numInputCols = numInputCols
g.keyToInputRowIndices = make(map[string][]int)
g.tableOrdToIndexOrd = tableOrdToIndexOrd
Expand Down Expand Up @@ -671,7 +675,7 @@ func (g *multiSpanGenerator) generateSpans(
if g.inequalityColIdx != -1 {
g.scratchSpans = append(g.scratchSpans, *generatedSpan)
} else {
g.scratchSpans = g.spanBuilder.MaybeSplitSpanIntoSeparateFamilies(
g.scratchSpans = g.spanSplitter.MaybeSplitSpanIntoSeparateFamilies(
g.scratchSpans, *generatedSpan, len(g.indexColInfos), false /* containsNull */)
}
}
Expand Down Expand Up @@ -748,7 +752,9 @@ type localityOptimizedSpanGenerator struct {
// close()d.
func (g *localityOptimizedSpanGenerator) init(
localSpanBuilder *span.Builder,
localSpanSplitter span.Splitter,
remoteSpanBuilder *span.Builder,
remoteSpanSplitter span.Splitter,
numKeyCols int,
numInputCols int,
localExprHelper *execinfrapb.ExprHelper,
Expand All @@ -758,12 +764,12 @@ func (g *localityOptimizedSpanGenerator) init(
remoteSpanGenMemAcc *mon.BoundAccount,
) error {
if err := g.localSpanGen.init(
localSpanBuilder, numKeyCols, numInputCols, localExprHelper, tableOrdToIndexOrd, localSpanGenMemAcc,
localSpanBuilder, localSpanSplitter, numKeyCols, numInputCols, localExprHelper, tableOrdToIndexOrd, localSpanGenMemAcc,
); err != nil {
return err
}
if err := g.remoteSpanGen.init(
remoteSpanBuilder, numKeyCols, numInputCols, remoteExprHelper, tableOrdToIndexOrd, remoteSpanGenMemAcc,
remoteSpanBuilder, remoteSpanSplitter, numKeyCols, numInputCols, remoteExprHelper, tableOrdToIndexOrd, remoteSpanGenMemAcc,
); err != nil {
return err
}
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/set_var.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (

"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/paramparse"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand Down Expand Up @@ -61,6 +62,15 @@ func (p *planner) SetVar(ctx context.Context, n *tree.SetVar) (planNode, error)
if err != nil {
return nil, err
}
if _, ok := settings.Lookup(name, settings.LookupForLocalAccess, p.ExecCfg().Codec.ForSystemTenant()); ok {
p.BufferClientNotice(
ctx,
errors.WithHint(
pgnotice.Newf("setting custom variable %q", name),
"did you mean SET CLUSTER SETTING?",
),
)
}

var typedValues []tree.TypedExpr
if len(n.Values) > 0 {
Expand Down
25 changes: 19 additions & 6 deletions pkg/sql/span/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "span",
srcs = ["span_builder.go"],
srcs = [
"span_builder.go",
"span_splitter.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/span",
visibility = ["//visibility:public"],
deps = [
Expand All @@ -12,7 +15,6 @@ go_library(
"//pkg/sql/catalog/descpb",
"//pkg/sql/inverted",
"//pkg/sql/opt/constraint",
"//pkg/sql/opt/exec",
"//pkg/sql/rowenc",
"//pkg/sql/rowenc/keyside",
"//pkg/sql/sem/tree",
Expand All @@ -26,12 +28,23 @@ go_library(
go_test(
name = "span_test",
size = "small",
srcs = ["span_builder_test.go"],
embed = [":span"],
srcs = [
"main_test.go",
"span_splitter_test.go",
],
deps = [
":span",
"//pkg/keys",
"//pkg/settings/cluster",
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql/catalog/desctestutils",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/sem/tree",
"//pkg/sql/tests",
"//pkg/testutils/serverutils",
"//pkg/util",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
],
)
Loading

0 comments on commit 1a8570f

Please sign in to comment.