Skip to content

Commit

Permalink
Merge #61610 #62238
Browse files Browse the repository at this point in the history
61610: kvserver: expose a more user friendly protected ts error r=pbardea,ajwerner a=adityamaru

We have seen several support issues related to BACKUPs where the gc.ttl
being too short for the backup cadence results in the protected ts
record failing to be "verified". The error bubbled up in this scenario
is a proto dump of the rangedescs and protected ts record we are
attempting to verify. While this has most information, it is not very
parseable. Further, in the case where the gc is the reason for failure
to verify, the error does not expose information about where the
threshold currently is.

This change strips out relevant information, and bubbles up a more
informative error message. The change in tandem with docs about possible
bakcup failure modes, should help reduce the number of support
escalations.

Informs: #57064

62238: workload/schemachange: put ADD REGION/PRIMARY REGION into operation_generator r=ajwerner a=otan

see individual commits for details

Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
  • Loading branch information
3 people committed Mar 23, 2021
3 parents 18297c1 + c3fc31d + ce69d70 commit 0796cbc
Show file tree
Hide file tree
Showing 18 changed files with 1,425 additions and 795 deletions.
2 changes: 1 addition & 1 deletion pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ ALL_TESTS = [
"//pkg/ccl/streamingccl/streamingutils:streamingutils_test",
"//pkg/ccl/streamingccl/streamproducer:streamproducer_test",
"//pkg/ccl/telemetryccl:telemetryccl_test",
"//pkg/ccl/testccl/workload/schemachange:schemachange_test",
"//pkg/ccl/utilccl/sampledataccl:sampledataccl_test",
"//pkg/ccl/utilccl:utilccl_test",
"//pkg/ccl/workloadccl/allccl:allccl_test",
Expand Down Expand Up @@ -345,7 +346,6 @@ ALL_TESTS = [
"//pkg/workload/cli:cli_test",
"//pkg/workload/faker:faker_test",
"//pkg/workload/movr:movr_test",
"//pkg/workload/schemachange:schemachange_test",
"//pkg/workload/tpcc:tpcc_test",
"//pkg/workload/workloadimpl:workloadimpl_test",
"//pkg/workload/workloadsql:workloadsql_test",
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/multi_region
Original file line number Diff line number Diff line change
Expand Up @@ -588,7 +588,7 @@ DATABASE alter_test_db ALTER DATABASE alter_test_db CONFIGURE ZONE USING
statement error region "test" does not exist
ALTER DATABASE alter_test_db ADD REGION "test"

statement error region "ap-southeast-2" already added to database
statement error pgcode 42710 region "ap-southeast-2" already added to database
ALTER DATABASE alter_test_db ADD REGION "ap-southeast-2"

statement error cannot add region
Expand Down
28 changes: 28 additions & 0 deletions pkg/ccl/testccl/workload/schemachange/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "schemachange_test",
srcs = [
"main_test.go",
"schema_change_external_test.go",
],
deps = [
"//pkg/base",
"//pkg/ccl",
"//pkg/ccl/utilccl",
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"//pkg/util/randutil",
"//pkg/workload",
"//pkg/workload/histogram",
"//pkg/workload/schemachange",
"@com_github_stretchr_testify//require",
"@org_golang_x_sync//errgroup",
],
)
Original file line number Diff line number Diff line change
@@ -1,12 +1,10 @@
// Copyright 2020 The Cockroach Authors.
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package schemachange_test

Expand All @@ -31,4 +29,4 @@ func TestMain(m *testing.M) {
os.Exit(m.Run())
}

//go:generate ../../util/leaktest/add-leaktest.sh *_test.go
//go:generate ../../../../util/leaktest/add-leaktest.sh *_test.go
Original file line number Diff line number Diff line change
@@ -1,12 +1,10 @@
// Copyright 2020 The Cockroach Authors.
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package schemachange_test

Expand All @@ -22,6 +20,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
_ "github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
Expand All @@ -35,6 +34,7 @@ import (

func TestWorkload(t *testing.T) {
defer leaktest.AfterTest(t)()
defer utilccl.TestingEnableEnterprise()()

skip.UnderStressRace(t, "times out")

Expand Down
46 changes: 36 additions & 10 deletions pkg/kv/kvserver/protectedts/ptverifier/verifier.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,9 @@
package ptverifier

import (
"bytes"
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
Expand Down Expand Up @@ -95,20 +97,44 @@ func makeVerificationBatch(r *ptpb.Record, aliveAt hlc.Timestamp) kv.Batch {

func parseResponse(b *kv.Batch, r *ptpb.Record) error {
rawResponse := b.RawResponse()
var failed []roachpb.RangeDescriptor
for _, r := range rawResponse.Responses {
resp := r.GetInner().(*roachpb.AdminVerifyProtectedTimestampResponse)
if len(resp.FailedRanges) == 0 {
var errBuilder bytes.Buffer
for _, resp := range rawResponse.Responses {
resp := resp.GetInner().(*roachpb.AdminVerifyProtectedTimestampResponse)
if len(resp.DeprecatedFailedRanges) == 0 && len(resp.VerificationFailedRanges) == 0 {
continue
}
if len(failed) == 0 {
failed = resp.FailedRanges
} else {
failed = append(failed, resp.FailedRanges...)

// Write the error header the first time we encounter failed ranges.
if errBuilder.Len() == 0 {
_, _ = errBuilder.WriteString(fmt.Sprintf("failed to verify protection record %s with ts: %s:\n",
r.ID.String(), r.Timestamp.String()))
}

useDeprecated := len(resp.VerificationFailedRanges) == 0
for _, failedRange := range resp.VerificationFailedRanges {
if failedRange.Reason != "" {
// Write the per range reason for failure.
_, _ = errBuilder.WriteString(fmt.Sprintf("range ID: %d, range span: %s - %s: %s\n",
failedRange.RangeID, failedRange.StartKey.String(), failedRange.EndKey.String(),
failedRange.Reason))
} else {
// If no reason was saved, dump relevant information.
_, _ = errBuilder.WriteString(fmt.Sprintf("range ID: %d, range span: %s - %s\n",
failedRange.RangeID, failedRange.StartKey.String(), failedRange.EndKey.String()))
}
}

if !useDeprecated {
continue
}

for _, rangeDesc := range resp.DeprecatedFailedRanges {
_, _ = errBuilder.WriteString(fmt.Sprintf("range ID: %d, range span: %s - %s\n",
rangeDesc.RangeID, rangeDesc.StartKey.String(), rangeDesc.EndKey.String()))
}
}
if len(failed) > 0 {
return errors.Errorf("failed to verify protection %v on %v", r, failed)
if errBuilder.Len() > 0 {
return errors.Newf("protected ts verification error: %s", errBuilder.String())
}
return nil
}
47 changes: 42 additions & 5 deletions pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ func TestVerifier(t *testing.T) {
if _, ok := ba.GetArg(roachpb.AdminVerifyProtectedTimestamp); ok {
var resp roachpb.BatchResponse
resp.Add(&roachpb.AdminVerifyProtectedTimestampResponse{
FailedRanges: []roachpb.RangeDescriptor{{
VerificationFailedRanges: []roachpb.AdminVerifyProtectedTimestampResponse_FailedRange{{
RangeID: 42,
StartKey: roachpb.RKey(r.Spans[0].Key),
EndKey: roachpb.RKey(r.Spans[0].EndKey),
Expand All @@ -146,7 +146,9 @@ func TestVerifier(t *testing.T) {
}
return ds.Send(ctx, ba)
}))
require.Regexp(t, "failed to verify protection.*r42", ptv.Verify(ctx, r.ID).Error())
require.Regexp(t, "protected ts verification error: failed to verify protection.*\n"+
"range ID: 42, range span: /Table/42 - /Table/43",
ptv.Verify(ctx, r.ID).Error())
ensureVerified(t, r.ID, false)
release(t, r.ID)
},
Expand All @@ -162,24 +164,59 @@ func TestVerifier(t *testing.T) {
if _, ok := ba.GetArg(roachpb.AdminVerifyProtectedTimestamp); ok {
var resp roachpb.BatchResponse
resp.Add(&roachpb.AdminVerifyProtectedTimestampResponse{
FailedRanges: []roachpb.RangeDescriptor{{
VerificationFailedRanges: []roachpb.AdminVerifyProtectedTimestampResponse_FailedRange{{
RangeID: 42,
StartKey: roachpb.RKey(r.Spans[0].Key),
EndKey: roachpb.RKey(r.Spans[0].EndKey),
Reason: "foo",
}},
})
resp.Add(&roachpb.AdminVerifyProtectedTimestampResponse{
FailedRanges: []roachpb.RangeDescriptor{{
VerificationFailedRanges: []roachpb.AdminVerifyProtectedTimestampResponse_FailedRange{{
RangeID: 12,
StartKey: roachpb.RKey(r.Spans[1].Key),
EndKey: roachpb.RKey(r.Spans[1].EndKey),
Reason: "bar",
}},
})
return &resp, nil
}
return ds.Send(ctx, ba)
}))
require.Regexp(t, "failed to verify protection.*r42.*r12", ptv.Verify(ctx, r.ID).Error())
require.Regexp(t, "protected ts verification error: failed to verify protection.*\n"+
"range ID: 42, "+
"range span: /Table/42 - /Table/43: foo\nrange ID: 12, "+
"range span: /Table/12 - /Table/13: bar",
ptv.Verify(ctx, r.ID).Error())
ensureVerified(t, r.ID, false)
release(t, r.ID)
},
},
{
// TODO(adityamaru): Remove in 21.2.
name: "verification failed with deprecated failed ranges response",
test: func(t *testing.T) {
defer senderFunc.Store(senderFunc.Load())
r := createRecord(t, 42)
senderFunc.Store(kv.SenderFunc(func(
ctx context.Context, ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, *roachpb.Error) {
if _, ok := ba.GetArg(roachpb.AdminVerifyProtectedTimestamp); ok {
var resp roachpb.BatchResponse
resp.Add(&roachpb.AdminVerifyProtectedTimestampResponse{
DeprecatedFailedRanges: []roachpb.RangeDescriptor{{
RangeID: 42,
StartKey: roachpb.RKey(r.Spans[0].Key),
EndKey: roachpb.RKey(r.Spans[0].EndKey),
}},
})
return &resp, nil
}
return ds.Send(ctx, ba)
}))
require.Regexp(t, "protected ts verification error: failed to verify protection."+
"*\nrange ID: 42, range span: /Table/42 - /Table/43",
ptv.Verify(ctx, r.ID).Error())
ensureVerified(t, r.ID, false)
release(t, r.ID)
},
Expand Down
29 changes: 25 additions & 4 deletions pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -3118,9 +3118,30 @@ func (r *Replica) adminScatter(
func (r *Replica) adminVerifyProtectedTimestamp(
ctx context.Context, args roachpb.AdminVerifyProtectedTimestampRequest,
) (resp roachpb.AdminVerifyProtectedTimestampResponse, err error) {
resp.Verified, err = r.protectedTimestampRecordApplies(ctx, &args)
if err == nil && !resp.Verified {
resp.FailedRanges = append(resp.FailedRanges, *r.Desc())
var doesNotApplyReason string
resp.Verified, doesNotApplyReason, err = r.protectedTimestampRecordApplies(ctx, &args)
if err != nil {
return resp, err
}

// In certain cases we do not want to return an error even if we failed to
// verify the protected ts record. This ensures that executeAdminBatch adds
// the response to the batch, thereby allowing us to aggregate the
// verification failures across all AdminVerifyProtectedTimestampRequests and
// construct a more informative error to show to the user.
if doesNotApplyReason != "" {
if !resp.Verified {
failedRange := roachpb.AdminVerifyProtectedTimestampResponse_FailedRange{
RangeID: int64(r.Desc().GetRangeID()),
StartKey: r.Desc().GetStartKey(),
EndKey: r.Desc().EndKey,
Reason: doesNotApplyReason,
}
resp.VerificationFailedRanges = append(resp.VerificationFailedRanges, failedRange)
// TODO(adityamaru): This is here for compatibility with 20.2, remove in
// 21.2.
resp.DeprecatedFailedRanges = append(resp.DeprecatedFailedRanges, *r.Desc())
}
}
return resp, err
return resp, nil
}
Loading

0 comments on commit 0796cbc

Please sign in to comment.