Skip to content

Commit

Permalink
Merge #75248 #90377
Browse files Browse the repository at this point in the history
75248: kvserver: use EncodedError in SnapshotResponse r=aayushshah15 a=tbg

We were previously using a "Message" string to indicate details about an
error. We can do so much better now and actually encode the error. This
wasn't possible when this field was first added, but it is now, so let's
use it. As always, there's a migration concern, which means the old
field stays around & is populated as well as interpreted for one
release.

We then use this new-found freedom to improve which errors were marked
as "failed snapshot" errors. Previously, any error coming in on a
`SnapshotResponse` were considered snapshot errors and were considered
retriable. This was causing `TestLearnerSnapshotFailsRollback` to run
for 90s, as `TestCluster`'s replication changes use a [SucceedsSoon] to
retry snapshot errors - but that test actually injects an error that it
wants to fail-fast.  Now, since snapshot error marks propagate over the
wire, we can do the marking on the *sender* of the SnapshotResponse, and
we can only mark messages that correspond to an actual failure to apply
the snapshot (as opposed to an injected error, or a hard error due to a
malformed request). The test now takes around one second, for a rare 90x
speed-up.

As a drive-by, we're also removing `errMalformedSnapshot`, which became
unused when we stopped sending the raft log in raft snaps a few releases
back, and which had managed to hide from the `unused` lint.

[SucceedsSoon]: https://github.com/cockroachdb/cockroach/blob/37175f77bf374d1bcb76bc39a65149788be06134/pkg/testutils/testcluster/testcluster.go#L628-L631

Fixes #74621.
Fixes #87337.

Release note: None


90377: roachtest: configure ignore and block lists for 23.1 in roachtests r=rhu713 a=rhu713

Configure ignore and block lists for 23.1 in roachtests.

Release note: None
Epic: RE-253

Co-authored-by: Tobias Grieger <[email protected]>
Co-authored-by: Rui Hu <[email protected]>
  • Loading branch information
3 people committed Nov 3, 2022
3 parents 9c9d55d + 4490ef1 + 71dfc09 commit d2171af
Show file tree
Hide file tree
Showing 18 changed files with 164 additions and 72 deletions.
5 changes: 5 additions & 0 deletions pkg/cmd/roachtest/tests/activerecord_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ var activeRecordBlocklists = blocklistsForVersion{
{"v21.2", "activeRecordBlockList21_2", activeRecordBlockList21_2, "activeRecordIgnoreList21_2", activeRecordIgnoreList21_2},
{"v22.1", "activeRecordBlockList22_1", activeRecordBlockList22_1, "activeRecordIgnoreList22_1", activeRecordIgnoreList22_1},
{"v22.2", "activeRecordBlockList22_2", activeRecordBlockList22_2, "activeRecordIgnoreList22_2", activeRecordIgnoreList22_2},
{"v23.1", "activeRecordBlockList23_1", activeRecordBlockList23_1, "activeRecordIgnoreList23_1", activeRecordIgnoreList23_1},
}

