Skip to content

Commit

Permalink
sstable: trace/expensive log when sstable footer read is slow
Browse files Browse the repository at this point in the history
Relates to #3728
  • Loading branch information
sumeerbhola authored and RaduBerinde committed Jul 18, 2024
1 parent 9a7ff8c commit 5bc59f8
Show file tree
Hide file tree
Showing 5 changed files with 44 additions and 17 deletions.
2 changes: 1 addition & 1 deletion db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1437,7 +1437,7 @@ func TestTracing(t *testing.T) {
_, closer, err := d.Get([]byte("hello"))
require.NoError(t, err)
closer.Close()
readerInitTraceString := "reading 37 bytes took 5ms\nreading 419 bytes took 5ms\n"
readerInitTraceString := "reading 53 bytes took 5ms\nreading 37 bytes took 5ms\nreading 419 bytes took 5ms\n"
iterTraceString := "reading 27 bytes took 5ms\nreading 29 bytes took 5ms\n"
require.Equal(t, readerInitTraceString+iterTraceString, tracer.buf.String())

Expand Down
10 changes: 6 additions & 4 deletions internal/base/logger.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,11 +94,13 @@ func (b *InMemLogger) Fatalf(format string, args ...interface{}) {
type LoggerAndTracer interface {
Logger
// Eventf formats and emits a tracing log, if tracing is enabled in the
// current context.
// current context. It can also emit to a regular log, if expensive
// logging is enabled.
Eventf(ctx context.Context, format string, args ...interface{})
// IsTracingEnabled returns true if tracing is enabled. It can be used as an
// optimization to avoid calling Eventf (which will be a noop when tracing
// is not enabled) to avoid the overhead of boxing the args.
// IsTracingEnabled returns true if tracing is enabled for this context,
// or expensive logging is enabled. It can be used as an optimization to
// avoid calling Eventf (which will be a noop when tracing or expensive
// logging is not enabled) to avoid the overhead of boxing the args.
IsTracingEnabled(ctx context.Context) bool
}

Expand Down
28 changes: 19 additions & 9 deletions sstable/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -533,20 +533,14 @@ func (r *Reader) readBlock(
}

compressed := block.Alloc(int(bh.Length+block.TrailerLen), bufferPool)
readStartTime := time.Now()
readStopwatch := makeStopwatch()
var err error
if readHandle != nil {
err = readHandle.ReadAt(ctx, compressed.Get(), int64(bh.Offset))
} else {
err = r.readable.ReadAt(ctx, compressed.Get(), int64(bh.Offset))
}
readDuration := time.Since(readStartTime)
// TODO(sumeer): should the threshold be configurable.
const slowReadTracingThreshold = 5 * time.Millisecond
// For deterministic testing.
if deterministicReadBlockDurationForTesting {
readDuration = slowReadTracingThreshold
}
readDuration := readStopwatch.stop()
// Call IsTracingEnabled to avoid the allocations of boxing integers into an
// interface{}, unless necessary.
if readDuration >= slowReadTracingThreshold && r.opts.LoggerAndTracer.IsTracingEnabled(ctx) {
Expand Down Expand Up @@ -1067,7 +1061,7 @@ func NewReader(f objstorage.Readable, o ReaderOptions, extraOpts ...ReaderOption
ctx, r.readable, objstorage.ReadBeforeForNewReader, &preallocRH)
defer rh.Close()

footer, err := readFooter(ctx, f, rh)
footer, err := readFooter(ctx, f, rh, r.logger)
if err != nil {
r.err = err
return nil, r.Close()
Expand Down Expand Up @@ -1184,3 +1178,19 @@ func errCorruptIndexEntry(err error) error {
}
return err
}

type deterministicStopwatchForTesting struct {
startTime time.Time
}

func makeStopwatch() deterministicStopwatchForTesting {
return deterministicStopwatchForTesting{startTime: time.Now()}
}

func (w deterministicStopwatchForTesting) stop() time.Duration {
dur := time.Since(w.startTime)
if deterministicReadBlockDurationForTesting {
dur = slowReadTracingThreshold
}
return dur
}
17 changes: 16 additions & 1 deletion sstable/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ package sstable // import "github.com/cockroachdb/pebble/sstable"
import (
"context"
"encoding/binary"
"time"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
Expand Down Expand Up @@ -297,9 +298,15 @@ type footer struct {
footerBH block.Handle
}

// TODO(sumeer): should the threshold be configurable.
const slowReadTracingThreshold = 5 * time.Millisecond

// readHandle is optional.
func readFooter(
ctx context.Context, f objstorage.Readable, readHandle objstorage.ReadHandle,
ctx context.Context,
f objstorage.Readable,
readHandle objstorage.ReadHandle,
logger base.LoggerAndTracer,
) (footer, error) {
var footer footer
size := f.Size()
Expand All @@ -313,12 +320,20 @@ func readFooter(
off = 0
buf = buf[:size]
}
readStopwatch := makeStopwatch()
var err error
if readHandle != nil {
err = readHandle.ReadAt(ctx, buf, off)
} else {
err = f.ReadAt(ctx, buf, off)
}
readDuration := readStopwatch.stop()
// Call IsTracingEnabled to avoid the allocations of boxing integers into an
// interface{}, unless necessary.
if readDuration >= slowReadTracingThreshold && logger.IsTracingEnabled(ctx) {
logger.Eventf(ctx, "reading %d bytes took %s",
len(buf), readDuration.String())
}
if err != nil {
return footer, errors.Wrap(err, "pebble/table: invalid table (could not read footer)")
}
Expand Down
4 changes: 2 additions & 2 deletions sstable/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -588,7 +588,7 @@ func TestFooterRoundTrip(t *testing.T) {
readable, err := NewSimpleReadable(f)
require.NoError(t, err)

result, err := readFooter(context.Background(), readable, nil)
result, err := readFooter(context.Background(), readable, nil, base.NoopLoggerAndTracer{})
require.NoError(t, err)
require.NoError(t, readable.Close())

Expand Down Expand Up @@ -641,7 +641,7 @@ func TestReadFooter(t *testing.T) {
readable, err := NewSimpleReadable(f)
require.NoError(t, err)

if _, err := readFooter(context.Background(), readable, nil); err == nil {
if _, err := readFooter(context.Background(), readable, nil, base.NoopLoggerAndTracer{}); err == nil {
t.Fatalf("expected %q, but found success", c.expected)
} else if !strings.Contains(err.Error(), c.expected) {
t.Fatalf("expected %q, but found %v", c.expected, err)
Expand Down

0 comments on commit 5bc59f8

Please sign in to comment.