github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/craft/craft_decoder.go (about) 1 // Copyright 2022 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.orglicensesLICENSE-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 craft 15 16 import ( 17 "github.com/pingcap/errors" 18 "github.com/pingcap/tiflow/cdc/model" 19 cerror "github.com/pingcap/tiflow/pkg/errors" 20 "github.com/pingcap/tiflow/pkg/sink/codec" 21 ) 22 23 // batchDecoder decodes the byte of a batch into the original messages. 24 type batchDecoder struct { 25 headers *Headers 26 decoder *MessageDecoder 27 index int 28 29 allocator *SliceAllocator 30 } 31 32 // HasNext implements the RowEventDecoder interface 33 func (b *batchDecoder) HasNext() (model.MessageType, bool, error) { 34 if b.index >= b.headers.Count() { 35 return model.MessageTypeUnknown, false, nil 36 } 37 return b.headers.GetType(b.index), true, nil 38 } 39 40 // NextResolvedEvent implements the RowEventDecoder interface 41 func (b *batchDecoder) NextResolvedEvent() (uint64, error) { 42 ty, hasNext, err := b.HasNext() 43 if err != nil { 44 return 0, errors.Trace(err) 45 } 46 if !hasNext || ty != model.MessageTypeResolved { 47 return 0, cerror.ErrCraftCodecInvalidData.GenWithStack("not found resolved event message") 48 } 49 ts := b.headers.GetTs(b.index) 50 b.index++ 51 return ts, nil 52 } 53 54 // NextRowChangedEvent implements the RowEventDecoder interface 55 func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { 56 ty, hasNext, err := b.HasNext() 57 if err != nil { 58 return nil, errors.Trace(err) 59 } 60 if !hasNext || ty != model.MessageTypeRow { 61 return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found row changed event message") 62 } 63 oldValue, newValue, err := b.decoder.RowChangedEvent(b.index) 64 if err != nil { 65 return nil, errors.Trace(err) 66 } 67 ev := &model.RowChangedEvent{} 68 var cols, preCols []*model.Column 69 if oldValue != nil { 70 if preCols, err = oldValue.ToModel(); err != nil { 71 return nil, errors.Trace(err) 72 } 73 } 74 if newValue != nil { 75 if cols, err = newValue.ToModel(); err != nil { 76 return nil, errors.Trace(err) 77 } 78 } 79 ev.CommitTs = b.headers.GetTs(b.index) 80 if len(preCols) > 0 { 81 indexColumns := model.GetHandleAndUniqueIndexOffsets4Test(preCols) 82 ev.TableInfo = model.BuildTableInfo(b.headers.GetSchema(b.index), b.headers.GetTable(b.index), preCols, indexColumns) 83 } else { 84 indexColumns := model.GetHandleAndUniqueIndexOffsets4Test(cols) 85 ev.TableInfo = model.BuildTableInfo(b.headers.GetSchema(b.index), b.headers.GetTable(b.index), cols, indexColumns) 86 } 87 if len(preCols) > 0 { 88 ev.PreColumns = model.Columns2ColumnDatas(preCols, ev.TableInfo) 89 } 90 if len(cols) > 0 { 91 ev.Columns = model.Columns2ColumnDatas(cols, ev.TableInfo) 92 } 93 partition := b.headers.GetPartition(b.index) 94 if partition >= 0 { 95 ev.PhysicalTableID = partition 96 ev.TableInfo.TableName.IsPartition = true 97 } 98 b.index++ 99 return ev, nil 100 } 101 102 // NextDDLEvent implements the RowEventDecoder interface 103 func (b *batchDecoder) NextDDLEvent() (*model.DDLEvent, error) { 104 ty, hasNext, err := b.HasNext() 105 if err != nil { 106 return nil, errors.Trace(err) 107 } 108 if !hasNext || ty != model.MessageTypeDDL { 109 return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found ddl event message") 110 } 111 ddlType, query, err := b.decoder.DDLEvent(b.index) 112 if err != nil { 113 return nil, errors.Trace(err) 114 } 115 event := &model.DDLEvent{ 116 CommitTs: b.headers.GetTs(b.index), 117 Query: query, 118 Type: ddlType, 119 TableInfo: &model.TableInfo{ 120 TableName: model.TableName{ 121 Schema: b.headers.GetSchema(b.index), 122 Table: b.headers.GetTable(b.index), 123 }, 124 }, 125 } 126 b.index++ 127 return event, nil 128 } 129 130 func newBatchDecoder(_, value []byte) (codec.RowEventDecoder, error) { 131 decoder := NewBatchDecoderWithAllocator(NewSliceAllocator(64)) 132 err := decoder.AddKeyValue(nil, value) 133 return decoder, err 134 } 135 136 // NewBatchDecoderWithAllocator creates a new batchDecoder with given allocator. 137 func NewBatchDecoderWithAllocator( 138 allocator *SliceAllocator, 139 ) codec.RowEventDecoder { 140 return &batchDecoder{ 141 allocator: allocator, 142 } 143 } 144 145 // AddKeyValue implements the RowEventDecoder interface 146 func (b *batchDecoder) AddKeyValue(_, value []byte) error { 147 decoder, err := NewMessageDecoder(value, b.allocator) 148 if err != nil { 149 return errors.Trace(err) 150 } 151 headers, err := decoder.Headers() 152 if err != nil { 153 return errors.Trace(err) 154 } 155 b.decoder = decoder 156 b.headers = headers 157 158 return nil 159 }