// These are lists of known activerecord test errors and failures.
Expand All @@ -29,6 +30,8 @@ var activeRecordBlocklists = blocklistsForVersion{
// Please keep these lists alphabetized for easy diffing.
// After a failed run, an updated version of this blocklist should be available
// in the test log.
var activeRecordBlockList23_1 = blocklist{}

var activeRecordBlockList22_2 = blocklist{}

var activeRecordBlockList22_1 = blocklist{}
Expand All @@ -39,6 +42,8 @@ var activeRecordBlockList21_1 = blocklist{}

var activeRecordBlockList20_2 = blocklist{}

var activeRecordIgnoreList23_1 = activeRecordIgnoreList22_2

var activeRecordIgnoreList22_2 = activeRecordIgnoreList22_1

var activeRecordIgnoreList22_1 = blocklist{
Expand Down
5 changes: 5 additions & 0 deletions pkg/cmd/roachtest/tests/gopg_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ var gopgBlocklists = blocklistsForVersion{
{"v21.2", "gopgBlockList21_2", gopgBlockList21_2, "gopgIgnoreList21_2", gopgIgnoreList21_2},
{"v22.1", "gopgBlockList22_1", gopgBlockList22_1, "gopgIgnoreList22_1", gopgIgnoreList22_1},
{"v22.2", "gopgBlockList22_2", gopgBlockList22_2, "gopgIgnoreList22_2", gopgIgnoreList22_2},
{"v23.1", "gopgBlockList23_1", gopgBlockList23_1, "gopgIgnoreList23_1", gopgIgnoreList23_1},
}

// These are lists of known gopg test errors and failures.
Expand All @@ -27,6 +28,8 @@ var gopgBlocklists = blocklistsForVersion{
// After a failed run, an updated version of this blocklist should be available
// in the test log.

var gopgBlockList23_1 = gopgBlockList22_2

var gopgBlockList22_2 = gopgBlockList22_1

var gopgBlockList22_1 = gopgBlockList21_2
Expand Down Expand Up @@ -81,6 +84,8 @@ var gopgBlockList20_2 = blocklist{
"v10.TestUnixSocket": "31113",
}

var gopgIgnoreList23_1 = gopgIgnoreList22_2

var gopgIgnoreList22_2 = gopgIgnoreList22_1

var gopgIgnoreList22_1 = gopgIgnoreList21_2
Expand Down
8 changes: 8 additions & 0 deletions pkg/cmd/roachtest/tests/hibernate_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,18 +16,22 @@ var hibernateBlocklists = blocklistsForVersion{
{"v21.2", "hibernateBlockList21_2", hibernateBlockList21_2, "hibernateIgnoreList21_2", hibernateIgnoreList21_2},
{"v22.1", "hibernateBlockList22_1", hibernateBlockList22_1, "hibernateIgnoreList22_1", hibernateIgnoreList22_1},
{"v22.2", "hibernateBlockList22_2", hibernateBlockList22_2, "hibernateIgnoreList22_2", hibernateIgnoreList22_2},
{"v23.1", "hibernateBlockList23_1", hibernateBlockList23_1, "hibernateIgnoreList23_1", hibernateIgnoreList23_1},
}

var hibernateSpatialBlocklists = blocklistsForVersion{
{"v21.1", "hibernateSpatialBlockList21_1", hibernateSpatialBlockList21_1, "", nil},
{"v21.2", "hibernateSpatialBlockList21_2", hibernateSpatialBlockList21_2, "", nil},
{"v22.1", "hibernateSpatialBlockList22_1", hibernateSpatialBlockList22_1, "", nil},
{"v22.2", "hibernateSpatialBlockList22_2", hibernateSpatialBlockList22_2, "", nil},
{"v23.1", "hibernateSpatialBlockList23_1", hibernateSpatialBlockList23_1, "", nil},
}

// Please keep these lists alphabetized for easy diffing.
// After a failed run, an updated version of this blocklist should be available
// in the test log.
var hibernateSpatialBlockList23_1 = blocklist{}

var hibernateSpatialBlockList22_2 = blocklist{}

var hibernateSpatialBlockList22_1 = blocklist{}
Expand All @@ -36,6 +40,8 @@ var hibernateSpatialBlockList21_2 = blocklist{}

var hibernateSpatialBlockList21_1 = blocklist{}

var hibernateBlockList23_1 = hibernateBlockList22_2

var hibernateBlockList22_2 = hibernateBlockList22_1

var hibernateBlockList22_1 = blocklist{
Expand Down Expand Up @@ -238,6 +244,8 @@ var hibernateBlockList20_2 = blocklist{
"org.hibernate.test.where.annotations.EagerManyToOneFetchModeSelectWhereTest.testAssociatedWhereClause": "unknown",
}

var hibernateIgnoreList23_1 = hibernateIgnoreList22_2

var hibernateIgnoreList22_2 = hibernateIgnoreList22_1

var hibernateIgnoreList22_1 = hibernateIgnoreList21_2
Expand Down
5 changes: 5 additions & 0 deletions pkg/cmd/roachtest/tests/jasyncsql_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,11 @@ package tests
var jasyncsqlBlocklists = blocklistsForVersion{
{"v22.1", "jasyncsqlBlocklist22_1", jasyncBlocklist22_1, "jasyncsqlIgnoreList22_1", jasyncsqlIgnoreList22_1},
{"v22.2", "jasyncsqlBlocklist22_2", jasyncBlocklist22_2, "jasyncsqlIgnoreList22_2", jasyncsqlIgnoreList22_2},
{"v23.1", "jasyncsqlBlocklist23_1", jasyncBlocklist23_1, "jasyncsqlIgnoreList23_1", jasyncsqlIgnoreList23_1},
}

var jasyncBlocklist23_1 = jasyncBlocklist22_2

var jasyncBlocklist22_2 = jasyncBlocklist22_1

var jasyncBlocklist22_1 = blocklist{
Expand Down Expand Up @@ -75,6 +78,8 @@ var jasyncBlocklist22_1 = blocklist{
"com.github.aysnc.sql.db.integration.pool.SuspendingPoolSpec.transactions with pool should commit simple inserts , prepared statements": "unknown",
}

var jasyncsqlIgnoreList23_1 = jasyncsqlIgnoreList22_2

var jasyncsqlIgnoreList22_2 = jasyncsqlIgnoreList22_1

var jasyncsqlIgnoreList22_1 = blocklist{}
5 changes: 5 additions & 0 deletions pkg/cmd/roachtest/tests/libpq_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,11 @@ var libPQBlocklists = blocklistsForVersion{
{"v21.2", "libPQBlocklist21_2", libPQBlocklist21_2, "libPQIgnorelist21_2", libPQIgnorelist21_2},
{"v22.1", "libPQBlocklist22_1", libPQBlocklist22_1, "libPQIgnorelist22_1", libPQIgnorelist22_1},
{"v22.2", "libPQBlocklist22_2", libPQBlocklist22_2, "libPQIgnorelist22_2", libPQIgnorelist22_2},
{"v23.1", "libPQBlocklist23_1", libPQBlocklist23_1, "libPQIgnorelist23_1", libPQIgnorelist23_1},
}

var libPQBlocklist23_1 = libPQBlocklist22_2

var libPQBlocklist22_2 = libPQBlocklist22_1

var libPQBlocklist22_1 = blocklist{
Expand Down Expand Up @@ -126,6 +129,8 @@ var libPQBlocklist20_2 = blocklist{
"pq.TestStringWithNul": "26366",
}

var libPQIgnorelist23_1 = libPQIgnorelist22_2

var libPQIgnorelist22_2 = libPQIgnorelist22_1

var libPQIgnorelist22_1 = libPQIgnorelist21_2
Expand Down
5 changes: 5 additions & 0 deletions pkg/cmd/roachtest/tests/liquibase_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,11 @@ var liquibaseBlocklists = blocklistsForVersion{
{"v21.2", "liquibaseBlocklist21_2", liquibaseBlocklist21_2, "liquibaseIgnorelist21_2", liquibaseIgnorelist21_2},
{"v22.1", "liquibaseBlocklist22_1", liquibaseBlocklist22_1, "liquibaseIgnorelist21_2", liquibaseIgnorelist22_1},
{"v22.2", "liquibaseBlocklist22_2", liquibaseBlocklist22_2, "liquibaseIgnorelist21_2", liquibaseIgnorelist22_2},
{"v23.1", "liquibaseBlocklist23_1", liquibaseBlocklist23_1, "liquibaseIgnorelist23_1", liquibaseIgnorelist23_1},
}

var liquibaseBlocklist23_1 = liquibaseBlocklist22_2

var liquibaseBlocklist22_2 = liquibaseBlocklist22_1

var liquibaseBlocklist22_1 = blocklist{
Expand All @@ -34,6 +37,8 @@ var liquibaseBlocklist21_1 = liquibaseBlocklist20_2

var liquibaseBlocklist20_2 = blocklist{}

var liquibaseIgnorelist23_1 = liquibaseIgnorelist22_2

var liquibaseIgnorelist22_2 = liquibaseIgnorelist22_1

var liquibaseIgnorelist22_1 = liquibaseIgnorelist21_2
Expand Down
5 changes: 5 additions & 0 deletions pkg/cmd/roachtest/tests/pgx_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,14 @@ var pgxBlocklists = blocklistsForVersion{
{"v21.2", "pgxBlocklist21_2", pgxBlocklist21_2, "pgxIgnorelist21_2", pgxIgnorelist21_2},
{"v22.1", "pgxBlocklist22_1", pgxBlocklist22_1, "pgxIgnorelist22_1", pgxIgnorelist22_1},
{"v22.2", "pgxBlocklist22_2", pgxBlocklist22_2, "pgxIgnorelist22_2", pgxIgnorelist22_2},
{"v23.1", "pgxBlocklist23_1", pgxBlocklist23_1, "pgxIgnorelist23_1", pgxIgnorelist23_1},
}

// Please keep these lists alphabetized for easy diffing.
// After a failed run, an updated version of this blocklist should be available
// in the test log.
var pgxBlocklist23_1 = blocklist{}

var pgxBlocklist22_2 = blocklist{}

var pgxBlocklist22_1 = blocklist{}
Expand All @@ -31,6 +34,8 @@ var pgxBlocklist21_1 = blocklist{}

var pgxBlocklist20_2 = blocklist{}

var pgxIgnorelist23_1 = pgxIgnorelist22_2

var pgxIgnorelist22_2 = pgxIgnorelist22_1

var pgxIgnorelist22_1 = pgxIgnorelist21_2
Expand Down
5 changes: 5 additions & 0 deletions pkg/cmd/roachtest/tests/psycopg_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ var psycopgBlocklists = blocklistsForVersion{
{"v21.2", "psycopgBlockList21_2", psycopgBlockList21_2, "psycopgIgnoreList21_2", psycopgIgnoreList21_2},
{"v22.1", "psycopgBlockList22_1", psycopgBlockList22_1, "psycopgIgnoreList22_1", psycopgIgnoreList22_1},
{"v22.2", "psycopgBlockList22_2", psycopgBlockList22_2, "psycopgIgnoreList22_2", psycopgIgnoreList22_2},
{"v23.1", "psycopgBlockList23_1", psycopgBlockList23_1, "psycopgIgnoreList23_1", psycopgIgnoreList23_1},
}

// These are lists of known psycopg test errors and failures.
Expand All @@ -29,6 +30,8 @@ var psycopgBlocklists = blocklistsForVersion{
// Please keep these lists alphabetized for easy diffing.
// After a failed run, an updated version of this blocklist should be available
// in the test log.
var psycopgBlockList23_1 = blocklist{}

var psycopgBlockList22_2 = blocklist{}

var psycopgBlockList22_1 = blocklist{}
Expand All @@ -50,6 +53,8 @@ var psycopgBlockList20_2 = blocklist{
"tests.test_async_keyword.CancelTests.test_async_cancel": "41335",
}

var psycopgIgnoreList23_1 = psycopgIgnoreList22_2

var psycopgIgnoreList22_2 = psycopgIgnoreList22_1

var psycopgIgnoreList22_1 = psycopgIgnoreList21_2
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/kvserverpb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_library(
deps = [
"//pkg/roachpb",
"//pkg/util/hlc",
"@com_github_cockroachdb_errors//:errors",
],
)

Expand All @@ -38,6 +39,7 @@ proto_library(
"//pkg/storage/enginepb:enginepb_proto",
"//pkg/util/hlc:hlc_proto",
"//pkg/util/tracing/tracingpb:tracingpb_proto",
"@com_github_cockroachdb_errors//errorspb:errorspb_proto",
"@com_github_gogo_protobuf//gogoproto:gogo_proto",
"@com_google_protobuf//:timestamp_proto",
"@io_etcd_go_etcd_raft_v3//raftpb:raftpb_proto",
Expand All @@ -59,6 +61,7 @@ go_proto_library(
"//pkg/util/hlc",
"//pkg/util/tracing/tracingpb",
"//pkg/util/uuid", # keep
"@com_github_cockroachdb_errors//errorspb",
"@com_github_gogo_protobuf//gogoproto",
"@io_etcd_go_etcd_raft_v3//raftpb",
],
Expand Down
28 changes: 28 additions & 0 deletions pkg/kv/kvserver/kvserverpb/raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,5 +10,33 @@

package kvserverpb

import (
"context"

"github.com/cockroachdb/errors"
)

// SafeValue implements the redact.SafeValue interface.
func (SnapshotRequest_Type) SafeValue() {}

// Error returns the error contained in the snapshot response, if any.
//
// The bool indicates whether this message uses the deprecated behavior of
// encoding an error as a string.
func (m *DelegateSnapshotResponse) Error() (deprecated bool, _ error) {
return m.SnapResponse.Error()
}

// Error returns the error contained in the snapshot response, if any.
//
// The bool indicates whether this message uses the deprecated behavior of
// encoding an error as a string.
func (m *SnapshotResponse) Error() (deprecated bool, _ error) {
if m.Status != SnapshotResponse_ERROR {
return false, nil
}
if m.EncodedError.IsSet() {
return false, errors.DecodeError(context.Background(), m.EncodedError)
}
return true, errors.Newf("%s", m.DeprecatedMessage)
}
14 changes: 13 additions & 1 deletion pkg/kv/kvserver/kvserverpb/raft.proto
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ syntax = "proto3";
package cockroach.kv.kvserver.kvserverpb;
option go_package = "kvserverpb";

import "errorspb/errors.proto";
import "roachpb/errors.proto";
import "roachpb/internal_raft.proto";
import "roachpb/metadata.proto";
Expand Down Expand Up @@ -237,11 +238,22 @@ message SnapshotResponse {
reserved 4;
}
Status status = 1;
string message = 2;
// Message is a message explaining an ERROR return value. It is not set for any
// other status.
//
// As of 23.1, the encoded_error field is always used instead. 23.1 itself
// needs to populate both due to needing to be compatible with 22.2. Once
// the MinSupportedVersion is 23.1, this can be removed.
string deprecated_message = 2;
reserved 3;

// Traces from snapshot processing, returned on status APPLIED or ERROR.
repeated util.tracing.tracingpb.RecordedSpan collected_spans = 4 [(gogoproto.nullable) = false];

// encoded_error encodes the error when the status is ERROR.
//
// MIGRATION: only guaranteed to be set when the message field is no longer there.
errorspb.EncodedError encoded_error = 5 [(gogoproto.nullable) = false];
}

// DelegateSnapshotRequest is the request used to delegate send snapshot requests.
Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/markers.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,10 @@ import (
"github.com/cockroachdb/errors"
)

// errMarkSnapshotError is used as an error mark for errors that get returned
// to the initiator of a snapshot. This generally classifies errors as transient,
// i.e. communicates an intention for the caller to retry.
//
// NB: don't change the string here; this will cause cross-version issues
// since this singleton is used as a marker.
var errMarkSnapshotError = errors.New("snapshot failed")
Expand Down
15 changes: 6 additions & 9 deletions pkg/kv/kvserver/raft_transport.go
Original file line number Diff line number Diff line change
Expand Up @@ -329,12 +329,10 @@ func (t *RaftTransport) DelegateRaftSnapshot(stream MultiRaft_DelegateRaftSnapsh
}
// Check to ensure the header is valid.
if req == nil {
err := errors.New("client error: no message in first delegated snapshot request")
return stream.Send(
&kvserverpb.DelegateSnapshotResponse{
SnapResponse: &kvserverpb.SnapshotResponse{
Status: kvserverpb.SnapshotResponse_ERROR,
Message: "client error: no message in first delegated snapshot request",
},
SnapResponse: snapRespErr(err),
},
)
}
Expand Down Expand Up @@ -364,9 +362,8 @@ func (t *RaftTransport) RaftSnapshot(stream MultiRaft_RaftSnapshotServer) error
return err
}
if req.Header == nil {
return stream.Send(&kvserverpb.SnapshotResponse{
Status: kvserverpb.SnapshotResponse_ERROR,
Message: "client error: no header in first snapshot request message"})
err := errors.New("client error: no header in first snapshot request message")
return stream.Send(snapRespErr(err))
}
rmr := req.Header.RaftMessageRequest
handler, ok := t.getHandler(rmr.ToReplica.StoreID)
Expand Down Expand Up @@ -651,14 +648,14 @@ func (t *RaftTransport) DelegateSnapshot(
nodeID := req.DelegatedSender.NodeID
conn, err := t.dialer.Dial(ctx, nodeID, rpc.DefaultClass)
if err != nil {
return err
return errors.Mark(err, errMarkSnapshotError)
}
client := NewMultiRaftClient(conn)

// Creates a rpc stream between the leaseholder and sender.
stream, err := client.DelegateRaftSnapshot(ctx)
if err != nil {
return err
return errors.Mark(err, errMarkSnapshotError)
}
defer func() {
if err := stream.CloseSend(); err != nil {
Expand Down
25 changes: 6 additions & 19 deletions pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -2725,11 +2725,12 @@ func (r *Replica) sendSnapshot(
)
},
)

if err != nil {
return errors.Mark(err, errMarkSnapshotError)
// Only mark explicitly as snapshot error (which is retriable) if we timed out.
// Otherwise, it's up to the remote to add this mark where appropriate.
if errors.HasType(err, (*contextutil.TimeoutError)(nil)) {
err = errors.Mark(err, errMarkSnapshotError)
}
return nil
return err
}

// followerSnapshotsEnabled is used to enable or disable follower snapshots.
Expand Down Expand Up @@ -2884,7 +2885,7 @@ func (r *Replica) followerSendSnapshot(
}
}

err = contextutil.RunWithTimeout(
return contextutil.RunWithTimeout(
ctx, "send-snapshot", sendSnapshotTimeout, func(ctx context.Context) error {
return r.store.cfg.Transport.SendSnapshot(
ctx,
Expand All @@ -2897,20 +2898,6 @@ func (r *Replica) followerSendSnapshot(
)
},
)
if err != nil {
if errors.Is(err, errMalformedSnapshot) {
tag := fmt.Sprintf("r%d_%s", r.RangeID, snap.SnapUUID.Short())
if dir, err := r.store.checkpoint(ctx, tag); err != nil {
log.Warningf(ctx, "unable to create checkpoint %s: %+v", dir, err)
} else {
log.Warningf(ctx, "created checkpoint %s", dir)
}

log.Fatal(ctx, "malformed snapshot generated")
}
return errors.Mark(err, errMarkSnapshotError)
}
return nil
}

// replicasCollocated is used in AdminMerge to ensure that the ranges are
Expand Down
Loading

0 comments on commit d2171af

Please sign in to comment.