-
Notifications
You must be signed in to change notification settings - Fork 289
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
sorter: Stabilize Unified Sorter #1210
Changes from 41 commits
c588f9b
b6dd64e
3b0f216
1361920
2431bff
a42ffd5
a0757c5
20b8763
0a3044a
fd03c77
d5f2c1c
20f7896
11e13a6
c643e9c
98f2abf
31ab7ca
3868d41
3942738
d252c2f
1c2dc5f
113816b
a875a7c
f7f27b1
c36fef8
dd2b7e4
8e00de3
b5513d4
9523591
b845ee2
94b6cc1
c8f5d4a
09d1253
d5ea5a2
35cf992
dddf707
83f8d51
18d5f01
a53bbb5
938e15f
78d6c7e
049e69f
31c046c
6679ff6
d8448f1
ba910b8
e1412e3
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,14 +16,17 @@ package sorter | |
import ( | ||
"container/heap" | ||
"context" | ||
"sync" | ||
"sync/atomic" | ||
"time" | ||
|
||
"github.com/pingcap/errors" | ||
"github.com/pingcap/failpoint" | ||
"github.com/pingcap/log" | ||
"github.com/pingcap/ticdc/cdc/model" | ||
"github.com/pingcap/ticdc/pkg/config" | ||
"github.com/pingcap/ticdc/pkg/util" | ||
"github.com/pingcap/ticdc/pkg/workerpool" | ||
"go.uber.org/zap" | ||
) | ||
|
||
|
@@ -50,6 +53,9 @@ type heapSorter struct { | |
inputCh chan *model.PolymorphicEvent | ||
outputCh chan *flushTask | ||
heap sortHeap | ||
|
||
poolHandle workerpool.EventHandle | ||
internalState *heapSorterInternalState | ||
} | ||
|
||
func newHeapSorter(id int, out chan *flushTask) *heapSorter { | ||
|
@@ -61,30 +67,43 @@ func newHeapSorter(id int, out chan *flushTask) *heapSorter { | |
} | ||
} | ||
|
||
// flush should only be called within the main loop in run(). | ||
// flush should only be called in the same goroutine where the heap is being written to. | ||
func (h *heapSorter) flush(ctx context.Context, maxResolvedTs uint64) error { | ||
captureAddr := util.CaptureAddrFromCtx(ctx) | ||
changefeedID := util.ChangefeedIDFromCtx(ctx) | ||
_, tableName := util.TableIDFromCtx(ctx) | ||
sorterFlushCountHistogram.WithLabelValues(captureAddr, changefeedID, tableName).Observe(float64(h.heap.Len())) | ||
|
||
isEmptyFlush := h.heap.Len() == 0 | ||
if isEmptyFlush { | ||
return nil | ||
} | ||
var ( | ||
backEnd backEnd | ||
lowerBound uint64 | ||
) | ||
|
||
if h.heap.Len() > 0 { | ||
lowerBound = h.heap[0].entry.CRTs | ||
} else { | ||
return nil | ||
} | ||
|
||
// We check if the heap contains only one entry and that entry is a ResolvedEvent. | ||
// As an optimization, when the condition is true, we clear the heap and send an empty flush. | ||
// Sending an empty flush saves CPU and potentially IO. | ||
// Since when a table is mostly idle or near-idle, most flushes would contain one ResolvedEvent alone, | ||
// this optimization will greatly improve performance when (1) total number of table is large, | ||
// and (2) most tables do not have many events. | ||
if h.heap.Len() == 1 && h.heap[0].entry.RawKV.OpType == model.OpTypeResolved { | ||
liuzix marked this conversation as resolved.
Show resolved
Hide resolved
|
||
h.heap.Pop() | ||
} | ||
|
||
isEmptyFlush := h.heap.Len() == 0 | ||
var finishCh chan error | ||
if !isEmptyFlush { | ||
var err error | ||
backEnd, err = pool.alloc(ctx) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
|
||
lowerBound = h.heap[0].entry.CRTs | ||
finishCh = make(chan error, 1) | ||
} | ||
|
||
task := &flushTask{ | ||
|
@@ -93,7 +112,7 @@ func (h *heapSorter) flush(ctx context.Context, maxResolvedTs uint64) error { | |
backend: backEnd, | ||
tsLowerBound: lowerBound, | ||
maxResolvedTs: maxResolvedTs, | ||
finished: make(chan error, 2), | ||
finished: finishCh, | ||
} | ||
h.taskCounter++ | ||
|
||
|
@@ -113,73 +132,75 @@ func (h *heapSorter) flush(ctx context.Context, maxResolvedTs uint64) error { | |
return nil | ||
} | ||
} | ||
failpoint.Inject("sorterDebug", func() { | ||
log.Debug("Unified Sorter new flushTask", | ||
zap.String("table", tableNameFromCtx(ctx)), | ||
zap.Int("heap-id", task.heapSorterID), | ||
zap.Uint64("resolvedTs", task.maxResolvedTs)) | ||
}) | ||
|
||
log.Debug("Unified Sorter new flushTask", | ||
zap.String("table", tableNameFromCtx(ctx)), | ||
zap.Int("heap-id", task.heapSorterID), | ||
zap.Uint64("resolvedTs", task.maxResolvedTs)) | ||
|
||
go func() { | ||
if isEmptyFlush { | ||
return | ||
} | ||
if !isEmptyFlush { | ||
backEndFinal := backEnd | ||
writer, err := backEnd.writer() | ||
if err != nil { | ||
if backEndFinal != nil { | ||
_ = task.dealloc() | ||
err := heapSorterIOPool.Go(ctx, func() { | ||
writer, err := backEnd.writer() | ||
if err != nil { | ||
if backEndFinal != nil { | ||
_ = task.dealloc() | ||
} | ||
task.finished <- errors.Trace(err) | ||
return | ||
} | ||
task.finished <- errors.Trace(err) | ||
return | ||
} | ||
|
||
defer func() { | ||
// handle errors (or aborts) gracefully to prevent resource leaking (especially FD's) | ||
if writer != nil { | ||
_ = writer.flushAndClose() | ||
} | ||
if backEndFinal != nil { | ||
_ = task.dealloc() | ||
} | ||
close(task.finished) | ||
}() | ||
defer func() { | ||
// handle errors (or aborts) gracefully to prevent resource leaking (especially FD's) | ||
if writer != nil { | ||
_ = writer.flushAndClose() | ||
} | ||
if backEndFinal != nil { | ||
_ = task.dealloc() | ||
} | ||
close(task.finished) | ||
}() | ||
|
||
for oldHeap.Len() > 0 { | ||
select { | ||
case <-ctx.Done(): | ||
task.finished <- ctx.Err() | ||
default: | ||
for oldHeap.Len() > 0 { | ||
event := heap.Pop(&oldHeap).(*sortItem).entry | ||
err := writer.writeNext(event) | ||
if err != nil { | ||
task.finished <- errors.Trace(err) | ||
return | ||
} | ||
} | ||
|
||
event := heap.Pop(&oldHeap).(*sortItem).entry | ||
err := writer.writeNext(event) | ||
dataSize := writer.dataSize() | ||
atomic.StoreInt64(&task.dataSize, int64(dataSize)) | ||
eventCount := writer.writtenCount() | ||
|
||
writer1 := writer | ||
writer = nil | ||
err = writer1.flushAndClose() | ||
if err != nil { | ||
task.finished <- errors.Trace(err) | ||
return | ||
} | ||
} | ||
|
||
dataSize := writer.dataSize() | ||
atomic.StoreInt64(&task.dataSize, int64(dataSize)) | ||
eventCount := writer.writtenCount() | ||
backEndFinal = nil | ||
|
||
writer1 := writer | ||
writer = nil | ||
err = writer1.flushAndClose() | ||
failpoint.Inject("sorterDebug", func() { | ||
log.Debug("Unified Sorter flushTask finished", | ||
zap.Int("heap-id", task.heapSorterID), | ||
zap.String("table", tableNameFromCtx(ctx)), | ||
zap.Uint64("resolvedTs", task.maxResolvedTs), | ||
zap.Uint64("data-size", dataSize), | ||
zap.Int("size", eventCount)) | ||
}) | ||
|
||
task.finished <- nil // DO NOT access `task` beyond this point in this function | ||
}) | ||
if err != nil { | ||
task.finished <- errors.Trace(err) | ||
return | ||
close(task.finished) | ||
return errors.Trace(err) | ||
} | ||
|
||
backEndFinal = nil | ||
task.finished <- nil // DO NOT access `task` beyond this point in this function | ||
log.Debug("Unified Sorter flushTask finished", | ||
zap.Int("heap-id", task.heapSorterID), | ||
zap.String("table", tableNameFromCtx(ctx)), | ||
zap.Uint64("resolvedTs", task.maxResolvedTs), | ||
zap.Uint64("data-size", dataSize), | ||
zap.Int("size", eventCount)) | ||
}() | ||
} | ||
|
||
select { | ||
case <-ctx.Done(): | ||
|
@@ -189,63 +210,78 @@ func (h *heapSorter) flush(ctx context.Context, maxResolvedTs uint64) error { | |
return nil | ||
} | ||
|
||
func (h *heapSorter) run(ctx context.Context) error { | ||
var ( | ||
maxResolved uint64 | ||
heapSizeBytesEstimate int64 | ||
rateCounter int | ||
) | ||
var ( | ||
heapSorterPool workerpool.WorkerPool | ||
heapSorterIOPool workerpool.AsyncPool | ||
poolOnce sync.Once | ||
) | ||
|
||
rateTicker := time.NewTicker(1 * time.Second) | ||
defer rateTicker.Stop() | ||
|
||
flushTicker := time.NewTicker(5 * time.Second) | ||
defer flushTicker.Stop() | ||
|
||
sorterConfig := config.GetSorterConfig() | ||
for { | ||
select { | ||
case <-ctx.Done(): | ||
return ctx.Err() | ||
case event := <-h.inputCh: | ||
heap.Push(&h.heap, &sortItem{entry: event}) | ||
isResolvedEvent := event.RawKV != nil && event.RawKV.OpType == model.OpTypeResolved | ||
|
||
if isResolvedEvent { | ||
if event.RawKV.CRTs < maxResolved { | ||
log.Panic("ResolvedTs regression, bug?", zap.Uint64("event-resolvedTs", event.RawKV.CRTs), | ||
zap.Uint64("max-resolvedTs", maxResolved)) | ||
} | ||
maxResolved = event.RawKV.CRTs | ||
} | ||
type heapSorterInternalState struct { | ||
maxResolved uint64 | ||
heapSizeBytesEstimate int64 | ||
rateCounter int | ||
sorterConfig *config.SorterConfig | ||
timerMultiplier int | ||
} | ||
|
||
if event.RawKV.CRTs < maxResolved { | ||
log.Panic("Bad input to sorter", zap.Uint64("cur-ts", event.RawKV.CRTs), zap.Uint64("maxResolved", maxResolved)) | ||
func (h *heapSorter) init(ctx context.Context, onError func(err error)) { | ||
state := &heapSorterInternalState{ | ||
sorterConfig: config.GetSorterConfig(), | ||
} | ||
|
||
poolHandle := heapSorterPool.RegisterEvent(func(ctx context.Context, eventI interface{}) error { | ||
event := eventI.(*model.PolymorphicEvent) | ||
heap.Push(&h.heap, &sortItem{entry: event}) | ||
isResolvedEvent := event.RawKV != nil && event.RawKV.OpType == model.OpTypeResolved | ||
|
||
if isResolvedEvent { | ||
if event.RawKV.CRTs < state.maxResolved { | ||
log.Panic("ResolvedTs regression, bug?", zap.Uint64("event-resolvedTs", event.RawKV.CRTs), | ||
zap.Uint64("max-resolvedTs", state.maxResolved)) | ||
} | ||
state.maxResolved = event.RawKV.CRTs | ||
} | ||
|
||
// 5 * 8 is for the 5 fields in PolymorphicEvent | ||
heapSizeBytesEstimate += event.RawKV.ApproximateSize() + 40 | ||
needFlush := heapSizeBytesEstimate >= int64(sorterConfig.ChunkSizeLimit) || | ||
(isResolvedEvent && rateCounter < flushRateLimitPerSecond) | ||
if event.RawKV.CRTs < state.maxResolved { | ||
log.Panic("Bad input to sorter", zap.Uint64("cur-ts", event.RawKV.CRTs), zap.Uint64("maxResolved", state.maxResolved)) | ||
} | ||
|
||
if needFlush { | ||
rateCounter++ | ||
err := h.flush(ctx, maxResolved) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
heapSizeBytesEstimate = 0 | ||
// 5 * 8 is for the 5 fields in PolymorphicEvent | ||
state.heapSizeBytesEstimate += event.RawKV.ApproximateSize() + 40 | ||
needFlush := state.heapSizeBytesEstimate >= int64(state.sorterConfig.ChunkSizeLimit) || | ||
(isResolvedEvent && state.rateCounter < flushRateLimitPerSecond) | ||
|
||
if needFlush { | ||
state.rateCounter++ | ||
err := h.flush(ctx, state.maxResolved) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
case <-flushTicker.C: | ||
if rateCounter < flushRateLimitPerSecond { | ||
err := h.flush(ctx, maxResolved) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
heapSizeBytesEstimate = 0 | ||
state.heapSizeBytesEstimate = 0 | ||
} | ||
|
||
return nil | ||
}).SetTimer(ctx, 1*time.Second, func(ctx context.Context) error { | ||
state.rateCounter = 0 | ||
state.timerMultiplier = (state.timerMultiplier + 1) % 5 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why adding the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I wanted to trigger a flush periodically even if there is no resolved event passed down from TiKV, mainly for breaking up incremental scans. But the 1 second period used for rate limiting was too short, which lead to performance degradation when the incremental scan is huge, so I chose the flush period to be 5 seconds. |
||
if state.timerMultiplier == 0 && state.rateCounter < flushRateLimitPerSecond { | ||
err := h.flush(ctx, state.maxResolved) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
case <-rateTicker.C: | ||
rateCounter = 0 | ||
state.heapSizeBytesEstimate = 0 | ||
} | ||
} | ||
return nil | ||
}).OnExit(onError) | ||
|
||
h.poolHandle = poolHandle | ||
h.internalState = state | ||
} | ||
|
||
func lazyInitPool() { | ||
poolOnce.Do(func() { | ||
sorterConfig := config.GetSorterConfig() | ||
heapSorterPool = workerpool.NewDefaultWorkerPool(sorterConfig.NumWorkerPoolGoroutine) | ||
heapSorterIOPool = workerpool.NewDefaultAsyncPool(sorterConfig.NumWorkerPoolGoroutine * 2) | ||
}) | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will this default value be picked to 4.0?