Skip to content

Commit 5da9204

Browse files
craig[bot]tbgdanhhz
committed
Merge #37668 #37701
37668: storage: fix and test a bogus source of replica divergence errors r=nvanbenschoten a=tbg An incompatibility in the consistency checks was introduced between v2.1 and v19.1. See individual commit messages and #37425 for details. Release note (bug fix): Fixed a potential source of (faux) replica inconsistencies that can be reported while running a mixed v19.1 / v2.1 cluster. This error (in that situation only) is benign and can be resolved by upgrading to the latest v19.1 patch release. Every time this error occurs a "checkpoint" is created which will occupy a large amount of disk space and which needs to be removed manually (see <store directory>/auxiliary/checkpoints). Release note (bug fix): Fixed a case in which `./cockroach quit` would return success even though the server process was still running in a severely degraded state. 37701: workloadcccl: fix two regressions in fixtures make/load r=nvanbenschoten a=danhhz The SQL database for all the tables in the BACKUPs created by `fixtures make` used to be "csv" (an artifact of the way we made them), but as of #37343 it's the name of the generator. This seems better so change `fixtures load` to match. The same PR also (accidentally) started adding foreign keys in the BACKUPs, but since there's one table per BACKUP (another artifact of the way we used to make fixtures), we can't restore the foreign keys. It'd be nice to switch to one BACKUP with all tables and get the foreign keys, but the UX of the postLoad hook becomes tricky and I don't have time right now to sort it all out. So, revert to the previous behavior (no fks in fixtures) for now. Release note: None Co-authored-by: Tobias Schottdorf <[email protected]> Co-authored-by: Daniel Harrison <[email protected]>
3 parents 5141c2c + e3ae436 + 407a4de commit 5da9204

File tree

10 files changed

+112
-42
lines changed

10 files changed

+112
-42
lines changed

pkg/ccl/workloadccl/allccl/all_test.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ const (
2626
directIngestion = true
2727
oneFilePerNode = 1
2828
noInjectStats = false
29+
noSkipPostLoad = false
2930
skipCSVRoundtrip = ``
3031
)
3132

