Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
97587: allocator: check IO overload on lease transfer r=andrewbaptist a=kvoli

Previously, the allocator would return lease transfer targets without
considering the IO overload of stores involved. When leases would
transfer to the IO overloaded stores, service latency tended to degrade.

This commit adds IO overload checks prior to lease transfers. The IO
overload checks are similar to the IO overload checks for allocating
replicas in #97142.

The checks work by comparing a candidate store against
`kv.allocator.lease_io_overload_threshold` and the mean of other candidates.
If the candidate store is equal to or greater than both these values, it
is considered IO overloaded. The default value is 0.5.

The current leaseholder has to meet a higher bar to be considered IO
overloaded. It must have an IO overload score greater or equal to
`kv.allocator.lease_shed_io_overload_threshold`. The default value is
0.9.

The level of enforcement for IO overload is controlled by
`kv.allocator.lease_io_overload_threshold_enforcement` controls the
action taken when a candidate store for a lease transfer is IO overloaded.

- `ignore`: ignore IO overload scores entirely during lease transfers
  (effectively disabling this mechanism);
- `block_transfer_to`: lease transfers only consider stores that aren't
  IO overloaded (existing leases on IO overloaded stores are left as
  is);
- `shed`: actively shed leases from IO overloaded stores to less IO
  overloaded stores (this is a super-set of block_transfer_to).

The default is `block_transfer_to`.

This commit also updates the existing replica IO overload checks to be
prefixed with `Replica`, to avoid confusion between lease and replica
IO overload checks.

Resolves: #96508

Release note (ops change): Range leases will no longer be transferred to
stores which are IO overloaded.

98041: backupccl: fix off by one index in fileSSTSink file extension r=rhu713 a=rhu713

Currently, the logic that extends the last flushed file fileSSTSink does not trigger if there is only one flushed file. This failure to extend the first flushed file can result in file entries in the backup manifest with duplicate start keys. For example, if the first export response written to the sink contains partial entries of a single key `a`, then the span of the first file will be `a-a`, and the span of the subsequent file will always be `a-<end_key>`. The presence of these duplicate start keys breaks the encoding of the external manifest files list SST as the file path + start key combination in the manifest are assumed to be unique.

Fixes #97953 

Release note: None

98072: backupccl: replace restore2TB and restoretpccInc tests r=lidorcarmel a=msbutler

This patch removes the restore2TB* roachtests which ran a 2TB bank restore to
benchmark restore performance across a few hardware configurations. This patch
also replaces the `restoreTPCCInc/nodes=10` test which tested our ability to
handle a backup with a long chain.

This patch also adds:
1. `restore/tpce/400GB/aws/nodes=4/cpus=16` to measure how per-node throughput
scales when the per node vcpu count doubles relative to default.
2. `restore/tpce/400GB/aws/nodes=8/cpus=8` to measure how per-node throughput
scales when the number of nodes doubles relative to default.
3. `restore/tpce/400GB/aws/backupsIncluded=48/nodes=4/cpus=8` to measure
restore reliability and performance on 48 length long backup chain relative to
default.

A future patch will update the fixtures used in the restore node shutdown
scripts, and add more perf based tests.

Fixes #92699

Release note: None

Co-authored-by: Austen McClernon <austen@cockroachlabs.com>
Co-authored-by: Rui Hu <rui@cockroachlabs.com>
Co-authored-by: Michael Butler <butler@cockroachlabs.com>
  • Loading branch information
