Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
84875: backupccl: handle range keys in BACKUP r=erikgrinaker a=msbutler

Previously BACKUP would not back up range tombstones. With this patch, BACKUPs
with revision_history will backup range tombstones. Non-revision history backups
are not affected by this diff because MVCCExportToSST filters all tombstones
out of the backup already.

Specifically, this patch replaces the iterators used in the backup_processor
with the pebbleIterator, which has baked in range key support. This refactor
introduces a 5% regression in backup runtime, even when the backup has no range
keys, though #83051 hopes to address this gap. See details below on the
benchmark experiment.

At this point a backup with range keys is restorable, thanks to #84214. Note
that the restore codebase still touches iterators that are not range key aware.
This is not a problem because restored data does not have range keys, nor do
the empty ranges restore dumps data into. These iterators (e.g. in SSTBatcher
and in CheckSSTConflicts) will be updated when #70428 gets fixed.

Fixes #71155

Release note: none

To benchmark this diff, the following commands were used on the following sha
a5ccdc3, with and without this commit, over
three trials:
```
roachprod create -n 5 --gce-machine-type=n2-standard-16 $CLUSTER
roachprod put $CLUSTER [build] cockroach

roachprod wipe $CLUSTER; roachprod start $CLUSTER;
roachprod run $CLUSTER:1 -- "./cockroach workload init bank --rows 1000000000"
roachprod sql $CLUSTER:1 -- -e "BACKUP INTO 'gs://somebucket/michael-rangkey?AUTH=implicit'"
```

The backup on the control binary took on average 478 seconds with a stdev of 13
seconds, while the backup with the treatment binary took on average 499 seconds
with stddev of 8 seconds.

84883: kvserver: add server-side transaction retry metrics r=arulajmani a=arulajmani

This patch adds a few new metrics to track successful/failed
server-side transaction retries. Specifically, whenever we attempt
to retry a read or write batch or run into a read within uncertainty
interval error, we increment specific counters indicating if the
retry was successful or not.

Release note: None

85074: upgrades: add checkpointing for `raftAppliedIndexTermMigration` r=irfansharif a=erikgrinaker

Forward-port of #84909, for posterity.

----

The `raftAppliedIndexTermMigration` upgrade migration could be
unreliable. It iterates over all ranges and runs a `Migrate` request
which must be applied on all replicas. However, if any ranges merge or
replicas are unavailable, the migration fails and starts over from the
beginning. In large clusters with many ranges, this meant that it might
never complete.

This patch makes the upgrade more robust, by retrying each `Migrate`
request 5 times, and checkpointing the progress after every fifth batch
(1000 ranges), allowing resumption on failure. At some point this should
be made part of the migration infrastructure.

NB: This fix was initially submitted for 22.1, and even though the
migration will be removed for 22.2, it is forward-ported for posterity.

Release note: None

85086: eval: stop ignoring all ResolveOIDFromOID errors r=ajwerner a=rafiss

fixes #84448

The decision about whether an error is safe to ignore is made at the
place where the error is created/returned. This way, the callers don't
need to be aware of any new error codes that the implementation may
start returning in the future.

Release note (bug fix): Fixed incorrect error handling that could cause
casts to OID types to fail in some cases.

Co-authored-by: Michael Butler <[email protected]>
Co-authored-by: Arul Ajmani <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
5 people committed Jul 27, 2022
5 parents 0f100f3 + d5de88d + c2dd3e9 + 33f4a9d + 6817eb6 commit 811ca1f
Show file tree
Hide file tree
Showing 17 changed files with 513 additions and 54 deletions.
45 changes: 45 additions & 0 deletions pkg/ccl/backupccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -327,6 +328,15 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string)
// + expect-pausepoint: expects the schema change job to end up in a paused state because
// of a pausepoint error.
//
// - "kv" [args]
// Issues a kv request
//
// Supported arguments:
//
// + type: kv request type. Currently, only DeleteRange is supported
//
// + target: SQL target. Currently, only table names are supported.
//
// - "nudge-and-wait-for-temp-cleanup"
// Nudges the temporary object reconciliation loop to run, and waits for completion.
func TestDataDriven(t *testing.T) {
Expand Down Expand Up @@ -626,6 +636,15 @@ func TestDataDriven(t *testing.T) {
}
return ""

case "kv":
var request string
d.ScanArgs(t, "request", &request)

var target string
d.ScanArgs(t, "target", &target)
handleKVRequest(ctx, t, lastCreatedServer, ds, request, target)
return ""

case "save-cluster-ts":
server := lastCreatedServer
user := "root"
Expand Down Expand Up @@ -677,6 +696,32 @@ func TestDataDriven(t *testing.T) {
})
}

func handleKVRequest(
ctx context.Context, t *testing.T, server string, ds datadrivenTestState, request, target string,
) {
user := "root"
if request == "DeleteRange" {
var tableID uint32
err := ds.getSQLDB(t, server, user).QueryRow(`SELECT id FROM system.namespace WHERE name = $1`,
target).Scan(&tableID)
require.NoError(t, err)
bankSpan := makeTableSpan(tableID)
dr := roachpb.DeleteRangeRequest{
// Bogus span to make it a valid request.
RequestHeader: roachpb.RequestHeader{
Key: bankSpan.Key,
EndKey: bankSpan.EndKey,
},
UseRangeTombstone: true,
}
if _, err := kv.SendWrapped(ctx, ds.servers[server].DistSenderI().(*kvcoord.DistSender), &dr); err != nil {
t.Fatal(err)
}
} else {
t.Fatalf("Unknown kv request")
}
}

