Skip to content

Commit 5798d47

Browse files
craig[bot]msbutleryuzefovich
committed
150978: roachtest: add logging to jobs/stress r=kev-cao a=msbutler job/stress recently timed out, and I couldn't figure out why. This patch adds a bit more logging for the next failure, and removes some deprecated machinery. Informs #150801 Release note: none 151500: sql: make DeleteRange chunk size metamorphic r=yuzefovich a=yuzefovich Previously, we used the hard-coded value of 600 for `row.TableTruncateChunkSize` which originally dictated the usage of DelRange KV requests when truncating the table and over time the constant's usage has been expanded to other places where DelRange was used. In particular, we have a fast-path deleteRange operator that uses it, and this commit makes the constant there metamorphic. Additionally, this commit separates out the constants that were unified in 3d19b62 so that the index deletion code (that also uses DelRange requests) has its own, and in tests (which seem to have rotted a bit over time), we now hard-code the previous value. Informs: #151294. Epic: None Release note: None 151758: roachtest: route schema workload flakes in pcr tests to foundations r=kev-cao a=msbutler Informs #150548 Informs #151645 Release note: none 151768: tests,sqlsmith: skip st_snap in TestRandomSyntaxSQLSmith r=yuzefovich a=yuzefovich There is a known issue with `st_snap` that causes a crash in geos, so we avoid generating a stmt with this builtin for now. Additionally, I noticed some random stmts that always will hit an error because ORDER BY clause contains a type that doesn't define an ordering, so this commit also adjusts the sqlsmith generation for REFCURSOR, JSONPATH, and PGVECTOR (to match `ensureColumnOrderable`). Informs: #151103 Fixes: #151580 Release note: None 151787: fingerprintutils: select database and tables with aost if specified r=kev-cao a=msbutler Previously, if a user wanted to fingerprint a database(s) with an AOST, the utility mistakenly selected the database/tables without an aost. This patch fixes this. Fixes #150548 Release note: none Co-authored-by: Michael Butler <butler@cockroachlabs.com> Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
6 parents f62865d + 8dbfd5c + 417eb04 + 9d9bce1 + 1b145e4 + 5c4e19e commit 5798d47

File tree

17 files changed

+132
-52
lines changed

17 files changed

+132
-52
lines changed

pkg/cmd/roachtest/tests/backup_restore_roundtrip.go

Lines changed: 12 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,16 @@ var (
4545
}
4646
)
4747

48+
func handleSchemaChangeWorkloadError(err error) error {
49+
// If the UNEXPECTED ERROR detail appears, the workload likely flaked.
50+
// Otherwise, the workload could have failed due to other reasons like a node
51+
// crash.
52+
if err != nil && strings.Contains(errors.FlattenDetails(err), "UNEXPECTED ERROR") {
53+
return registry.ErrorWithOwner(registry.OwnerSQLFoundations, errors.Wrapf(err, "schema change workload failed"))
54+
}
55+
return err
56+
}
57+
4858
const numFullBackups = 3
4959

5060
type roundTripSpecs struct {
@@ -261,18 +271,9 @@ func startBackgroundWorkloads(
261271
return nil, err
262272
}
263273

264-
handleChemaChangeError := func(err error) error {
265-
// If the UNEXPECTED ERROR detail appears, the workload likely flaked.
266-
// Otherwise, the workload could have failed due to other reasons like a node
267-
// crash.
268-
if err != nil && strings.Contains(errors.FlattenDetails(err), "UNEXPECTED ERROR") {
269-
return registry.ErrorWithOwner(registry.OwnerSQLFoundations, errors.Wrapf(err, "schema change workload failed"))
270-
}
271-
return err
272-
}
273274
err = c.RunE(ctx, option.WithNodes(workloadNode), scInit.String())
274275
if err != nil {
275-
return nil, handleChemaChangeError(err)
276+
return nil, handleSchemaChangeWorkloadError(err)
276277
}
277278

278279
run := func() (func(), error) {
@@ -289,7 +290,7 @@ func startBackgroundWorkloads(
289290
})
290291
stopSC := workloadWithCancel(m, func(ctx context.Context) error {
291292
if err := c.RunE(ctx, option.WithNodes(workloadNode), scRun.String()); err != nil {
292-
return handleChemaChangeError(err)
293+
return handleSchemaChangeWorkloadError(err)
293294
}
294295
return nil
295296
})

pkg/cmd/roachtest/tests/cluster_to_cluster.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1012,7 +1012,7 @@ func (rd *replicationDriver) maybeRunSchemaChangeWorkload(
10121012
if err != nil && ctx.Err() == nil {
10131013
// Implies the workload context was not cancelled and the workload cmd returned a
10141014
// different error.
1015-
return errors.Wrapf(err, `schema change workload context was not cancelled. Error returned by workload cmd`)
1015+
return errors.Wrapf(handleSchemaChangeWorkloadError(err), `schema change workload context was not cancelled. Error returned by workload cmd`)
10161016
}
10171017
return nil
10181018
})