4 people committed Mar 7, 2023
4 parents ceb2940 + 3f1263d + 6b39306 + b588f1f commit a95ffcd
Show file tree
Hide file tree
Showing 13 changed files with 732 additions and 444 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -172,6 +172,7 @@ go_test(
"bench_test.go",
"create_scheduled_backup_test.go",
"datadriven_test.go",
"file_sst_sink_test.go",
"full_cluster_backup_restore_test.go",
"generative_split_and_scatter_processor_test.go",
"key_rewriter_test.go",
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4000,7 +4000,7 @@ func TestBackupRestoreChecksum(t *testing.T) {
}

// Corrupt one of the files in the backup.
f, err := os.OpenFile(filepath.Join(dir, backupManifest.Files[1].Path), os.O_WRONLY, 0)
f, err := os.OpenFile(filepath.Join(dir, backupManifest.Files[0].Path), os.O_WRONLY, 0)
if err != nil {
t.Fatalf("%+v", err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/file_sst_sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ func (s *fileSSTSink) write(ctx context.Context, resp exportedSpan) error {
// If this span extended the last span added -- that is, picked up where it
// ended and has the same time-bounds -- then we can simply extend that span
// and add to its entry counts. Otherwise we need to record it separately.
if l := len(s.flushedFiles) - 1; l > 0 && s.flushedFiles[l].Span.EndKey.Equal(span.Key) &&
if l := len(s.flushedFiles) - 1; l >= 0 && s.flushedFiles[l].Span.EndKey.Equal(span.Key) &&
s.flushedFiles[l].EndTime.EqOrdering(resp.metadata.EndTime) &&
s.flushedFiles[l].StartTime.EqOrdering(resp.metadata.StartTime) {
s.flushedFiles[l].Span.EndKey = span.EndKey
Expand Down
137 changes: 137 additions & 0 deletions pkg/ccl/backupccl/file_sst_sink_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
// Copyright 2023 The Cockroach Authors.
//
// 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
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package backupccl

import (
"bytes"
"context"
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/blobs"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/gogo/protobuf/types"
"github.com/stretchr/testify/require"
)

// TestFileSSTSinkExtendOneFile is a regression test for a bug in fileSSTSink in
// which the sink fails to extend its last span added if there's only one file
// in the sink so far.
func TestFileSSTSinkExtendOneFile(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()
tc, sqlDB, _, cleanup := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication)
defer cleanup()

store, err := cloud.ExternalStorageFromURI(ctx, "userfile:///0",
base.ExternalIODirConfig{},
tc.Servers[0].ClusterSettings(),
blobs.TestEmptyBlobClientFactory,
username.RootUserName(),
tc.Servers[0].InternalDB().(isql.DB),
nil, /* limiters */
cloud.NilMetrics,
)
require.NoError(t, err)
sqlDB.Exec(t, `SET CLUSTER SETTING bulkio.backup.file_size = '20B'`)

// Never block.
progCh := make(chan execinfrapb.RemoteProducerMetadata_BulkProcessorProgress, 10)

sinkConf := sstSinkConf{
id: 1,
enc: nil,
progCh: progCh,
settings: &tc.Servers[0].ClusterSettings().SV,
}

sink := makeFileSSTSink(sinkConf, store)

getKeys := func(prefix string, n int) []byte {
var b bytes.Buffer
sst := storage.MakeBackupSSTWriter(ctx, nil, &b)
for i := 0; i < n; i++ {
require.NoError(t, sst.PutUnversioned([]byte(fmt.Sprintf("%s%08d", prefix, i)), nil))
}
sst.Close()
return b.Bytes()
}

exportResponse1 := exportedSpan{
metadata: backuppb.BackupManifest_File{
Span: roachpb.Span{
Key: []byte("b"),
EndKey: []byte("b"),
},
EntryCounts: roachpb.RowCount{
DataSize: 100,
Rows: 1,
},
StartTime: hlc.Timestamp{},
EndTime: hlc.Timestamp{},
LocalityKV: "",
},
dataSST: getKeys("b", 100),
revStart: hlc.Timestamp{},
completedSpans: 1,
atKeyBoundary: false,
}

exportResponse2 := exportedSpan{
metadata: backuppb.BackupManifest_File{
Span: roachpb.Span{
Key: []byte("b"),
EndKey: []byte("z"),
},
EntryCounts: roachpb.RowCount{
DataSize: 100,
Rows: 1,
},
StartTime: hlc.Timestamp{},
EndTime: hlc.Timestamp{},
LocalityKV: "",
},
dataSST: getKeys("c", 100),
revStart: hlc.Timestamp{},
completedSpans: 1,
atKeyBoundary: true,
}

require.NoError(t, sink.write(ctx, exportResponse1))
require.NoError(t, sink.write(ctx, exportResponse2))

close(progCh)

var progs []execinfrapb.RemoteProducerMetadata_BulkProcessorProgress
for p := range progCh {
progs = append(progs, p)
}

require.Equal(t, 1, len(progs))
var progDetails backuppb.BackupManifest_Progress
if err := types.UnmarshalAny(&progs[0].ProgressDetails, &progDetails); err != nil {
t.Fatal(err)
}

// Verify that the file in the sink was properly extended and there is only 1
// file in the progress details.
require.Equal(t, 1, len(progDetails.Files))
}
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ func TestShowBackup(t *testing.T) {
defer cleanupEmptyCluster()
sqlDB.ExecMultiple(t, strings.Split(`
SET CLUSTER SETTING sql.cross_db_fks.enabled = TRUE;
SET CLUSTER SETTING bulkio.backup.file_size = '1';
CREATE TYPE data.welcome AS ENUM ('hello', 'hi');
USE data; CREATE SCHEMA sc;
CREATE TABLE data.sc.t1 (a INT);
Expand Down
Loading

0 comments on commit a95ffcd

Please sign in to comment.