Skip to content

Commit

Permalink
storageccl: add trace to ExportMVCCToSST to track duration
Browse files Browse the repository at this point in the history
Release note: None
  • Loading branch information
adityamaru committed Jul 13, 2021
1 parent 68f87f5 commit fe57f71
Show file tree
Hide file tree
Showing 10 changed files with 29 additions and 15 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/storageccl/export.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ func evalExport(
var curSizeOfExportedSSTs int64
for start := args.Key; start != nil; {
destFile := &storage.MemFile{}
summary, resume, err := reader.ExportMVCCToSst(start, args.EndKey, args.StartTime,
summary, resume, err := reader.ExportMVCCToSst(ctx, start, args.EndKey, args.StartTime,
h.Timestamp, exportAllRevisions, targetSize, maxSize, useTBI, destFile)
if err != nil {
return result.Result{}, err
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/storageccl/export_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -569,7 +569,7 @@ func assertEqualKVs(
maxSize := uint64(0)
prevStart := start
sstFile := &storage.MemFile{}
summary, start, err = e.ExportMVCCToSst(start, endKey, startTime, endTime,
summary, start, err = e.ExportMVCCToSst(ctx, start, endKey, startTime, endTime,
exportAllRevisions, targetSize, maxSize, enableTimeBoundIteratorOptimization, sstFile)
require.NoError(t, err)
sst = sstFile.Data()
Expand Down Expand Up @@ -609,7 +609,7 @@ func assertEqualKVs(
if dataSizeWhenExceeded == maxSize {
maxSize--
}
_, _, err = e.ExportMVCCToSst(prevStart, endKey, startTime, endTime,
_, _, err = e.ExportMVCCToSst(ctx, prevStart, endKey, startTime, endTime,
exportAllRevisions, targetSize, maxSize, enableTimeBoundIteratorOptimization, &storage.MemFile{})
require.Regexp(t, fmt.Sprintf("export size \\(%d bytes\\) exceeds max size \\(%d bytes\\)",
dataSizeWhenExceeded, maxSize), err)
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -399,14 +399,15 @@ func (s spanSetReader) Closed() bool {

// ExportMVCCToSst is part of the storage.Reader interface.
func (s spanSetReader) ExportMVCCToSst(
ctx context.Context,
startKey, endKey roachpb.Key,
startTS, endTS hlc.Timestamp,
exportAllRevisions bool,
targetSize, maxSize uint64,
useTBI bool,
dest io.Writer,
) (roachpb.BulkOpSummary, roachpb.Key, error) {
return s.r.ExportMVCCToSst(startKey, endKey, startTS, endTS, exportAllRevisions, targetSize,
return s.r.ExportMVCCToSst(ctx, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize,
maxSize, useTBI, dest)
}

Expand Down
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ go_library(
"//pkg/util/stop",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/tracing",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1434,7 +1434,7 @@ func runExportToSst(
for i := 0; i < b.N; i++ {
startTS := hlc.Timestamp{WallTime: int64(numRevisions / 2)}
endTS := hlc.Timestamp{WallTime: int64(numRevisions + 2)}
_, _, err := engine.ExportMVCCToSst(keys.LocalMax, roachpb.KeyMax, startTS, endTS,
_, _, err := engine.ExportMVCCToSst(context.Background(), keys.LocalMax, roachpb.KeyMax, startTS, endTS,
exportAllRevisions, 0 /* targetSize */, 0 /* maxSize */, useTBI, noopWriter{})
if err != nil {
b.Fatal(err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -409,7 +409,7 @@ type Reader interface {
// This function looks at MVCC versions and intents, and returns an error if an
// intent is found.
ExportMVCCToSst(
startKey, endKey roachpb.Key, startTS, endTS hlc.Timestamp,
ctx context.Context, startKey, endKey roachpb.Key, startTS, endTS hlc.Timestamp,
exportAllRevisions bool, targetSize uint64, maxSize uint64, useTBI bool,
dest io.Writer,
) (_ roachpb.BulkOpSummary, resumeKey roachpb.Key, _ error)
Expand Down
8 changes: 4 additions & 4 deletions pkg/storage/mvcc_incremental_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,8 +152,8 @@ func assertExportedErrs(
) {
const big = 1 << 30
sstFile := &MemFile{}
_, _, err := e.ExportMVCCToSst(startKey, endKey, startTime, endTime, revisions, big, big,
useTBI, sstFile)
_, _, err := e.ExportMVCCToSst(context.Background(), startKey, endKey, startTime, endTime,
revisions, big, big, useTBI, sstFile)
require.Error(t, err)

if intentErr := (*roachpb.WriteIntentError)(nil); errors.As(err, &intentErr) {
Expand Down Expand Up @@ -181,8 +181,8 @@ func assertExportedKVs(
) {
const big = 1 << 30
sstFile := &MemFile{}
_, _, err := e.ExportMVCCToSst(startKey, endKey, startTime, endTime, revisions, big, big,
useTBI, sstFile)
_, _, err := e.ExportMVCCToSst(context.Background(), startKey, endKey, startTime, endTime,
revisions, big, big, useTBI, sstFile)
require.NoError(t, err)
data := sstFile.Data()
if data == nil {
Expand Down
19 changes: 15 additions & 4 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/oserror"
Expand Down Expand Up @@ -688,6 +689,7 @@ func (p *Pebble) Closed() bool {

// ExportMVCCToSst is part of the engine.Reader interface.
func (p *Pebble) ExportMVCCToSst(
ctx context.Context,
startKey, endKey roachpb.Key,
startTS, endTS hlc.Timestamp,
exportAllRevisions bool,
Expand All @@ -698,8 +700,8 @@ func (p *Pebble) ExportMVCCToSst(
r := wrapReader(p)
// Doing defer r.Free() does not inline.
maxIntentCount := MaxIntentsPerWriteIntentError.Get(&p.settings.SV)
summary, k, err := pebbleExportToSst(r, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize,
maxSize, useTBI, dest, maxIntentCount)
summary, k, err := pebbleExportToSst(ctx, r, startKey, endKey, startTS, endTS, exportAllRevisions,
targetSize, maxSize, useTBI, dest, maxIntentCount)
r.Free()
return summary, k, err
}
Expand Down Expand Up @@ -1407,6 +1409,7 @@ func (p *pebbleReadOnly) Closed() bool {

// ExportMVCCToSst is part of the engine.Reader interface.
func (p *pebbleReadOnly) ExportMVCCToSst(
ctx context.Context,
startKey, endKey roachpb.Key,
startTS, endTS hlc.Timestamp,
exportAllRevisions bool,
Expand All @@ -1418,7 +1421,8 @@ func (p *pebbleReadOnly) ExportMVCCToSst(
// Doing defer r.Free() does not inline.
maxIntentCount := MaxIntentsPerWriteIntentError.Get(&p.parent.settings.SV)
summary, k, err := pebbleExportToSst(
r, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize, maxSize, useTBI, dest, maxIntentCount)
ctx, r, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize, maxSize, useTBI, dest,
maxIntentCount)
r.Free()
return summary, k, err
}
Expand Down Expand Up @@ -1680,6 +1684,7 @@ func (p *pebbleSnapshot) Closed() bool {

// ExportMVCCToSst is part of the engine.Reader interface.
func (p *pebbleSnapshot) ExportMVCCToSst(
ctx context.Context,
startKey, endKey roachpb.Key,
startTS, endTS hlc.Timestamp,
exportAllRevisions bool,
Expand All @@ -1691,7 +1696,8 @@ func (p *pebbleSnapshot) ExportMVCCToSst(
// Doing defer r.Free() does not inline.
maxIntentCount := MaxIntentsPerWriteIntentError.Get(&p.settings.SV)
summary, k, err := pebbleExportToSst(
r, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize, maxSize, useTBI, dest, maxIntentCount)
ctx, r, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize, maxSize, useTBI, dest,
maxIntentCount)
r.Free()
return summary, k, err
}
Expand Down Expand Up @@ -1798,6 +1804,7 @@ func pebbleGetProto(
}

func pebbleExportToSst(
ctx context.Context,
reader Reader,
startKey, endKey roachpb.Key,
startTS, endTS hlc.Timestamp,
Expand All @@ -1807,6 +1814,10 @@ func pebbleExportToSst(
dest io.Writer,
maxIntentCount int64,
) (roachpb.BulkOpSummary, roachpb.Key, error) {
var span *tracing.Span
ctx, span = tracing.ChildSpan(ctx, "pebbleExportToSst")
_ = ctx // ctx is currently unused, but this new ctx should be used below in the future.
defer span.Finish()
sstWriter := MakeBackupSSTWriter(dest)
defer sstWriter.Close()

Expand Down
1 change: 1 addition & 0 deletions pkg/storage/pebble_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,7 @@ func (p *pebbleBatch) Closed() bool {

// ExportMVCCToSst is part of the engine.Reader interface.
func (p *pebbleBatch) ExportMVCCToSst(
ctx context.Context,
startKey, endKey roachpb.Key,
startTS, endTS hlc.Timestamp,
exportAllRevisions bool,
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -634,7 +634,7 @@ func TestSstExportFailureIntentBatching(t *testing.T) {
require.NoError(t, fillInData(ctx, engine, data))

destination := &MemFile{}
_, _, err := engine.ExportMVCCToSst(key(10), key(20000), ts(999), ts(2000),
_, _, err := engine.ExportMVCCToSst(ctx, key(10), key(20000), ts(999), ts(2000),
true, 0, 0, true, destination)
if len(expectedIntentIndices) == 0 {
require.NoError(t, err)
Expand Down

0 comments on commit fe57f71

Please sign in to comment.