@@ -84,7 +85,8 @@ func TestAllRegisteredImportFixture(t *testing.T) {
8485
sqlutils.MakeSQLRunner(db).Exec(t, `CREATE DATABASE d`)
8586

8687
if _, err := workloadccl.ImportFixture(
87-
ctx, db, gen, `d`, directIngestion, oneFilePerNode, noInjectStats, skipCSVRoundtrip,
88+
ctx, db, gen, `d`, directIngestion, oneFilePerNode, noInjectStats, noSkipPostLoad,
89+
skipCSVRoundtrip,
8890
); err != nil {
8991
t.Fatal(err)
9092
}

pkg/ccl/workloadccl/bench_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -33,9 +33,9 @@ func benchmarkImportFixture(b *testing.B, gen workload.Generator) {
3333

3434
b.StartTimer()
3535
const filesPerNode = 1
36+
const directIngest, noInjectStats, skipPostLoad, csvServer = true, false, true, ``
3637
importBytes, err := ImportFixture(
37-
ctx, db, gen, `d`, true /* directIngestion */, filesPerNode, false, /* injectStats */
38-
``, /* csvServer */
38+
ctx, db, gen, `d`, directIngest, filesPerNode, noInjectStats, skipPostLoad, csvServer,
3939
)
4040
require.NoError(b, err)
4141
bytes += importBytes

pkg/ccl/workloadccl/cliccl/fixtures.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -333,9 +333,10 @@ func fixturesImport(gen workload.Generator, urls []string, dbName string) error
333333
directIngestion := *fixturesImportDirectIngestionTable
334334
filesPerNode := *fixturesImportFilesPerNode
335335
injectStats := *fixturesImportInjectStats
336+
noSkipPostLoad := false
336337
csvServer := *fixturesMakeImportCSVServerURL
337338
bytes, err := workloadccl.ImportFixture(
338-
ctx, sqlDB, gen, dbName, directIngestion, filesPerNode, injectStats, csvServer,
339+
ctx, sqlDB, gen, dbName, directIngestion, filesPerNode, injectStats, noSkipPostLoad, csvServer,
339340
)
340341
if err != nil {
341342
return errors.Wrap(err, `importing fixture`)

pkg/ccl/workloadccl/fixture.go

Lines changed: 11 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -258,8 +258,10 @@ func MakeFixture(
258258
if _, err := sqlDB.Exec(`CREATE DATABASE IF NOT EXISTS ` + dbName); err != nil {
259259
return Fixture{}, err
260260
}
261-
const direct, stats, csvServer = false, false, ""
262-
if _, err := ImportFixture(ctx, sqlDB, gen, dbName, direct, filesPerNode, stats, csvServer); err != nil {
261+
const direct, stats, skipPostLoad, csvServer = false, false, true, ""
262+
if _, err := ImportFixture(
263+
ctx, sqlDB, gen, dbName, direct, filesPerNode, stats, skipPostLoad, csvServer,
264+
); err != nil {
263265
return Fixture{}, err
264266
}
265267
g := ctxgroup.WithContext(ctx)
@@ -293,6 +295,7 @@ func ImportFixture(
293295
directIngestion bool,
294296
filesPerNode int,
295297
injectStats bool,
298+
skipPostLoad bool,
296299
csvServer string,
297300
) (int64, error) {
298301
for _, t := range gen.Tables() {
@@ -339,8 +342,10 @@ func ImportFixture(
339342
if err := g.Wait(); err != nil {
340343
return 0, err
341344
}
342-
if err := runPostLoadSteps(ctx, sqlDB, gen); err != nil {
343-
return 0, err
345+
if !skipPostLoad {
346+
if err := runPostLoadSteps(ctx, sqlDB, gen); err != nil {
347+
return 0, err
348+
}
344349
}
345350
return atomic.LoadInt64(&bytesAtomic), nil
346351
}
@@ -449,13 +454,12 @@ func RestoreFixture(
449454
) (int64, error) {
450455
var bytesAtomic int64
451456
g := ctxgroup.WithContext(ctx)
457+
genName := fixture.Generator.Meta().Name
452458
for _, table := range fixture.Tables {
453459
table := table
454460
g.GoCtx(func(ctx context.Context) error {
455-
// The IMPORT ... CSV DATA command generates a backup with the table in
456-
// database `csv`.
457461
start := timeutil.Now()
458-
importStmt := fmt.Sprintf(`RESTORE csv.%s FROM $1 WITH into_db=$2`, table.TableName)
462+
importStmt := fmt.Sprintf(`RESTORE %s.%s FROM $1 WITH into_db=$2`, genName, table.TableName)
459463
var rows, index, tableBytes int64
460464
var discard interface{}
461465
if err := sqlDB.QueryRow(importStmt, table.BackupURI, database).Scan(

pkg/ccl/workloadccl/fixture_test.go

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -183,10 +183,11 @@ func TestImportFixture(t *testing.T) {
183183
}
184184

185185
const filesPerNode = 1
186+
const noSkipPostLoad = false
186187
sqlDB.Exec(t, `CREATE DATABASE distsort`)
187188
_, err := ImportFixture(
188189
ctx, db, gen, `distsort`, false /* directIngestion */, filesPerNode, true, /* injectStats */
189-
``, /* csvServer */
190+
noSkipPostLoad, ``, /* csvServer */
190191
)
191192
require.NoError(t, err)
192193
sqlDB.CheckQueryResults(t,
@@ -203,7 +204,7 @@ func TestImportFixture(t *testing.T) {
203204
sqlDB.Exec(t, `CREATE DATABASE direct`)
204205
_, err = ImportFixture(
205206
ctx, db, gen, `direct`, true /* directIngestion */, filesPerNode, false, /* injectStats */
206-
``, /* csvServer */
207+
noSkipPostLoad, ``, /* csvServer */
207208
)
208209
require.NoError(t, err)
209210
sqlDB.CheckQueryResults(t,
@@ -240,9 +241,10 @@ func TestImportFixtureCSVServer(t *testing.T) {
240241
}
241242

242243
const filesPerNode = 1
244+
const noDirectIngest, noInjectStats, noSkipPostLoad = false, false, true
243245
sqlDB.Exec(t, `CREATE DATABASE d`)
244246
_, err := ImportFixture(
245-
ctx, db, gen, `d`, false /* directIngestion */, filesPerNode, false /* injectStats */, ts.URL,
247+
ctx, db, gen, `d`, noDirectIngest, filesPerNode, noInjectStats, noSkipPostLoad, ts.URL,
246248
)
247249
require.NoError(t, err)
248250
sqlDB.CheckQueryResults(t,

pkg/cmd/roachtest/cluster.go

Lines changed: 55 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -1041,39 +1041,75 @@ func (c *cluster) FailOnDeadNodes(ctx context.Context, t *test) {
10411041
})
10421042
}
10431043

1044-
// FailOnReplicaDivergence fails the test if
1045-
// crdb_internal.check_consistency(true, '', '') indicates that any ranges'
1046-
// replicas are inconsistent with each other.
1047-
func (c *cluster) FailOnReplicaDivergence(ctx context.Context, t *test) {
1048-
if c.nodes < 1 {
1049-
return // unit tests
1050-
}
1051-
// TODO(tbg): n1 isn't necessarily online at this point. Try to sniff out
1052-
// a node that is.
1053-
db := c.Conn(ctx, 1)
1054-
defer db.Close()
1055-
1056-
c.l.Printf("running (fast) consistency checks")
1044+
// CheckReplicaDivergenceOnDB runs a fast consistency check of the whole keyspace
1045+
// against the provided db. If an inconsistency is found, it returns it in the
1046+
// error. Note that this will swallow errors returned directly from the consistency
1047+
// check since we know that such spurious errors are possibly without any relation
1048+
// to the check having failed.
1049+
func (c *cluster) CheckReplicaDivergenceOnDB(ctx context.Context, db *gosql.DB) error {
10571050
rows, err := db.QueryContext(ctx, `
10581051
SELECT t.range_id, t.start_key_pretty, t.status, t.detail
10591052
FROM
10601053
crdb_internal.check_consistency(true, '', '') as t
10611054
WHERE t.status NOT IN ('RANGE_CONSISTENT', 'RANGE_INDETERMINATE')`)
10621055
if err != nil {
1063-
c.l.Printf("%s", err)
1064-
return
1056+
// TODO(tbg): the checks can fail for silly reasons like missing gossiped
1057+
// descriptors, etc. -- not worth failing the test for. Ideally this would
1058+
// be rock solid.
1059+
c.l.Printf("consistency check failed with %v; ignoring", err)
1060+
return nil
10651061
}
1062+
var buf bytes.Buffer
10661063
for rows.Next() {
10671064
var rangeID int32
10681065
var prettyKey, status, detail string
10691066
if err := rows.Scan(&rangeID, &prettyKey, &status, &detail); err != nil {
1070-
c.l.Printf("%s", err)
1071-
break
1067+
return err
10721068
}
1073-
t.Fatalf("r%d (%s) is inconsistent: %s %s", rangeID, prettyKey, status, detail)
1069+
fmt.Fprintf(&buf, "r%d (%s) is inconsistent: %s %s\n", rangeID, prettyKey, status, detail)
10741070
}
10751071
if err := rows.Err(); err != nil {
1076-
c.l.Printf("%s", err)
1072+
return err
1073+
}
1074+
1075+
msg := buf.String()
1076+
if msg != "" {
1077+
return errors.New(msg)
1078+
}
1079+
return nil
1080+
}
1081+
1082+
// FailOnReplicaDivergence fails the test if
1083+
// crdb_internal.check_consistency(true, '', '') indicates that any ranges'
1084+
// replicas are inconsistent with each other. It uses the first node that
1085+
// is up to run the query.
1086+
func (c *cluster) FailOnReplicaDivergence(ctx context.Context, t *test) {
1087+
if c.nodes < 1 {
1088+
return // unit tests
1089+
}
1090+
1091+
// Find a live node to run against, if one exists.
1092+
var db *gosql.DB
1093+
for i := 1; i <= c.nodes; i++ {
1094+
db = c.Conn(ctx, i)
1095+
_, err := db.Exec(`SELECT 1`)
1096+
if err != nil {
1097+
_ = db.Close()
1098+
db = nil
1099+
continue
1100+
}
1101+
c.l.Printf("running (fast) consistency checks on node %d", i)
1102+
break
1103+
}
1104+
if db == nil {
1105+
c.l.Printf("no live node found, skipping consistency check")
1106+
return
1107+
}
1108+
1109+
defer db.Close()
1110+
1111+
if err := c.CheckReplicaDivergenceOnDB(ctx, db); err != nil {
1112+
t.Fatal(err)
10771113
}
10781114
}
10791115

pkg/cmd/roachtest/version.go

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import (
2424

2525
"github.com/cockroachdb/cockroach/pkg/util/binfetcher"
2626
_ "github.com/lib/pq"
27+
"github.com/pkg/errors"
2728
)
2829

2930
func registerVersion(r *registry) {
@@ -95,7 +96,7 @@ func registerVersion(r *registry) {
9596
// Make sure everyone is still running.
9697
for i := 1; i <= nodes; i++ {
9798
t.WorkerStatus("checking ", i)
98-
db := c.Conn(ctx, 1)
99+
db := c.Conn(ctx, i)
99100
defer db.Close()
100101
rows, err := db.Query(`SHOW DATABASES`)
101102
if err != nil {
@@ -104,6 +105,10 @@ func registerVersion(r *registry) {
104105
if err := rows.Close(); err != nil {
105106
return err
106107
}
108+
// Regression test for #37425.
109+
if err := c.CheckReplicaDivergenceOnDB(ctx, db); err != nil {
110+
return errors.Wrapf(err, "node %d", i)
111+
}
107112
}
108113
return nil
109114
}

pkg/server/admin.go

Lines changed: 23 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ import (
1919
"context"
2020
"encoding/json"
2121
"fmt"
22+
"os"
2223
"sort"
2324
"strconv"
2425
"strings"
@@ -1353,12 +1354,12 @@ func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_Dr
13531354
return nil
13541355
}
13551356

1356-
s.server.grpc.Stop()
1357-
13581357
go func() {
1359-
// The explicit closure here allows callers.Lookup() to return something
1360-
// sensible referring to this file (otherwise it ends up in runtime
1361-
// internals).
1358+
// TODO(tbg): why don't we stop the stopper first? Stopping the stopper
1359+
// first seems more reasonable since grpc.Stop closes the listener right
1360+
// away (and who knows whether gRPC-goroutines are tied up in some
1361+
// stopper task somewhere).
1362+
s.server.grpc.Stop()
13621363
s.server.stopper.Stop(ctx)
13631364
}()
13641365

@@ -1367,6 +1368,23 @@ func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_Dr
13671368
return nil
13681369
case <-ctx.Done():
13691370
return ctx.Err()
1371+
case <-time.After(10 * time.Second):
1372+
// This is a hack to work around the problem in
1373+
// https://github.com/cockroachdb/cockroach/issues/37425#issuecomment-494336131
1374+
//
1375+
// There appear to be deadlock scenarios in which we don't manage to
1376+
// fully stop the grpc server (which implies closing the listener, i.e.
1377+
// seeming dead to the outside world) or don't manage to shut down the
1378+
// stopper (the evidence in #37425 is inconclusive which one it is).
1379+
//
1380+
// Other problems in this area are known, such as
1381+
// https://github.com/cockroachdb/cockroach/pull/31692
1382+
//
1383+
// The signal-based shutdown path uses a similar time-based escape hatch.
1384+
// Until we spend (potentially lots of time to) understand and fix this
1385+
// issue, this will serve us well.
1386+
os.Exit(1)
1387+
return errors.New("unreachable")
13701388
}
13711389
}
13721390

pkg/sql/sem/builtins/generator_builtins.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -962,8 +962,10 @@ func (c *checkConsistencyGenerator) Start() error {
962962
Key: c.from,
963963
EndKey: c.to,
964964
},
965-
Mode: c.mode,
966-
WithDiff: true,
965+
Mode: c.mode,
966+
// No meaningful diff can be created if we're checking the stats only,
967+
// so request one only if a full check is run.
968+
WithDiff: c.mode == roachpb.ChecksumMode_CHECK_FULL,
967969
})
968970
if err := c.db.Run(c.ctx, &b); err != nil {
969971
return err

pkg/storage/batcheval/cmd_compute_checksum.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,7 @@ func declareKeysComputeChecksum(
4242
// Version numbers for Replica checksum computation. Requests silently no-op
4343
// unless the versions are compatible.
4444
const (
45-
ReplicaChecksumVersion = 3
45+
ReplicaChecksumVersion = 4
4646
ReplicaChecksumGCInterval = time.Hour
4747
)
4848

0 commit comments

Comments
 (0)