// findMostRecentJobWithType returns the most recently created job of `job_type`
// jobType.
func findMostRecentJobWithType(
Expand Down
92 changes: 68 additions & 24 deletions pkg/ccl/backupccl/file_sst_sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -268,32 +268,16 @@ func (s *fileSSTSink) write(ctx context.Context, resp exportedSpan) error {

log.VEventf(ctx, 2, "writing %s to backup file %s", span, s.outName)

// Copy SST content.
sst, err := storage.NewMemSSTIterator(resp.dataSST, false)
if err != nil {
// To speed up SST reading, surface all the point keys first, flush,
// then surface all the range keys and flush.
//
// TODO(msbutler): investigate using single a single iterator that surfaces
// all point keys first and then all range keys
if err := s.copyPointKeys(resp.dataSST); err != nil {
return err
}
defer sst.Close()

sst.SeekGE(storage.MVCCKey{Key: keys.MinKey})
for {
if valid, err := sst.Valid(); !valid || err != nil {
if err != nil {
return err
}
break
}
k := sst.UnsafeKey()
if k.Timestamp.IsEmpty() {
if err := s.sst.PutUnversioned(k.Key, sst.UnsafeValue()); err != nil {
return err
}
} else {
if err := s.sst.PutRawMVCC(sst.UnsafeKey(), sst.UnsafeValue()); err != nil {
return err
}
}
sst.Next()
if err := s.copyRangeKeys(resp.dataSST); err != nil {
return err
}

// If this span extended the last span added -- that is, picked up where it
Expand Down Expand Up @@ -328,6 +312,66 @@ func (s *fileSSTSink) write(ctx context.Context, resp exportedSpan) error {
return nil
}

func (s *fileSSTSink) copyPointKeys(dataSST []byte) error {
iterOpts := storage.IterOptions{
KeyTypes: storage.IterKeyTypePointsOnly,
LowerBound: keys.LocalMax,
UpperBound: keys.MaxKey,
}
iter, err := storage.NewPebbleMemSSTIterator(dataSST, false, iterOpts)
if err != nil {
return err
}
defer iter.Close()

for iter.SeekGE(storage.MVCCKey{Key: keys.MinKey}); ; iter.Next() {
if valid, err := iter.Valid(); !valid || err != nil {
if err != nil {
return err
}
break
}
k := iter.UnsafeKey()
if k.Timestamp.IsEmpty() {
if err := s.sst.PutUnversioned(k.Key, iter.UnsafeValue()); err != nil {
return err
}
} else {
if err := s.sst.PutRawMVCC(iter.UnsafeKey(), iter.UnsafeValue()); err != nil {
return err
}
}
}
return nil
}

func (s *fileSSTSink) copyRangeKeys(dataSST []byte) error {
iterOpts := storage.IterOptions{
KeyTypes: storage.IterKeyTypeRangesOnly,
LowerBound: keys.LocalMax,
UpperBound: keys.MaxKey,
}
iter, err := storage.NewPebbleMemSSTIterator(dataSST, false, iterOpts)
if err != nil {
return err
}
defer iter.Close()

for iter.SeekGE(storage.MVCCKey{Key: keys.MinKey}); ; iter.Next() {
if ok, err := iter.Valid(); err != nil {
return err
} else if !ok {
break
}
for _, rkv := range iter.RangeKeys() {
if err := s.sst.PutRawMVCCRangeKey(rkv.RangeKey, rkv.Value); err != nil {
return err
}
}
}
return nil
}

func generateUniqueSSTName(nodeID base.SQLInstanceID) string {
// The data/ prefix, including a /, is intended to group SSTs in most of the
// common file/bucket browse UIs.
Expand Down
84 changes: 84 additions & 0 deletions pkg/ccl/backupccl/testdata/backup-restore/rangekeys
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
# Tests that Backups without Revisions History and Restore properly handle
# range keys

new-server name=s1
----

exec-sql
CREATE DATABASE orig;
USE orig;
CREATE TABLE foo (i INT PRIMARY KEY, s STRING);
INSERT INTO foo VALUES (1, 'x'),(2,'y');
CREATE TABLE baz (i INT PRIMARY KEY, s STRING);
INSERT INTO baz VALUES (11, 'xx'),(22,'yy');
----

# Ensure a full backup properly captures range keys
# - with foo, delete then insert, and ensure no original data surfaces in restore
# - with baz: chill for now

kv request=DeleteRange target=foo
----

exec-sql
INSERT INTO foo VALUES (3,'z');
----

exec-sql
BACKUP INTO 'nodelocal://0/test-root/';
----

exec-sql
RESTORE DATABASE orig FROM LATEST IN 'nodelocal://0/test-root/' with new_db_name='orig1';
----

query-sql
SELECT count(*) from orig1.foo;
----
1

query-sql
SELECT count(*) from orig1.baz;
----
2

exec-sql
DROP DATABASE orig1 CASCADE
----

# Ensure incremental backup without revision history
# handles range tombstones:
# - with foo, insert and ensure latest data from previous backup surfaces in RESTORE
# - with baz, delete then insert, and ensure no data from previous backup surfaces in RESTORE

exec-sql
INSERT INTO foo VALUES (4,'a'),(5,'b');
----

kv request=DeleteRange target=baz
----

exec-sql
INSERT INTO baz VALUES (33,'zz');
----

exec-sql
BACKUP INTO LATEST IN 'nodelocal://0/test-root/';
----

exec-sql
RESTORE DATABASE orig FROM LATEST IN 'nodelocal://0/test-root/' with new_db_name='orig1';
----

query-sql
SELECT count(*) from orig1.foo
----
3

query-sql
SELECT count(*) from orig1.baz
----
1



Loading

0 comments on commit 811ca1f

Please sign in to comment.