github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/codec/interface.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 codec 15 16 import ( 17 "strings" 18 "time" 19 20 "github.com/pingcap/log" 21 "github.com/pingcap/ticdc/cdc/model" 22 "github.com/pingcap/tidb/store/tikv/oracle" 23 "go.uber.org/zap" 24 ) 25 26 // EventBatchEncoder is an abstraction for events encoder 27 type EventBatchEncoder interface { 28 // EncodeCheckpointEvent appends a checkpoint event into the batch. 29 // This event will be broadcast to all partitions to signal a global checkpoint. 30 EncodeCheckpointEvent(ts uint64) (*MQMessage, error) 31 // AppendRowChangedEvent appends a row changed event into the batch 32 AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) 33 // AppendResolvedEvent appends a resolved event into the batch. 34 // This event is used to tell the encoder that no event prior to ts will be sent. 35 AppendResolvedEvent(ts uint64) (EncoderResult, error) 36 // EncodeDDLEvent appends a DDL event into the batch 37 EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) 38 // Build builds the batch and returns the bytes of key and value. 39 Build() []*MQMessage 40 // MixedBuild builds the batch and returns the bytes of mixed keys and values. 41 // This is used for cdc log, to merge key and value into one byte slice 42 // when first create file, we should set withVersion to true, to tell us that 43 // the first 8 byte represents the encoder version 44 // TODO decouple it out 45 MixedBuild(withVersion bool) []byte 46 // Size returns the size of the batch(bytes) 47 // Deprecated: Size is deprecated 48 Size() int 49 // Reset reset the kv buffer 50 Reset() 51 // SetParams provides the encoder with more info on the sink 52 SetParams(params map[string]string) error 53 } 54 55 // MQMessage represents an MQ message to the mqSink 56 type MQMessage struct { 57 Key []byte 58 Value []byte 59 Ts uint64 // reserved for possible output sorting 60 Schema *string // schema 61 Table *string // table 62 Type model.MqMessageType // type 63 Protocol Protocol // protocol 64 } 65 66 // Length returns the expected size of the Kafka message 67 func (m *MQMessage) Length() int { 68 return len(m.Key) + len(m.Value) 69 } 70 71 // PhysicalTime returns physical time part of Ts in time.Time 72 func (m *MQMessage) PhysicalTime() time.Time { 73 return oracle.GetTimeFromTS(m.Ts) 74 } 75 76 func newDDLMQMessage(proto Protocol, key, value []byte, event *model.DDLEvent) *MQMessage { 77 return NewMQMessage(proto, key, value, event.CommitTs, model.MqMessageTypeDDL, &event.TableInfo.Schema, &event.TableInfo.Table) 78 } 79 80 func newResolvedMQMessage(proto Protocol, key, value []byte, ts uint64) *MQMessage { 81 return NewMQMessage(proto, key, value, ts, model.MqMessageTypeResolved, nil, nil) 82 } 83 84 // NewMQMessage should be used when creating a MQMessage struct. 85 // It copies the input byte slices to avoid any surprises in asynchronous MQ writes. 86 func NewMQMessage(proto Protocol, key []byte, value []byte, ts uint64, ty model.MqMessageType, schema, table *string) *MQMessage { 87 ret := &MQMessage{ 88 Key: nil, 89 Value: nil, 90 Ts: ts, 91 Schema: schema, 92 Table: table, 93 Type: ty, 94 Protocol: proto, 95 } 96 97 if key != nil { 98 ret.Key = make([]byte, len(key)) 99 copy(ret.Key, key) 100 } 101 102 if value != nil { 103 ret.Value = make([]byte, len(value)) 104 copy(ret.Value, value) 105 } 106 107 return ret 108 } 109 110 // EventBatchDecoder is an abstraction for events decoder 111 // this interface is only for testing now 112 type EventBatchDecoder interface { 113 // HasNext returns 114 // 1. the type of the next event 115 // 2. a bool if the next event is exist 116 // 3. error 117 HasNext() (model.MqMessageType, bool, error) 118 // NextResolvedEvent returns the next resolved event if exists 119 NextResolvedEvent() (uint64, error) 120 // NextRowChangedEvent returns the next row changed event if exists 121 NextRowChangedEvent() (*model.RowChangedEvent, error) 122 // NextDDLEvent returns the next DDL event if exists 123 NextDDLEvent() (*model.DDLEvent, error) 124 } 125 126 // EncoderResult indicates an action request by the encoder to the mqSink 127 type EncoderResult uint8 128 129 // Enum types of EncoderResult 130 const ( 131 EncoderNoOperation EncoderResult = iota 132 EncoderNeedAsyncWrite 133 EncoderNeedSyncWrite 134 ) 135 136 // Protocol is the protocol of the mq message 137 type Protocol int 138 139 // Enum types of the Protocol 140 const ( 141 ProtocolDefault Protocol = iota 142 ProtocolCanal 143 ProtocolAvro 144 ProtocolMaxwell 145 ProtocolCanalJSON 146 ) 147 148 // FromString converts the protocol from string to Protocol enum type 149 func (p *Protocol) FromString(protocol string) { 150 switch strings.ToLower(protocol) { 151 case "default": 152 *p = ProtocolDefault 153 case "canal": 154 *p = ProtocolCanal 155 case "avro": 156 *p = ProtocolAvro 157 case "maxwell": 158 *p = ProtocolMaxwell 159 case "canal-json": 160 *p = ProtocolCanalJSON 161 default: 162 *p = ProtocolDefault 163 log.Warn("can't support codec protocol, using default protocol", zap.String("protocol", protocol)) 164 } 165 } 166 167 // NewEventBatchEncoder returns a function of creating an EventBatchEncoder 168 func NewEventBatchEncoder(p Protocol) func() EventBatchEncoder { 169 switch p { 170 case ProtocolDefault: 171 return NewJSONEventBatchEncoder 172 case ProtocolCanal: 173 return NewCanalEventBatchEncoder 174 case ProtocolAvro: 175 return NewAvroEventBatchEncoder 176 case ProtocolMaxwell: 177 return NewMaxwellEventBatchEncoder 178 case ProtocolCanalJSON: 179 return NewCanalFlatEventBatchEncoder 180 default: 181 log.Warn("unknown codec protocol value of EventBatchEncoder", zap.Int("protocol_value", int(p))) 182 return NewJSONEventBatchEncoder 183 } 184 }