github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/processor/pipeline/sink.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 pipeline 15 16 import ( 17 "sync/atomic" 18 "time" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/failpoint" 22 "github.com/pingcap/log" 23 "github.com/pingcap/ticdc/cdc/model" 24 "github.com/pingcap/ticdc/cdc/sink" 25 cerror "github.com/pingcap/ticdc/pkg/errors" 26 "github.com/pingcap/ticdc/pkg/pipeline" 27 ) 28 29 const ( 30 defaultSyncResolvedBatch = 64 31 ) 32 33 // TableStatus is status of the table pipeline 34 type TableStatus int32 35 36 // TableStatus for table pipeline 37 const ( 38 TableStatusInitializing TableStatus = iota 39 TableStatusRunning 40 TableStatusStopped 41 ) 42 43 func (s TableStatus) String() string { 44 switch s { 45 case TableStatusInitializing: 46 return "Initializing" 47 case TableStatusRunning: 48 return "Running" 49 case TableStatusStopped: 50 return "Stopped" 51 } 52 return "Unknown" 53 } 54 55 // Load TableStatus with THREAD-SAFE 56 func (s *TableStatus) Load() TableStatus { 57 return TableStatus(atomic.LoadInt32((*int32)(s))) 58 } 59 60 // Store TableStatus with THREAD-SAFE 61 func (s *TableStatus) Store(new TableStatus) { 62 atomic.StoreInt32((*int32)(s), int32(new)) 63 } 64 65 type sinkNode struct { 66 sink sink.Sink 67 status TableStatus 68 69 resolvedTs model.Ts 70 checkpointTs model.Ts 71 targetTs model.Ts 72 barrierTs model.Ts 73 74 eventBuffer []*model.PolymorphicEvent 75 rowBuffer []*model.RowChangedEvent 76 77 flowController tableFlowController 78 } 79 80 func newSinkNode(sink sink.Sink, startTs model.Ts, targetTs model.Ts, flowController tableFlowController) *sinkNode { 81 return &sinkNode{ 82 sink: sink, 83 status: TableStatusInitializing, 84 targetTs: targetTs, 85 resolvedTs: startTs, 86 checkpointTs: startTs, 87 barrierTs: startTs, 88 89 flowController: flowController, 90 } 91 } 92 93 func (n *sinkNode) ResolvedTs() model.Ts { return atomic.LoadUint64(&n.resolvedTs) } 94 func (n *sinkNode) CheckpointTs() model.Ts { return atomic.LoadUint64(&n.checkpointTs) } 95 func (n *sinkNode) Status() TableStatus { return n.status.Load() } 96 97 func (n *sinkNode) Init(ctx pipeline.NodeContext) error { 98 // do nothing 99 return nil 100 } 101 102 // stop is called when sink receives a stop command or checkpointTs reaches targetTs. 103 // In this method, the builtin table sink will be closed by calling `Close`, and 104 // no more events can be sent to this sink node afterwards. 105 func (n *sinkNode) stop(ctx pipeline.NodeContext) (err error) { 106 // table stopped status must be set after underlying sink is closed 107 defer n.status.Store(TableStatusStopped) 108 err = n.sink.Close(ctx) 109 if err != nil { 110 return 111 } 112 err = cerror.ErrTableProcessorStoppedSafely.GenWithStackByArgs() 113 return 114 } 115 116 func (n *sinkNode) flushSink(ctx pipeline.NodeContext, resolvedTs model.Ts) (err error) { 117 defer func() { 118 if err != nil { 119 n.status.Store(TableStatusStopped) 120 return 121 } 122 if n.checkpointTs >= n.targetTs { 123 err = n.stop(ctx) 124 } 125 }() 126 if resolvedTs > n.barrierTs { 127 resolvedTs = n.barrierTs 128 } 129 if resolvedTs > n.targetTs { 130 resolvedTs = n.targetTs 131 } 132 if resolvedTs <= n.checkpointTs { 133 return nil 134 } 135 if err := n.emitRow2Sink(ctx); err != nil { 136 return errors.Trace(err) 137 } 138 checkpointTs, err := n.sink.FlushRowChangedEvents(ctx, resolvedTs) 139 if err != nil { 140 return errors.Trace(err) 141 } 142 if checkpointTs <= n.checkpointTs { 143 return nil 144 } 145 atomic.StoreUint64(&n.checkpointTs, checkpointTs) 146 147 n.flowController.Release(checkpointTs) 148 return nil 149 } 150 151 func (n *sinkNode) emitEvent(ctx pipeline.NodeContext, event *model.PolymorphicEvent) error { 152 if event == nil || event.Row == nil { 153 return nil 154 } 155 156 colLen := len(event.Row.Columns) 157 preColLen := len(event.Row.PreColumns) 158 config := ctx.ChangefeedVars().Info.Config 159 160 // This indicates that it is an update event, 161 // and after enable old value internally by default(but disable in the configuration). 162 // We need to handle the update event to be compatible with the old format. 163 if !config.EnableOldValue && colLen != 0 && preColLen != 0 && colLen == preColLen { 164 if shouldSplitUpdateEvent(event) { 165 deleteEvent, insertEvent, err := splitUpdateEvent(event) 166 if err != nil { 167 return errors.Trace(err) 168 } 169 // NOTICE: Please do not change the order, the delete event always comes before the insert event. 170 n.eventBuffer = append(n.eventBuffer, deleteEvent, insertEvent) 171 } else { 172 // If the handle key columns are not updated, PreColumns is directly ignored. 173 event.Row.PreColumns = nil 174 n.eventBuffer = append(n.eventBuffer, event) 175 } 176 } else { 177 n.eventBuffer = append(n.eventBuffer, event) 178 } 179 180 if len(n.eventBuffer) >= defaultSyncResolvedBatch { 181 if err := n.emitRow2Sink(ctx); err != nil { 182 return errors.Trace(err) 183 } 184 } 185 return nil 186 } 187 188 // shouldSplitUpdateEvent determines if the split event is needed to align the old format based on 189 // whether the handle key column has been modified. 190 // If the handle key column is modified, 191 // we need to use splitUpdateEvent to split the update event into a delete and an insert event. 192 func shouldSplitUpdateEvent(updateEvent *model.PolymorphicEvent) bool { 193 // nil event will never be split. 194 if updateEvent == nil { 195 return false 196 } 197 198 handleKeyCount := 0 199 equivalentHandleKeyCount := 0 200 for i := range updateEvent.Row.Columns { 201 if updateEvent.Row.Columns[i].Flag.IsHandleKey() && updateEvent.Row.PreColumns[i].Flag.IsHandleKey() { 202 handleKeyCount++ 203 colValueString := model.ColumnValueString(updateEvent.Row.Columns[i].Value) 204 preColValueString := model.ColumnValueString(updateEvent.Row.PreColumns[i].Value) 205 if colValueString == preColValueString { 206 equivalentHandleKeyCount++ 207 } 208 } 209 } 210 211 // If the handle key columns are not updated, so we do **not** need to split the event row. 212 return !(handleKeyCount == equivalentHandleKeyCount) 213 } 214 215 // splitUpdateEvent splits an update event into a delete and an insert event. 216 func splitUpdateEvent(updateEvent *model.PolymorphicEvent) (*model.PolymorphicEvent, *model.PolymorphicEvent, error) { 217 if updateEvent == nil { 218 return nil, nil, errors.New("nil event cannot be split") 219 } 220 221 // If there is an update to handle key columns, 222 // we need to split the event into two events to be compatible with the old format. 223 // NOTICE: Here we don't need a full deep copy because our two events need Columns and PreColumns respectively, 224 // so it won't have an impact and no more full deep copy wastes memory. 225 deleteEvent := *updateEvent 226 deleteEventRow := *updateEvent.Row 227 deleteEventRowKV := *updateEvent.RawKV 228 deleteEvent.Row = &deleteEventRow 229 deleteEvent.RawKV = &deleteEventRowKV 230 231 deleteEvent.Row.Columns = nil 232 for i := range deleteEvent.Row.PreColumns { 233 // NOTICE: Only the handle key pre column is retained in the delete event. 234 if !deleteEvent.Row.PreColumns[i].Flag.IsHandleKey() { 235 deleteEvent.Row.PreColumns[i] = nil 236 } 237 } 238 // Align with the old format if old value disabled. 239 deleteEvent.Row.TableInfoVersion = 0 240 241 insertEvent := *updateEvent 242 insertEventRow := *updateEvent.Row 243 insertEventRowKV := *updateEvent.RawKV 244 insertEvent.Row = &insertEventRow 245 insertEvent.RawKV = &insertEventRowKV 246 // NOTICE: clean up pre cols for insert event. 247 insertEvent.Row.PreColumns = nil 248 249 return &deleteEvent, &insertEvent, nil 250 } 251 252 // clear event buffer and row buffer. 253 // Also, it unrefs data that are holded by buffers. 254 func (n *sinkNode) clearBuffers() { 255 // Do not hog memory. 256 if cap(n.rowBuffer) > defaultSyncResolvedBatch { 257 n.rowBuffer = make([]*model.RowChangedEvent, 0, defaultSyncResolvedBatch) 258 } else { 259 for i := range n.rowBuffer { 260 n.rowBuffer[i] = nil 261 } 262 n.rowBuffer = n.rowBuffer[:0] 263 } 264 265 if cap(n.eventBuffer) > defaultSyncResolvedBatch { 266 n.eventBuffer = make([]*model.PolymorphicEvent, 0, defaultSyncResolvedBatch) 267 } else { 268 for i := range n.eventBuffer { 269 n.eventBuffer[i] = nil 270 } 271 n.eventBuffer = n.eventBuffer[:0] 272 } 273 } 274 275 func (n *sinkNode) emitRow2Sink(ctx pipeline.NodeContext) error { 276 for _, ev := range n.eventBuffer { 277 err := ev.WaitPrepare(ctx) 278 if err != nil { 279 return errors.Trace(err) 280 } 281 if ev.Row == nil { 282 continue 283 } 284 ev.Row.ReplicaID = ev.ReplicaID 285 n.rowBuffer = append(n.rowBuffer, ev.Row) 286 } 287 failpoint.Inject("ProcessorSyncResolvedPreEmit", func() { 288 log.Info("Prepare to panic for ProcessorSyncResolvedPreEmit") 289 time.Sleep(10 * time.Second) 290 panic("ProcessorSyncResolvedPreEmit") 291 }) 292 err := n.sink.EmitRowChangedEvents(ctx, n.rowBuffer...) 293 if err != nil { 294 return errors.Trace(err) 295 } 296 n.clearBuffers() 297 return nil 298 } 299 300 // Receive receives the message from the previous node 301 func (n *sinkNode) Receive(ctx pipeline.NodeContext) error { 302 if n.status == TableStatusStopped { 303 return cerror.ErrTableProcessorStoppedSafely.GenWithStackByArgs() 304 } 305 msg := ctx.Message() 306 switch msg.Tp { 307 case pipeline.MessageTypePolymorphicEvent: 308 event := msg.PolymorphicEvent 309 if event.RawKV.OpType == model.OpTypeResolved { 310 if n.status == TableStatusInitializing { 311 n.status.Store(TableStatusRunning) 312 } 313 failpoint.Inject("ProcessorSyncResolvedError", func() { 314 failpoint.Return(errors.New("processor sync resolved injected error")) 315 }) 316 if err := n.flushSink(ctx, msg.PolymorphicEvent.CRTs); err != nil { 317 return errors.Trace(err) 318 } 319 atomic.StoreUint64(&n.resolvedTs, msg.PolymorphicEvent.CRTs) 320 return nil 321 } 322 if err := n.emitEvent(ctx, event); err != nil { 323 return errors.Trace(err) 324 } 325 case pipeline.MessageTypeTick: 326 if err := n.flushSink(ctx, n.resolvedTs); err != nil { 327 return errors.Trace(err) 328 } 329 case pipeline.MessageTypeCommand: 330 if msg.Command.Tp == pipeline.CommandTypeStop { 331 return n.stop(ctx) 332 } 333 case pipeline.MessageTypeBarrier: 334 n.barrierTs = msg.BarrierTs 335 if err := n.flushSink(ctx, n.resolvedTs); err != nil { 336 return errors.Trace(err) 337 } 338 } 339 return nil 340 } 341 342 func (n *sinkNode) Destroy(ctx pipeline.NodeContext) error { 343 n.status.Store(TableStatusStopped) 344 n.flowController.Abort() 345 return n.sink.Close(ctx) 346 }