Skip to content

Commit 3ac1c85

Browse files
craig[bot]mgartnerrafiss
committed
153599: sql: fix incorrect min timestamp error check r=mgartner a=mgartner Release note: None 153814: sql: set always_distribute_full_scans for internal validation queries r=rafiss a=rafiss The validation queries we run after a backfill all use full table scans, and usually there are no stats available. Since dd57482 was merged, we need to explicitly isntruct the optimizer to distribute these full scans. fixes #152859 Release note: None Co-authored-by: Marcus Gartner <[email protected]> Co-authored-by: Rafi Shamim <[email protected]>
3 parents f753ef0 + 63ee6a9 + 27c8606 commit 3ac1c85

File tree

6 files changed

+40
-11
lines changed

6 files changed

+40
-11
lines changed

pkg/cmd/roachtest/tests/schemachange.go

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -433,13 +433,6 @@ func makeSchemaChangeBulkIngestTest(
433433
db := c.Conn(ctx, t.L(), 1)
434434
defer db.Close()
435435

436-
// TODO(152859): Temporarily disable soft limit for distribute scan to
437-
// test performance regression fix (see #152295).
438-
t.L().Printf("Setting use_soft_limit_for_distribute_scan = false")
439-
if _, err := db.Exec("SET use_soft_limit_for_distribute_scan = false"); err != nil {
440-
t.Fatal(err)
441-
}
442-
443436
t.L().Printf("Computing table statistics manually")
444437
if _, err := db.Exec("CREATE STATISTICS stats from bulkingest.bulkingest"); err != nil {
445438
t.Fatal(err)

pkg/sql/backfill.go

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1541,6 +1541,10 @@ func ValidateConstraint(
15411541
runHistoricalTxn descs.HistoricalInternalExecTxnRunner,
15421542
execOverride sessiondata.InternalExecutorOverride,
15431543
) (err error) {
1544+
// Validation queries use full table scans which we always want to distribute.
1545+
// See https://github.com/cockroachdb/cockroach/issues/152859.
1546+
execOverride.AlwaysDistributeFullScans = true
1547+
15441548
tableDesc, err = tableDesc.MakeFirstMutationPublic(catalog.IgnoreConstraints)
15451549
if err != nil {
15461550
return err
@@ -1666,6 +1670,10 @@ func ValidateInvertedIndexes(
16661670
execOverride sessiondata.InternalExecutorOverride,
16671671
protectedTSManager scexec.ProtectedTimestampManager,
16681672
) (err error) {
1673+
// Validation queries use full table scans which we always want to distribute.
1674+
// See https://github.com/cockroachdb/cockroach/issues/152859.
1675+
execOverride.AlwaysDistributeFullScans = true
1676+
16691677
grp := ctxgroup.WithContext(ctx)
16701678
invalid := make(chan descpb.IndexID, len(indexes))
16711679

@@ -1769,6 +1777,11 @@ func countExpectedRowsForInvertedIndex(
17691777
) (int64, error) {
17701778
desc := tableDesc
17711779
start := timeutil.Now()
1780+
1781+
// Validation queries use full table scans which we always want to distribute.
1782+
// See https://github.com/cockroachdb/cockroach/issues/152859.
1783+
execOverride.AlwaysDistributeFullScans = true
1784+
17721785
if withFirstMutationPublic {
17731786
// Make the mutations public in an in-memory copy of the descriptor and
17741787
// add it to the Collection's synthetic descriptors, so that we can use
@@ -1867,6 +1880,10 @@ func ValidateForwardIndexes(
18671880
execOverride sessiondata.InternalExecutorOverride,
18681881
protectedTSManager scexec.ProtectedTimestampManager,
18691882
) (err error) {
1883+
// Validation queries use full table scans which we always want to distribute.
1884+
// See https://github.com/cockroachdb/cockroach/issues/152859.
1885+
execOverride.AlwaysDistributeFullScans = true
1886+
18701887
grp := ctxgroup.WithContext(ctx)
18711888

18721889
invalid := make(chan descpb.IndexID, len(indexes))
@@ -1983,6 +2000,10 @@ func populateExpectedCounts(
19832000
runHistoricalTxn descs.HistoricalInternalExecTxnRunner,
19842001
execOverride sessiondata.InternalExecutorOverride,
19852002
) (int64, error) {
2003+
// Validation queries use full table scans which we always want to distribute.
2004+
// See https://github.com/cockroachdb/cockroach/issues/152859.
2005+
execOverride.AlwaysDistributeFullScans = true
2006+
19862007
desc := tableDesc
19872008
if withFirstMutationPublic {
19882009
// The query to count the expected number of rows can reference columns
@@ -2049,6 +2070,10 @@ func countIndexRowsAndMaybeCheckUniqueness(
20492070
runHistoricalTxn descs.HistoricalInternalExecTxnRunner,
20502071
execOverride sessiondata.InternalExecutorOverride,
20512072
) (int64, error) {
2073+
// Validation queries use full table scans which we always want to distribute.
2074+
// See https://github.com/cockroachdb/cockroach/issues/152859.
2075+
execOverride.AlwaysDistributeFullScans = true
2076+
20522077
// If we are doing a REGIONAL BY ROW locality change, we can
20532078
// bypass the uniqueness check below as we are only adding or
20542079
// removing an implicit partitioning column. Scan the

pkg/sql/check.go

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -63,11 +63,17 @@ func validateCheckExpr(
6363
queryStr = fmt.Sprintf(`SELECT %s FROM [%d AS t]@[%d] WHERE NOT (%s) LIMIT 1`, columns, tableDesc.GetID(), indexIDForValidation, exprStr)
6464
}
6565
log.Dev.Infof(ctx, "validating check constraint %q with query %q", formattedCkExpr, queryStr)
66+
67+
// Validation queries use full table scans which we always want to distribute.
68+
// See https://github.com/cockroachdb/cockroach/issues/152859.
69+
execOverride := sessiondata.NodeUserSessionDataOverride
70+
execOverride.AlwaysDistributeFullScans = true
71+
6672
violatingRow, err = txn.QueryRowEx(
6773
ctx,
6874
"validate check constraint",
6975
txn.KV(),
70-
sessiondata.NodeUserSessionDataOverride,
76+
execOverride,
7177
queryStr)
7278
if err != nil {
7379
return nil, formattedCkExpr, err

pkg/sql/conn_executor.go

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3874,8 +3874,6 @@ func (ex *connExecutor) GetPCRReaderTimestamp() hlc.Timestamp {
38743874
return hlc.Timestamp{}
38753875
}
38763876

3877-
var minTSErr = kvpb.NewMinTimestampBoundUnsatisfiableError(hlc.Timestamp{}, hlc.Timestamp{})
3878-
38793877
// resetEvalCtx initializes the fields of evalCtx that can change
38803878
// during a session (i.e. the fields not set by initEvalCtx).
38813879
//
@@ -3914,7 +3912,8 @@ func (ex *connExecutor) resetEvalCtx(evalCtx *extendedEvalContext, txn *kv.Txn,
39143912

39153913
// See resetPlanner for more context on setting the maximum timestamp for
39163914
// AOST read retries.
3917-
if err := ex.state.mu.autoRetryReason; err != nil && errors.Is(err, minTSErr) {
3915+
var minTSErr *kvpb.MinTimestampBoundUnsatisfiableError
3916+
if err := ex.state.mu.autoRetryReason; err != nil && errors.HasType(err, minTSErr) {
39183917
evalCtx.AsOfSystemTime.MaxTimestampBound = ex.extraTxnState.descCollection.GetMaxTimestampBound()
39193918
} else if newTxn {
39203919
evalCtx.AsOfSystemTime = nil

pkg/sql/internal.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -965,6 +965,9 @@ func applyOverrides(o sessiondata.InternalExecutorOverride, sd *sessiondata.Sess
965965
if o.BufferedWritesEnabled != nil {
966966
sd.BufferedWritesEnabled = *o.BufferedWritesEnabled
967967
}
968+
if o.AlwaysDistributeFullScans {
969+
sd.AlwaysDistributeFullScans = true
970+
}
968971
// For 25.2, we're being conservative and explicitly disabling buffered
969972
// writes for the internal executor.
970973
// TODO(yuzefovich): remove this for 25.3.

pkg/sql/sessiondata/internal.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -87,6 +87,9 @@ type InternalExecutorOverride struct {
8787
// BufferedWritesEnabled, if set, controls whether the buffered writes KV transaction
8888
// protocol is used for user queries on the current session.
8989
BufferedWritesEnabled *bool
90+
// AlwaysDistributeFullScans, if true, overrides the
91+
// always_distribute_full_scans session variable.
92+
AlwaysDistributeFullScans bool
9093
}
9194

9295
// NoSessionDataOverride is the empty InternalExecutorOverride which does not

0 commit comments

Comments
 (0)