github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/model/mounter.go (about) 1 // Copyright 2020 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package model 15 16 import ( 17 "context" 18 "math" 19 20 "github.com/pingcap/log" 21 "github.com/pingcap/tidb/pkg/types" 22 "go.uber.org/zap" 23 ) 24 25 // RowChangedDatums is used to store the changed datums of a row. 26 type RowChangedDatums struct { 27 RowDatums []types.Datum 28 PreRowDatums []types.Datum 29 } 30 31 // IsEmpty returns true if the RowChangeDatums is empty. 32 func (r RowChangedDatums) IsEmpty() bool { 33 return len(r.RowDatums) == 0 && len(r.PreRowDatums) == 0 34 } 35 36 // PolymorphicEvent describes an event can be in multiple states. 37 type PolymorphicEvent struct { 38 StartTs uint64 39 CRTs uint64 40 Resolved *ResolvedTs 41 42 RawKV *RawKVEntry 43 Row *RowChangedEvent 44 45 finished chan struct{} 46 } 47 48 // NewEmptyPolymorphicEvent creates a new empty PolymorphicEvent. 49 func NewEmptyPolymorphicEvent(ts uint64) *PolymorphicEvent { 50 return &PolymorphicEvent{ 51 CRTs: ts, 52 RawKV: &RawKVEntry{}, 53 Row: &RowChangedEvent{}, 54 } 55 } 56 57 // NewPolymorphicEvent creates a new PolymorphicEvent with a raw KV. 58 func NewPolymorphicEvent(rawKV *RawKVEntry) *PolymorphicEvent { 59 if rawKV.OpType == OpTypeResolved { 60 return NewResolvedPolymorphicEvent(rawKV.RegionID, rawKV.CRTs) 61 } 62 return &PolymorphicEvent{ 63 StartTs: rawKV.StartTs, 64 CRTs: rawKV.CRTs, 65 RawKV: rawKV, 66 } 67 } 68 69 // NewResolvedPolymorphicEvent creates a new PolymorphicEvent with the resolved ts. 70 func NewResolvedPolymorphicEvent(regionID uint64, resolvedTs uint64) *PolymorphicEvent { 71 return &PolymorphicEvent{ 72 CRTs: resolvedTs, 73 RawKV: &RawKVEntry{CRTs: resolvedTs, OpType: OpTypeResolved, RegionID: regionID}, 74 Row: nil, 75 } 76 } 77 78 // RegionID returns the region ID where the event comes from. 79 func (e *PolymorphicEvent) RegionID() uint64 { 80 return e.RawKV.RegionID 81 } 82 83 // IsResolved returns true if the event is resolved. Note that this function can 84 // only be called when `RawKV != nil`. 85 func (e *PolymorphicEvent) IsResolved() bool { 86 return e.RawKV.OpType == OpTypeResolved 87 } 88 89 // SetUpFinishedCh set up the finished chan, should be called before mounting the event. 90 func (e *PolymorphicEvent) SetUpFinishedCh() { 91 if e.finished == nil { 92 e.finished = make(chan struct{}) 93 } 94 } 95 96 // MarkFinished is called to indicate that mount is finished. 97 func (e *PolymorphicEvent) MarkFinished() { 98 if e.finished != nil { 99 close(e.finished) 100 } 101 } 102 103 // WaitFinished is called by caller to wait for the mount finished. 104 func (e *PolymorphicEvent) WaitFinished(ctx context.Context) error { 105 if e.finished != nil { 106 select { 107 case <-ctx.Done(): 108 return ctx.Err() 109 case <-e.finished: 110 } 111 } 112 return nil 113 } 114 115 // ComparePolymorphicEvents compares two events by CRTs, Resolved, StartTs, Delete/Put order. 116 // It returns true if and only if i should precede j. 117 func ComparePolymorphicEvents(i, j *PolymorphicEvent) bool { 118 if i.CRTs == j.CRTs { 119 if i.IsResolved() { 120 return false 121 } else if j.IsResolved() { 122 return true 123 } 124 125 if i.StartTs > j.StartTs { 126 return false 127 } else if i.StartTs < j.StartTs { 128 return true 129 } 130 131 if i.RawKV.OpType == OpTypeDelete && j.RawKV.OpType != OpTypeDelete { 132 return true 133 } 134 // update DML 135 if i.RawKV.OldValue != nil && j.RawKV.OldValue == nil { 136 return true 137 } 138 } 139 return i.CRTs < j.CRTs 140 } 141 142 // ResolvedMode describes the batch type of a resolved event. 143 type ResolvedMode int 144 145 const ( 146 // NormalResolvedMode means that all events whose commitTs is less than or equal to 147 // `resolved.Ts` are sent to Sink. 148 NormalResolvedMode ResolvedMode = iota 149 // BatchResolvedMode means that all events whose commitTs is less than 150 // 'resolved.Ts' are sent to Sink. 151 BatchResolvedMode 152 ) 153 154 // ResolvedTs is the resolved timestamp of sink module. 155 type ResolvedTs struct { 156 Mode ResolvedMode 157 Ts uint64 158 BatchID uint64 159 } 160 161 // NewResolvedTs creates a normal ResolvedTs. 162 func NewResolvedTs(t uint64) ResolvedTs { 163 return ResolvedTs{Ts: t, Mode: NormalResolvedMode, BatchID: math.MaxUint64} 164 } 165 166 // IsBatchMode returns true if the resolved ts is BatchResolvedMode. 167 func (r ResolvedTs) IsBatchMode() bool { 168 return r.Mode == BatchResolvedMode 169 } 170 171 // AdvanceBatch advances the batch id of the resolved ts. 172 func (r ResolvedTs) AdvanceBatch() ResolvedTs { 173 if !r.IsBatchMode() { 174 log.Panic("can't advance batch since resolved ts is not in batch mode", 175 zap.Any("resolved", r)) 176 } 177 return ResolvedTs{ 178 Mode: BatchResolvedMode, 179 Ts: r.Ts, 180 BatchID: r.BatchID + 1, 181 } 182 } 183 184 // ResolvedMark returns a timestamp `ts` based on the r.mode, which marks that all events 185 // whose commitTs is less than or equal to `ts` are sent to Sink. 186 func (r ResolvedTs) ResolvedMark() uint64 { 187 switch r.Mode { 188 case NormalResolvedMode: 189 // with NormalResolvedMode, cdc guarantees all events whose commitTs is 190 // less than or equal to `resolved.Ts` are sent to Sink. 191 return r.Ts 192 case BatchResolvedMode: 193 // with BatchResolvedMode, cdc guarantees all events whose commitTs is 194 // less than `resolved.Ts` are sent to Sink. 195 return r.Ts - 1 196 default: 197 log.Error("unknown resolved mode", zap.Any("resolved", r)) 198 return 0 199 } 200 } 201 202 // EqualOrGreater judge whether the resolved ts is equal or greater than the given ts. 203 func (r ResolvedTs) EqualOrGreater(r1 ResolvedTs) bool { 204 if r.Ts == r1.Ts { 205 return r.BatchID >= r1.BatchID 206 } 207 return r.Ts > r1.Ts 208 } 209 210 // Less judge whether the resolved ts is less than the given ts. 211 func (r ResolvedTs) Less(r1 ResolvedTs) bool { 212 return !r.EqualOrGreater(r1) 213 } 214 215 // Greater judge whether the resolved ts is greater than the given ts. 216 func (r ResolvedTs) Greater(r1 ResolvedTs) bool { 217 if r.Ts == r1.Ts { 218 return r.BatchID > r1.BatchID 219 } 220 return r.Ts > r1.Ts 221 } 222 223 // Equal judge whether the resolved ts is equal to the given ts. 224 func (r ResolvedTs) Equal(r1 ResolvedTs) bool { 225 return r == r1 226 }