Skip to content

Commit

Permalink
Merge #74761
Browse files Browse the repository at this point in the history
74761: sql: remove PHYSICAL scrub code r=RaduBerinde a=RaduBerinde

The PHYSICAL scrub code is experimental and not considered production
ready. It complicates a lot of code paths involved in normal query
execution (it significantly overloads the semantics of TableReader and
of the Fetcher) and is getting in the way of some improvements in how
the fetchers work. In particular, we are trying to reduce the amount
of information passed to TableReader/Fetcher (which in the
non-scrubbing case should be a lot less than the full table
descriptor).

There are some proposals for a better design floating around, e.g.
provide a facility for returning KVs as results from DistSQL and have
some higher-level code run the scrub checks.

This change removes the code for the PHYSICAL scrub for now.

Release note (sql change): the experimental SCRUB PHYSICAL is no
longer implemented.

Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
craig[bot] and RaduBerinde committed Jan 13, 2022
2 parents ca44523 + 9dd34a6 commit 3186eeb
Show file tree
Hide file tree
Showing 28 changed files with 31 additions and 1,230 deletions.
3 changes: 1 addition & 2 deletions pkg/ccl/changefeedccl/rowfetcher_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,8 +177,7 @@ func (c *rowFetcherCache) RowFetcherForTableDesc(
false, /* reverse */
descpb.ScanLockingStrength_FOR_NONE,
descpb.ScanLockingWaitPolicy_BLOCK,
0, /* lockTimeout */
false, /* isCheck */
0, /* lockTimeout */
&c.a,
nil, /* memMonitor */
rfArgs,
Expand Down
3 changes: 1 addition & 2 deletions pkg/ccl/cliccl/debug_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -600,8 +600,7 @@ func makeRowFetcher(
false, /*reverse*/
descpb.ScanLockingStrength_FOR_NONE,
descpb.ScanLockingWaitPolicy_BLOCK,
0, /* lockTimeout */
false, /*isCheck*/
0, /* lockTimeout */
&tree.DatumAlloc{},
nil, /*mon.BytesMonitor*/
table,
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,6 @@ go_library(
"distsql_plan_bulk.go",
"distsql_plan_ctas.go",
"distsql_plan_join.go",
"distsql_plan_scrub_physical.go",
"distsql_plan_set_op.go",
"distsql_plan_stats.go",
"distsql_plan_window.go",
Expand Down Expand Up @@ -177,7 +176,6 @@ go_library(
"scrub_constraint.go",
"scrub_fk.go",
"scrub_index.go",
"scrub_physical.go",
"select_name_resolution.go",
"sequence.go",
"sequence_select.go",
Expand Down
6 changes: 2 additions & 4 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,8 +156,7 @@ func (cb *ColumnBackfiller) init(
false, /* reverse */
descpb.ScanLockingStrength_FOR_NONE,
descpb.ScanLockingWaitPolicy_BLOCK,
0, /* lockTimeout */
false, /* isCheck */
0, /* lockTimeout */
&cb.alloc,
cb.mon,
tableArgs,
Expand Down Expand Up @@ -839,8 +838,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
false, /* reverse */
descpb.ScanLockingStrength_FOR_NONE,
descpb.ScanLockingWaitPolicy_BLOCK,
0, /* lockTimeout */
false, /* isCheck */
0, /* lockTimeout */
&ib.alloc,
ib.mon,
tableArgs,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/colbuilder/execplan.go
Original file line number Diff line number Diff line change
Expand Up @@ -170,7 +170,7 @@ func supportedNatively(spec *execinfrapb.ProcessorSpec) error {
return nil

case spec.Core.TableReader != nil:
if spec.Core.TableReader.IsCheck {
if spec.Core.TableReader.DeprecatedIsCheck {
return errors.Newf("scrub table reader is unsupported in vectorized")
}
return nil
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colfetcher/colbatch_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,7 +194,7 @@ func NewColBatchScan(
if nodeID, ok := flowCtx.NodeID.OptionalNodeID(); nodeID == 0 && ok {
return nil, errors.Errorf("attempting to create a ColBatchScan with uninitialized NodeID")
}
if spec.IsCheck {
if spec.DeprecatedIsCheck {
// cFetchers don't support these checks.
return nil, errors.AssertionFailedf("attempting to create a cFetcher with the IsCheck flag set")
}
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,8 +99,7 @@ func (d *deleteRangeNode) startExec(params runParams) error {
false, /* reverse */
descpb.ScanLockingStrength_FOR_NONE,
descpb.ScanLockingWaitPolicy_BLOCK,
0, /* lockTimeout */
false, /* isCheck */
0, /* lockTimeout */
params.p.alloc,
nil, /* memMonitor */
table,
Expand Down
12 changes: 3 additions & 9 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -1102,11 +1102,13 @@ func getIndexIdx(index catalog.Index, desc catalog.TableDescriptor) (uint32, err
func initTableReaderSpec(
n *scanNode,
) (*execinfrapb.TableReaderSpec, execinfrapb.PostProcessSpec, error) {
if n.isCheck {
return nil, execinfrapb.PostProcessSpec{}, errors.AssertionFailedf("isCheck no longer supported")
}
s := physicalplan.NewTableReaderSpec()
*s = execinfrapb.TableReaderSpec{
Table: *n.desc.TableDesc(),
Reverse: n.reverse,
IsCheck: n.isCheck,
Visibility: n.colCfg.visibility,
LockingStrength: n.lockingStrength,
LockingWaitPolicy: n.lockingWaitPolicy,
Expand All @@ -1122,13 +1124,6 @@ func initTableReaderSpec(
}
s.IndexIdx = indexIdx

// When a TableReader is running scrub checks, do not allow a
// post-processor. This is because the outgoing stream is a fixed
// format (rowexec.ScrubTypes).
if n.isCheck {
return s, execinfrapb.PostProcessSpec{}, nil
}

var post execinfrapb.PostProcessSpec
if n.hardLimit != 0 {
post.Limit = uint64(n.hardLimit)
Expand Down Expand Up @@ -4044,7 +4039,6 @@ func checkScanParallelizationIfLocal(
}
return true, nil
case *scanNode:
prohibitParallelization = n.isCheck
if len(n.reqOrdering) == 0 && n.parallelize {
hasScanNodeToParallelize = true
}
Expand Down
53 changes: 0 additions & 53 deletions pkg/sql/distsql_plan_scrub_physical.go

This file was deleted.

1 change: 0 additions & 1 deletion pkg/sql/distsql_spec_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,6 @@ func (e *distSQLSpecExecFactory) ConstructScan(
*trSpec = execinfrapb.TableReaderSpec{
Table: *tabDesc.TableDesc(),
Reverse: params.Reverse,
IsCheck: false,
Visibility: colCfg.visibility,
HasSystemColumns: scanContainsSystemColumns(&colCfg),
}
Expand Down
11 changes: 4 additions & 7 deletions pkg/sql/execinfrapb/processors_sql.proto
Original file line number Diff line number Diff line change
Expand Up @@ -57,11 +57,8 @@ enum ScanVisibility {
// columns of the rows that pass a filter expression.
//
// The "internal columns" of a TableReader (see ProcessorSpec) are all the
// columns of the table. If is_check is set, the TableReader will run additional
// data checking procedures and the "internal columns" are:
// - Error type (string).
// - Primary key as a string, if it was obtainable.
// - JSON of all decoded column values.
// columns of the table. Internally, only the values for the columns specified
// by needed_columns are to be populated.
message TableReaderSpec {
optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false];
// If 0, we use the primary index. If non-zero, we use the index_idx-th index,
Expand All @@ -80,8 +77,8 @@ message TableReaderSpec {
optional int64 limit_hint = 5 [(gogoproto.nullable) = false];

// Indicates whether the TableReader is being run as an exhaustive
// check. This is only true during SCRUB commands.
optional bool is_check = 6 [(gogoproto.nullable) = false];
// check. This is only true during SCRUB commands. No longer supported.
optional bool deprecated_is_check = 6 [(gogoproto.nullable) = false];

// Indicates the visibility level of the columns that should be returned.
// Normally, will be set to PUBLIC. Will be set to PUBLIC_AND_NOT_PUBLIC if
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/indexbackfiller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -385,7 +385,6 @@ INSERT INTO foo VALUES (1), (10), (100);
descpb.ScanLockingStrength_FOR_NONE,
descpb.ScanLockingWaitPolicy_BLOCK,
0,
false,
&alloc,
mm.Monitor(),
row.FetcherTableArgs{
Expand Down
7 changes: 3 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/scrub
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,14 @@ query TTTTTTTT
EXPERIMENTAL SCRUB TABLE t
-----

query TTTTTTTT
statement error not implemented
EXPERIMENTAL SCRUB TABLE t WITH OPTIONS PHYSICAL
-----

query TTTTTTTT
EXPERIMENTAL SCRUB TABLE t WITH OPTIONS INDEX ALL
------

query TTTTTTTT
statement error not implemented
EXPERIMENTAL SCRUB TABLE t WITH OPTIONS PHYSICAL, INDEX (name_idx)
-----

Expand Down Expand Up @@ -145,7 +144,7 @@ CREATE TABLE test.order (a INT, b INT, c INT, CONSTRAINT "primary" PRIMARY KEY (
statement ok
INSERT INTO test.order VALUES (0, 0, 0), (0, 0, 1), (0, 1, 0), (0, 1, 1), (1, 0, 0);

query TTTTTTTT
statement error not implemented
EXPERIMENTAL SCRUB TABLE test.order WITH OPTIONS PHYSICAL

# Test that scrubbing timestamp works as expected.
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/row/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -242,8 +242,7 @@ func DecodeRowInfo(
false, /* reverse */
descpb.ScanLockingStrength_FOR_NONE,
descpb.ScanLockingWaitPolicy_BLOCK,
0, /* lockTimeout */
false, /* isCheck */
0, /* lockTimeout */
&tree.DatumAlloc{},
nil, /* memMonitor */
tableArgs,
Expand Down
Loading

0 comments on commit 3186eeb

Please sign in to comment.