Skip to content

Commit

Permalink
changefeedccl: Do not block on file size based flushes
Browse files Browse the repository at this point in the history
Prior to this change, cloud storage sink trigger
file sized based flush whenever new row would
would push the file size beyond configured threshold.

This had the effect of singificantly reducing the throughput
whenever such event occured -- no additional events could
be added to cloud storage sink, while the previus flush was
active.

This is not necessary.  Cloud storage sink can trigger
file based flushes asynchronously.  The only requirement
is that if a real, non file based, flush arrives, or if we
need to emit resolved timestamps, then we must wait for
all of the active flush requests to complete.

In addition, because every event added to cloud sink has
associate allocation, which is released when file is written
out, performing flushes asynchronously is safe with respect
to memory usage and accounting.

Release note (enterprise change): Changefeeds, using cloud
storage sink, now have better throughput.
Release justification: performance fix
  • Loading branch information
Yevgeniy Miretskiy committed Sep 22, 2022
1 parent bcd9d69 commit 81f9932
Show file tree
Hide file tree
Showing 2 changed files with 198 additions and 106 deletions.
105 changes: 89 additions & 16 deletions pkg/ccl/changefeedccl/sink_cloudstorage.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"net/url"
"path/filepath"
"strings"
"sync"
"sync/atomic"
"time"

Expand All @@ -27,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand Down Expand Up @@ -304,6 +306,11 @@ type cloudStorageSink struct {
dataFilePartition string
prevFilename string
metrics metricsRecorder

asyncFlushActive bool
flushCtx context.Context
flushGroup sync.WaitGroup
flushErr atomic.Value
}

const sinkCompressionGzip = "gzip"
Expand Down Expand Up @@ -364,8 +371,11 @@ func makeCloudStorageSink(
partitionFormat: defaultPartitionFormat,
timestampOracle: timestampOracle,
// TODO(dan,ajwerner): Use the jobs framework's session ID once that's available.
jobSessionID: sessID,
topicNamer: tn,
jobSessionID: sessID,
topicNamer: tn,
asyncFlushActive: enableAsyncFlush.Get(&settings.SV),
// TODO (yevgeniy): Consider adding ctx to Dial method instead.
flushCtx: ctx,
}

if partitionFormat := u.consumeParam(changefeedbase.SinkParamPartitionFormat); partitionFormat != "" {
Expand Down Expand Up @@ -503,6 +513,13 @@ func (s *cloudStorageSink) EmitResolvedTimestamp(
if err != nil {
return err
}

// Wait for previously issued async flush requests to complete
// before we write resolved time stamp file.
if err := s.waitAsyncFlush(); err != nil {
return errors.Wrapf(err, "while emitting resolved timestamp")
}

// Don't need to copy payload because we never buffer it anywhere.

part := resolved.GoTime().Format(s.partitionFormat)
Expand Down Expand Up @@ -572,29 +589,44 @@ func (s *cloudStorageSink) Flush(ctx context.Context) error {
// for an overview of the naming convention and proof of correctness.
s.dataFileTs = cloudStorageFormatTime(s.timestampOracle.inclusiveLowerBoundTS())
s.dataFilePartition = s.timestampOracle.inclusiveLowerBoundTS().GoTime().Format(s.partitionFormat)
return nil
return s.waitAsyncFlush()
}

// file should not be used after flushing.
func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSinkFile) error {
defer file.alloc.Release(ctx)
// enableAsyncFlush controls async flushing behavior for this sink.
var enableAsyncFlush = settings.RegisterBoolSetting(
settings.TenantWritable,
"changefeed.cloudstorage.async_flush.enabled",
"enable async flushing",
true,
)

if file.rawSize == 0 {
// This method shouldn't be called with an empty file, but be defensive
// about not writing empty files anyway.
return nil
// waitAsyncFlush waits until all async flushes complete.
func (s *cloudStorageSink) waitAsyncFlush() error {
s.flushGroup.Wait()
if v := s.flushErr.Load(); v != nil {
return v.(error)
}
return nil
}

if file.codec != nil {
if err := file.codec.Close(); err != nil {
// flushFile flushes file to the cloud storage.
// file should not be used after flushing.
func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSinkFile) error {
asyncFlushEnabled := enableAsyncFlush.Get(&s.settings.SV)
if s.asyncFlushActive && !asyncFlushEnabled {
// Async flush behavior was turned off -- drain any active flush requests
// before flushing this file.
if err := s.waitAsyncFlush(); err != nil {
return err
}
}
s.asyncFlushActive = asyncFlushEnabled

// We use this monotonically increasing fileID to ensure correct ordering
// among files emitted at the same timestamp during the same job session.
fileID := s.fileID
s.fileID++

// Pad file ID to maintain lexical ordering among files from the same sink.
// Note that we use `-` here to delimit the filename because we want
// `%d.RESOLVED` files to lexicographically succeed data files that have the
Expand All @@ -606,19 +638,60 @@ func (s *cloudStorageSink) flushFile(ctx context.Context, file *cloudStorageSink
"precedes a file emitted before: %s", filename, s.prevFilename)
}
s.prevFilename = filename
compressedBytes := file.buf.Len()
if err := cloud.WriteFile(ctx, s.es, filepath.Join(s.dataFilePartition, filename), bytes.NewReader(file.buf.Bytes())); err != nil {
dest := filepath.Join(s.dataFilePartition, filename)

if !asyncFlushEnabled {
return file.flushToStorage(ctx, s.es, dest, s.metrics)
}

s.flushGroup.Add(1)
go func() {
defer s.flushGroup.Done()
// NB: must use s.flushCtx; ctx may be short lived (i.e. cancelled).
if err := file.flushToStorage(s.flushCtx, s.es, dest, s.metrics); err != nil {
log.Errorf(ctx, "error flushing file to storage: %s", err)
// We must use the same type for error we store in flushErr.
s.flushErr.CompareAndSwap(nil, &flushError{error: err})
}
}()
return nil
}

type flushError struct {
error
}

// flushToStorage writes out file into external storage into 'dest'.
func (f *cloudStorageSinkFile) flushToStorage(
ctx context.Context, es cloud.ExternalStorage, dest string, m metricsRecorder,
) error {
defer f.alloc.Release(ctx)

if f.rawSize == 0 {
// This method shouldn't be called with an empty file, but be defensive
// about not writing empty files anyway.
return nil
}

if f.codec != nil {
if err := f.codec.Close(); err != nil {
return err
}
}

compressedBytes := f.buf.Len()
if err := cloud.WriteFile(ctx, es, dest, bytes.NewReader(f.buf.Bytes())); err != nil {
return err
}
s.metrics.recordEmittedBatch(file.created, file.numMessages, file.oldestMVCC, file.rawSize, compressedBytes)
m.recordEmittedBatch(f.created, f.numMessages, f.oldestMVCC, f.rawSize, compressedBytes)

return nil
}

// Close implements the Sink interface.
func (s *cloudStorageSink) Close() error {
s.files = nil
return s.es.Close()
return errors.CombineErrors(s.waitAsyncFlush(), s.es.Close())
}

// Dial implements the Sink interface.
Expand Down
Loading

0 comments on commit 81f9932

Please sign in to comment.