-
Notifications
You must be signed in to change notification settings - Fork 24
/
entry_sorter.go
228 lines (211 loc) · 6.63 KB
/
entry_sorter.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
// Copyright 2020 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package memory
import (
"context"
"sort"
"sync"
"sync/atomic"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/tikv/migration/cdc/cdc/model"
cerror "github.com/tikv/migration/cdc/pkg/errors"
"github.com/tikv/migration/cdc/pkg/notify"
"github.com/tikv/migration/cdc/pkg/util"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
// EntrySorter accepts out-of-order raw kv entries and output sorted entries
type EntrySorter struct {
unsorted []*model.PolymorphicEvent
lock sync.Mutex
resolvedTsGroup []uint64
closed int32
outputCh chan *model.PolymorphicEvent
resolvedNotifier *notify.Notifier
}
// NewEntrySorter creates a new EntrySorter
func NewEntrySorter() *EntrySorter {
return &EntrySorter{
resolvedNotifier: new(notify.Notifier),
outputCh: make(chan *model.PolymorphicEvent, 128000),
}
}
// Run runs EntrySorter
func (es *EntrySorter) Run(ctx context.Context) error {
captureAddr := util.CaptureAddrFromCtx(ctx)
changefeedID := util.ChangefeedIDFromCtx(ctx)
_, keyspanName := util.KeySpanInfoFromCtx(ctx)
metricEntrySorterResolvedChanSizeGuage := entrySorterResolvedChanSizeGauge.WithLabelValues(captureAddr, changefeedID, keyspanName)
metricEntrySorterOutputChanSizeGauge := entrySorterOutputChanSizeGauge.WithLabelValues(captureAddr, changefeedID, keyspanName)
metricEntryUnsortedSizeGauge := entrySorterUnsortedSizeGauge.WithLabelValues(captureAddr, changefeedID, keyspanName)
metricEntrySorterSortDuration := entrySorterSortDuration.WithLabelValues(captureAddr, changefeedID, keyspanName)
metricEntrySorterMergeDuration := entrySorterMergeDuration.WithLabelValues(captureAddr, changefeedID, keyspanName)
output := func(ctx context.Context, entry *model.PolymorphicEvent) {
select {
case <-ctx.Done():
return
case es.outputCh <- entry:
}
}
errg, ctx := errgroup.WithContext(ctx)
receiver, err := es.resolvedNotifier.NewReceiver(1000 * time.Millisecond)
if err != nil {
return err
}
defer es.resolvedNotifier.Close()
errg.Go(func() error {
var sorted []*model.PolymorphicEvent
for {
select {
case <-ctx.Done():
atomic.StoreInt32(&es.closed, 1)
close(es.outputCh)
return errors.Trace(ctx.Err())
case <-time.After(defaultMetricInterval):
metricEntrySorterOutputChanSizeGauge.Set(float64(len(es.outputCh)))
es.lock.Lock()
metricEntrySorterResolvedChanSizeGuage.Set(float64(len(es.resolvedTsGroup)))
metricEntryUnsortedSizeGauge.Set(float64(len(es.unsorted)))
es.lock.Unlock()
case <-receiver.C:
es.lock.Lock()
if len(es.resolvedTsGroup) == 0 {
es.lock.Unlock()
continue
}
resolvedTsGroup := es.resolvedTsGroup
es.resolvedTsGroup = nil
toSort := es.unsorted
es.unsorted = nil
es.lock.Unlock()
resEvents := make([]*model.PolymorphicEvent, len(resolvedTsGroup))
for i, rts := range resolvedTsGroup {
// regionID = 0 means the event is produced by TiKV CDC
resEvents[i] = model.NewResolvedPolymorphicEvent(0, rts, 0)
}
toSort = append(toSort, resEvents...)
startTime := time.Now()
sort.SliceStable(toSort, func(i, j int) bool {
return eventLess(toSort[i], toSort[j])
})
metricEntrySorterSortDuration.Observe(time.Since(startTime).Seconds())
maxResolvedTs := resolvedTsGroup[len(resolvedTsGroup)-1]
startTime = time.Now()
var merged []*model.PolymorphicEvent
mergeEvents(toSort, sorted, func(entry *model.PolymorphicEvent) {
if entry.CRTs <= maxResolvedTs {
output(ctx, entry)
} else {
merged = append(merged, entry)
}
})
metricEntrySorterMergeDuration.Observe(time.Since(startTime).Seconds())
sorted = merged
}
}
})
return errg.Wait()
}
// AddEntry adds an RawKVEntry to the EntryGroup
func (es *EntrySorter) AddEntry(_ context.Context, entry *model.PolymorphicEvent) {
if atomic.LoadInt32(&es.closed) != 0 {
return
}
es.lock.Lock()
defer es.lock.Unlock()
if entry.RawKV.OpType == model.OpTypeResolved {
es.resolvedTsGroup = append(es.resolvedTsGroup, entry.CRTs)
es.resolvedNotifier.Notify()
} else {
es.unsorted = append(es.unsorted, entry)
}
}
func (es *EntrySorter) TryAddEntry(ctx context.Context, entry *model.PolymorphicEvent) (bool, error) {
if atomic.LoadInt32(&es.closed) != 0 {
return false, cerror.ErrSorterClosed.GenWithStackByArgs()
}
es.AddEntry(ctx, entry)
return true, nil
}
// Output returns the sorted raw kv output channel
func (es *EntrySorter) Output() <-chan *model.PolymorphicEvent {
return es.outputCh
}
func eventLess(i *model.PolymorphicEvent, j *model.PolymorphicEvent) bool {
return model.ComparePolymorphicEvents(i, j)
}
func mergeEvents(kvsA []*model.PolymorphicEvent, kvsB []*model.PolymorphicEvent, output func(*model.PolymorphicEvent)) {
var i, j int
for i < len(kvsA) && j < len(kvsB) {
if eventLess(kvsA[i], kvsB[j]) {
output(kvsA[i])
i++
} else {
output(kvsB[j])
j++
}
}
for ; i < len(kvsA); i++ {
output(kvsA[i])
}
for ; j < len(kvsB); j++ {
output(kvsB[j])
}
}
// SortOutput receives a channel from a puller, then sort event and output to the channel returned.
func SortOutput(ctx context.Context, input <-chan *model.RawKVEntry) <-chan *model.RawKVEntry {
ctx, cancel := context.WithCancel(ctx)
sorter := NewEntrySorter()
outputCh := make(chan *model.RawKVEntry, 128)
output := func(rawKV *model.RawKVEntry) {
select {
case <-ctx.Done():
if errors.Cause(ctx.Err()) != context.Canceled {
log.Error("sorter exited with error", zap.Error(ctx.Err()))
}
return
case outputCh <- rawKV:
}
}
go func() {
for {
select {
case <-ctx.Done():
if errors.Cause(ctx.Err()) != context.Canceled {
log.Error("sorter exited with error", zap.Error(ctx.Err()))
}
return
case rawKV := <-input:
if rawKV == nil {
continue
}
sorter.AddEntry(ctx, model.NewPolymorphicEvent(rawKV))
case sorted := <-sorter.Output():
if sorted != nil {
output(sorted.RawKV)
}
}
}
}()
go func() {
if err := sorter.Run(ctx); err != nil {
if errors.Cause(ctx.Err()) != context.Canceled {
log.Error("sorter exited with error", zap.Error(ctx.Err()))
}
}
cancel()
}()
return outputCh
}