Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

br/lightning: add kv writer for external backend #46042

Merged
merged 10 commits into from
Aug 15, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 13 additions & 2 deletions br/pkg/lightning/backend/external/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,15 +8,22 @@ go_library(
"file.go",
"iter.go",
"kv_reader.go",
"sharedisk.go",
"stat_reader.go",
"writer.go",
],
importpath = "github.com/pingcap/tidb/br/pkg/lightning/backend/external",
visibility = ["//visibility:public"],
deps = [
"//br/pkg/lightning/backend",
"//br/pkg/lightning/backend/encode",
"//br/pkg/lightning/backend/kv",
"//br/pkg/lightning/common",
"//br/pkg/membuf",
"//br/pkg/storage",
"//kv",
"//util/logutil",
"//util/mathutil",
"//util/size",
"@com_github_pingcap_errors//:errors",
"@org_golang_x_sync//errgroup",
"@org_uber_go_zap//:zap",
Expand All @@ -31,15 +38,19 @@ go_test(
"codec_test.go",
"file_test.go",
"iter_test.go",
"writer_test.go",
],
embed = [":external"],
flaky = True,
shard_count = 13,
shard_count = 15,
deps = [
"//br/pkg/lightning/backend/kv",
"//br/pkg/lightning/common",
"//br/pkg/storage",
"@com_github_pingcap_errors//:errors",
"@com_github_stretchr_testify//require",
"@org_golang_x_exp//rand",
"@org_golang_x_exp//slices",
"@org_uber_go_atomic//:atomic",
],
)
12 changes: 10 additions & 2 deletions br/pkg/lightning/backend/external/file.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,8 +95,8 @@ func (s *KeyValueStore) AddKeyValue(key, value []byte) error {
s.rc.currProp.size += uint64(len(key) + len(value))
s.rc.currProp.keys++

if s.rc.currProp.size >= s.rc.propSizeIdxDistance ||
s.rc.currProp.keys >= s.rc.propKeysIdxDistance {
if s.rc.currProp.size >= s.rc.propSizeDist ||
s.rc.currProp.keys >= s.rc.propKeysDist {
newProp := *s.rc.currProp
s.rc.props = append(s.rc.props, &newProp)

Expand All @@ -109,6 +109,14 @@ func (s *KeyValueStore) AddKeyValue(key, value []byte) error {
return nil
}

// Close closes the KeyValueStore and append the last range property.
func (s *KeyValueStore) Close() {
lance6716 marked this conversation as resolved.
Show resolved Hide resolved
if s.rc.currProp.keys > 0 {
newProp := *s.rc.currProp
s.rc.props = append(s.rc.props, &newProp)
}
}

var statSuffix = filepath.Join("_stat", "0")

// GetAllFileNames returns a FilePathHandle that contains all data file paths
Expand Down
26 changes: 19 additions & 7 deletions br/pkg/lightning/backend/external/file_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,8 @@ func TestAddKeyValueMaintainRangeProperty(t *testing.T) {
writer, err := memStore.Create(ctx, "/test", nil)
require.NoError(t, err)
rc := &rangePropertiesCollector{
propSizeIdxDistance: 100,
propKeysIdxDistance: 2,
propSizeDist: 100,
propKeysDist: 2,
}
rc.reset()
initRC := *rc
Expand All @@ -49,7 +49,7 @@ func TestAddKeyValueMaintainRangeProperty(t *testing.T) {
// when not accumulated enough data, no range property will be added.
require.Equal(t, &initRC, rc)

// propKeysIdxDistance = 2, so after adding 2 keys, a new range property will be added.
// propKeysDist = 2, so after adding 2 keys, a new range property will be added.
k2, v2 := []byte("key2"), []byte("value2")
err = kvStore.AddKeyValue(k2, v2)
require.NoError(t, err)
Expand All @@ -72,12 +72,21 @@ func TestAddKeyValueMaintainRangeProperty(t *testing.T) {

err = writer.Close(ctx)
require.NoError(t, err)
kvStore.Close()
expected = &rangeProperty{
key: k3,
offset: uint64(len(k1) + len(v1) + 16 + len(k2) + len(v2) + 16),
size: uint64(len(k3) + len(v3)),
keys: 1,
}
require.Len(t, rc.props, 2)
require.Equal(t, expected, rc.props[1])

writer, err = memStore.Create(ctx, "/test2", nil)
require.NoError(t, err)
rc = &rangePropertiesCollector{
propSizeIdxDistance: 1,
propKeysIdxDistance: 100,
propSizeDist: 1,
propKeysDist: 100,
}
rc.reset()
kvStore, err = NewKeyValueStore(ctx, writer, rc, 2, 2)
Expand All @@ -103,6 +112,9 @@ func TestAddKeyValueMaintainRangeProperty(t *testing.T) {
keys: 1,
}
require.Equal(t, expected, rc.props[1])
kvStore.Close()
// Length of properties should not change after close.
require.Len(t, rc.props, 2)
err = writer.Close(ctx)
require.NoError(t, err)
}
Expand All @@ -116,8 +128,8 @@ func TestKVReadWrite(t *testing.T) {
writer, err := memStore.Create(ctx, "/test", nil)
require.NoError(t, err)
rc := &rangePropertiesCollector{
propSizeIdxDistance: 100,
propKeysIdxDistance: 2,
propSizeDist: 100,
propKeysDist: 2,
}
rc.reset()
kvStore, err := NewKeyValueStore(ctx, writer, rc, 1, 1)
Expand Down
16 changes: 8 additions & 8 deletions br/pkg/lightning/backend/external/iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,8 +66,8 @@ func TestMergeKVIter(t *testing.T) {
writer, err := memStore.Create(ctx, filename, nil)
require.NoError(t, err)
rc := &rangePropertiesCollector{
propSizeIdxDistance: 100,
propKeysIdxDistance: 2,
propSizeDist: 100,
propKeysDist: 2,
}
rc.reset()
kvStore, err := NewKeyValueStore(ctx, writer, rc, 1, 1)
Expand Down Expand Up @@ -118,8 +118,8 @@ func TestOneUpstream(t *testing.T) {
writer, err := memStore.Create(ctx, filename, nil)
require.NoError(t, err)
rc := &rangePropertiesCollector{
propSizeIdxDistance: 100,
propKeysIdxDistance: 2,
propSizeDist: 100,
propKeysDist: 2,
}
rc.reset()
kvStore, err := NewKeyValueStore(ctx, writer, rc, 1, 1)
Expand Down Expand Up @@ -196,8 +196,8 @@ func TestCorruptContent(t *testing.T) {
writer, err := memStore.Create(ctx, filename, nil)
require.NoError(t, err)
rc := &rangePropertiesCollector{
propSizeIdxDistance: 100,
propKeysIdxDistance: 2,
propSizeDist: 100,
propKeysDist: 2,
}
rc.reset()
kvStore, err := NewKeyValueStore(ctx, writer, rc, 1, 1)
Expand Down Expand Up @@ -249,8 +249,8 @@ func generateMockFileReader() *kvReader {
panic(err)
}
rc := &rangePropertiesCollector{
propSizeIdxDistance: 100,
propKeysIdxDistance: 2,
propSizeDist: 100,
propKeysDist: 2,
}
rc.reset()
kvStore, err := NewKeyValueStore(ctx, writer, rc, 1, 1)
Expand Down
36 changes: 0 additions & 36 deletions br/pkg/lightning/backend/external/sharedisk.go

This file was deleted.

Loading