Skip to content

Commit

Permalink
backupccl: fix off by one index in fileSSTSink file extension
Browse files Browse the repository at this point in the history
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.

Release note: None
  • Loading branch information
Rui Hu committed Mar 6, 2023
1 parent 7a3778b commit 6b39306
Show file tree
Hide file tree
Showing 5 changed files with 141 additions and 2 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

0 comments on commit 6b39306

Please sign in to comment.