pkg/cmd/roachtest/tests/jobs.go

Lines changed: 15 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,6 @@ func runJobsStress(ctx context.Context, t test.Test, c cluster.Cluster) {
6767
showJobsTimeout = 30 * time.Second
6868
pollerMinFrequencySeconds = 5
6969
workloadDuration = time.Minute * 5
70-
7170
}
7271
sqlDB := sqlutils.MakeSQLRunner(conn)
7372
sqlDB.Exec(t, "CREATE DATABASE d")
@@ -83,22 +82,24 @@ func runJobsStress(ctx context.Context, t test.Test, c cluster.Cluster) {
8382

8483
done := make(chan struct{})
8584
earlyExit := make(chan struct{}, 1)
86-
m := c.NewDeprecatedMonitor(ctx)
85+
group := t.NewErrorGroup()
8786

88-
m.Go(func(ctx context.Context) error {
87+
group.Go(func(ctx context.Context, l *logger.Logger) error {
8988
defer close(done)
9089
var testTimer timeutil.Timer
9190
testTimer.Reset(workloadDuration)
9291
select {
9392
case <-earlyExit:
93+
l.Printf("Exiting early")
9494
case <-testTimer.C:
95+
l.Printf("workload duration of %s elapsed", workloadDuration)
9596
}
9697
return nil
9798
})
9899

99-
randomPoller := func(f func(ctx context.Context, t test.Test, c cluster.Cluster, rng *rand.Rand) error) func(ctx context.Context) error {
100+
randomPoller := func(f func(ctx context.Context, t test.Test, c cluster.Cluster, rng *rand.Rand) error) func(ctx context.Context, _ *logger.Logger) error {
100101

101-
return func(ctx context.Context) error {
102+
return func(ctx context.Context, _ *logger.Logger) error {
102103
var pTimer timeutil.Timer
103104
defer pTimer.Stop()
104105
for {
@@ -111,6 +112,7 @@ func runJobsStress(ctx context.Context, t test.Test, c cluster.Cluster) {
111112
return nil
112113
case <-pTimer.C:
113114
if err := f(ctx, t, c, rng); err != nil {
115+
t.L().Printf("Error running periodic function: %s", err)
114116
earlyExit <- struct{}{}
115117
return err
116118
}
@@ -119,16 +121,19 @@ func runJobsStress(ctx context.Context, t test.Test, c cluster.Cluster) {
119121
}
120122
}
121123

122-
m.Go(randomPoller(checkJobQueryLatency))
124+
group.Go(randomPoller(checkJobQueryLatency))
123125

124-
m.Go(randomPoller(pauseResumeChangefeeds))
126+
group.Go(randomPoller(pauseResumeChangefeeds))
125127

126-
createTablesWithChangefeeds(ctx, t, c, rng)
128+
group.Go(func(ctx context.Context, _ *logger.Logger) error {
129+
createTablesWithChangefeeds(ctx, t, c, rng)
130+
return nil
131+
})
127132

128133
// TODO(msbutler): consider adding a schema change workload to the existing
129134
// tables to further stress the job system.
130135

131-
m.Wait()
136+
require.NoError(t, group.WaitE())
132137
checkJobSystemHealth(ctx, t, c, rng)
133138
}
134139

@@ -150,7 +155,7 @@ func createTablesWithChangefeeds(
150155
tableName := tableNamePrefix + fmt.Sprintf("%d", i)
151156
sqlDB.Exec(t, fmt.Sprintf(`CREATE TABLE %s %s`, tableName, tableSchema))
152157
sqlDB.Exec(t, fmt.Sprintf(`INSERT INTO %s VALUES (1, 'x'),(2,'y')`, tableName))
153-
sqlDB.Exec(t, fmt.Sprintf("CREATE CHANGEFEED FOR %s INTO 'null://' WITH gc_protect_expires_after='5m', protect_data_from_gc_on_pause", tableName))
158+
sqlDB.Exec(t, fmt.Sprintf("CREATE CHANGEFEED FOR %s INTO 'null://' WITH gc_protect_expires_after='2m', protect_data_from_gc_on_pause", tableName))
154159
if i%(tableCount/5) == 0 {
155160
t.L().Printf("Created %d tables so far", i)
156161
}

pkg/internal/sqlsmith/relational.go

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1790,12 +1790,19 @@ func (s *Smither) makeHaving(refs colRefs) *tree.Where {
17901790
}
17911791

17921792
func (s *Smither) isOrderable(typ *types.T) bool {
1793+
if typ.Family() == types.ArrayFamily {
1794+
typ = typ.ArrayContents()
1795+
}
1796+
if typ.Family() == types.RefCursorFamily || typ.Family() == types.JsonpathFamily {
1797+
// These types don't define an ordering function in PG.
1798+
return false
1799+
}
17931800
if s.postgres {
17941801
// PostGIS cannot order box2d types.
17951802
return typ.Family() != types.Box2DFamily
17961803
}
17971804
switch typ.Family() {
1798-
case types.TSQueryFamily, types.TSVectorFamily:
1805+
case types.TSQueryFamily, types.TSVectorFamily, types.PGVectorFamily:
17991806
// We can't order by these types - see #92165.
18001807
return false
18011808
default:

pkg/sql/backfill.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -57,7 +57,7 @@ const (
5757
// per chunk during an index truncation. This value is larger than the
5858
// other chunk constants because the operation involves only running a
5959
// DeleteRange().
60-
indexTruncateChunkSize = row.TableTruncateChunkSize
60+
indexTruncateChunkSize = 600
6161

6262
// indexTxnBackfillChunkSize is the maximum number index entries backfilled
6363
// per chunk during an index backfill done in a txn. The index backfill

pkg/sql/delete_range.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -120,7 +120,7 @@ func (d *deleteRangeNode) startExec(params runParams) error {
120120
// hits the key limit).
121121
for len(spans) != 0 {
122122
b := params.p.txn.NewBatch()
123-
b.Header.MaxSpanRequestKeys = row.TableTruncateChunkSize
123+
b.Header.MaxSpanRequestKeys = int64(row.DeleteRangeChunkSize(params.extendedEvalCtx.TestingKnobs.ForceProductionValues))
124124
b.Header.LockTimeout = params.SessionData().LockTimeout
125125
b.Header.DeadlockTimeout = params.SessionData().DeadlockTimeout
126126
d.deleteSpans(params, b, spans)

pkg/sql/drop_test.go

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,6 @@ import (
3333
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
3434
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
3535
"github.com/cockroachdb/cockroach/pkg/sql/isql"
36-
"github.com/cockroachdb/cockroach/pkg/sql/row"
3736
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
3837
"github.com/cockroachdb/cockroach/pkg/sql/tests"
3938
"github.com/cockroachdb/cockroach/pkg/testutils"
@@ -640,7 +639,8 @@ func TestDropTable(t *testing.T) {
640639
s := srv.ApplicationLayer()
641640
codec := s.Codec()
642641

643-
numRows := 2*row.TableTruncateChunkSize + 1
642+
const deprecatedTableTruncateChunkSize = 600
643+
numRows := 2*deprecatedTableTruncateChunkSize + 1
644644
if err := tests.CreateKVTable(sqlDB, "kv", numRows); err != nil {
645645
t.Fatal(err)
646646
}
@@ -754,7 +754,8 @@ func TestDropTableDeleteData(t *testing.T) {
754754
// TTL into the system with AddImmediateGCZoneConfig.
755755
defer sqltestutils.DisableGCTTLStrictEnforcement(t, systemDB)()
756756

757-
const numRows = 2*row.TableTruncateChunkSize + 1
757+
const deprecatedTableTruncateChunkSize = 600
758+
const numRows = 2*deprecatedTableTruncateChunkSize + 1
758759
const numKeys = 3 * numRows
759760
const numTables = 5
760761
var descs []catalog.TableDescriptor

pkg/sql/opt/exec/execbuilder/mutation.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -742,7 +742,7 @@ func (b *Builder) buildDeleteRange(del *memo.DeleteExpr) (execPlan, error) {
742742
// Mutations only allow auto-commit if there are no FK checks or cascades.
743743

744744
if maxRows, ok := b.indexConstraintMaxResults(&scan.ScanPrivate, scan.Relational()); ok {
745-
if maxKeys := maxRows * uint64(tab.FamilyCount()); maxKeys <= row.TableTruncateChunkSize {
745+
if maxKeys := maxRows * uint64(tab.FamilyCount()); maxKeys <= uint64(row.DeleteRangeChunkSize(b.evalCtx.TestingKnobs.ForceProductionValues)) {
746746
autoCommit = true
747747
}
748748
}

pkg/sql/opt/exec/execbuilder/testdata/delete

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
# LogicTest: local
1+
# LogicTest: local !metamorphic-batch-sizes
22

33
statement ok
44
CREATE TABLE kv (

pkg/sql/opt/exec/execbuilder/testdata/select_index

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
# LogicTest: local
1+
# LogicTest: local !metamorphic-batch-sizes
22

33
statement ok
44
CREATE TABLE t (

0 commit comments

Comments
 (0)