github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/open/open_protocol_message.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.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 open
    15  
    16  import (
    17  	"bytes"
    18  	"encoding/json"
    19  	"sort"
    20  	"strings"
    21  
    22  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    23  	"github.com/pingcap/tiflow/cdc/model"
    24  	cerror "github.com/pingcap/tiflow/pkg/errors"
    25  	"github.com/pingcap/tiflow/pkg/sink/codec"
    26  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    27  	"github.com/pingcap/tiflow/pkg/sink/codec/internal"
    28  )
    29  
    30  type columnsArray []*model.Column
    31  
    32  func (a columnsArray) Len() int {
    33  	return len(a)
    34  }
    35  
    36  func (a columnsArray) Less(i, j int) bool {
    37  	return a[i].Name < a[j].Name
    38  }
    39  
    40  func (a columnsArray) Swap(i, j int) {
    41  	a[i], a[j] = a[j], a[i]
    42  }
    43  
    44  // sortColumnArrays sort column arrays by name
    45  func sortColumnArrays(arrays ...[]*model.Column) {
    46  	for _, array := range arrays {
    47  		if array != nil {
    48  			sort.Sort(columnsArray(array))
    49  		}
    50  	}
    51  }
    52  
    53  type messageRow struct {
    54  	Update     map[string]internal.Column `json:"u,omitempty"`
    55  	PreColumns map[string]internal.Column `json:"p,omitempty"`
    56  	Delete     map[string]internal.Column `json:"d,omitempty"`
    57  }
    58  
    59  func (m *messageRow) encode() ([]byte, error) {
    60  	data, err := json.Marshal(m)
    61  	return data, cerror.WrapError(cerror.ErrMarshalFailed, err)
    62  }
    63  
    64  func (m *messageRow) decode(data []byte) error {
    65  	decoder := json.NewDecoder(bytes.NewReader(data))
    66  	decoder.UseNumber()
    67  	err := decoder.Decode(m)
    68  	if err != nil {
    69  		return cerror.WrapError(cerror.ErrUnmarshalFailed, err)
    70  	}
    71  	for colName, column := range m.Update {
    72  		m.Update[colName] = internal.FormatColumn(column)
    73  	}
    74  	for colName, column := range m.Delete {
    75  		m.Delete[colName] = internal.FormatColumn(column)
    76  	}
    77  	for colName, column := range m.PreColumns {
    78  		m.PreColumns[colName] = internal.FormatColumn(column)
    79  	}
    80  	return nil
    81  }
    82  
    83  func (m *messageRow) dropNotUpdatedColumns() {
    84  	// if the column is not updated, do not output it.
    85  	for col, value := range m.Update {
    86  		oldValue, ok := m.PreColumns[col]
    87  		if !ok {
    88  			continue
    89  		}
    90  		// sql type is not equal
    91  		if value.Type != oldValue.Type {
    92  			continue
    93  		}
    94  		// value equal
    95  		if codec.IsColumnValueEqual(oldValue.Value, value.Value) {
    96  			delete(m.PreColumns, col)
    97  		}
    98  	}
    99  }
   100  
   101  type messageDDL struct {
   102  	Query string             `json:"q"`
   103  	Type  timodel.ActionType `json:"t"`
   104  }
   105  
   106  func (m *messageDDL) encode() ([]byte, error) {
   107  	data, err := json.Marshal(m)
   108  	return data, cerror.WrapError(cerror.ErrMarshalFailed, err)
   109  }
   110  
   111  func (m *messageDDL) decode(data []byte) error {
   112  	return cerror.WrapError(cerror.ErrUnmarshalFailed, json.Unmarshal(data, m))
   113  }
   114  
   115  func newResolvedMessage(ts uint64) *internal.MessageKey {
   116  	return &internal.MessageKey{
   117  		Ts:   ts,
   118  		Type: model.MessageTypeResolved,
   119  	}
   120  }
   121  
   122  func rowChangeToMsg(
   123  	e *model.RowChangedEvent,
   124  	config *common.Config,
   125  	largeMessageOnlyHandleKeyColumns bool) (*internal.MessageKey, *messageRow, error) {
   126  	var partition *int64
   127  	if e.TableInfo.IsPartitionTable() {
   128  		partition = &e.PhysicalTableID
   129  	}
   130  	key := &internal.MessageKey{
   131  		Ts:            e.CommitTs,
   132  		Schema:        e.TableInfo.GetSchemaName(),
   133  		Table:         e.TableInfo.GetTableName(),
   134  		RowID:         e.RowID,
   135  		Partition:     partition,
   136  		Type:          model.MessageTypeRow,
   137  		OnlyHandleKey: largeMessageOnlyHandleKeyColumns,
   138  	}
   139  	value := &messageRow{}
   140  	if e.IsDelete() {
   141  		onlyHandleKeyColumns := config.DeleteOnlyHandleKeyColumns || largeMessageOnlyHandleKeyColumns
   142  		value.Delete = rowChangeColumns2CodecColumns(e.GetPreColumns(), onlyHandleKeyColumns)
   143  		if onlyHandleKeyColumns && len(value.Delete) == 0 {
   144  			return nil, nil, cerror.ErrOpenProtocolCodecInvalidData.GenWithStack("not found handle key columns for the delete event")
   145  		}
   146  	} else if e.IsUpdate() {
   147  		value.Update = rowChangeColumns2CodecColumns(e.GetColumns(), largeMessageOnlyHandleKeyColumns)
   148  		if config.OpenOutputOldValue {
   149  			value.PreColumns = rowChangeColumns2CodecColumns(e.GetPreColumns(), largeMessageOnlyHandleKeyColumns)
   150  		}
   151  		if largeMessageOnlyHandleKeyColumns && (len(value.Update) == 0 ||
   152  			(len(value.PreColumns) == 0 && !config.OpenOutputOldValue)) {
   153  			return nil, nil, cerror.ErrOpenProtocolCodecInvalidData.GenWithStack("not found handle key columns for the update event")
   154  		}
   155  		if config.OnlyOutputUpdatedColumns {
   156  			value.dropNotUpdatedColumns()
   157  		}
   158  	} else {
   159  		value.Update = rowChangeColumns2CodecColumns(e.GetColumns(), largeMessageOnlyHandleKeyColumns)
   160  		if largeMessageOnlyHandleKeyColumns && len(value.Update) == 0 {
   161  			return nil, nil, cerror.ErrOpenProtocolCodecInvalidData.GenWithStack("not found handle key columns for the insert event")
   162  		}
   163  	}
   164  
   165  	return key, value, nil
   166  }
   167  
   168  func msgToRowChange(key *internal.MessageKey, value *messageRow) *model.RowChangedEvent {
   169  	e := new(model.RowChangedEvent)
   170  	// TODO: we lost the startTs from kafka message
   171  	// startTs-based txn filter is out of work
   172  	e.CommitTs = key.Ts
   173  
   174  	if len(value.Delete) != 0 {
   175  		preCols := codecColumns2RowChangeColumns(value.Delete)
   176  		sortColumnArrays(preCols)
   177  		indexColumns := model.GetHandleAndUniqueIndexOffsets4Test(preCols)
   178  		e.TableInfo = model.BuildTableInfo(key.Schema, key.Table, preCols, indexColumns)
   179  		e.PreColumns = model.Columns2ColumnDatas(preCols, e.TableInfo)
   180  	} else {
   181  		cols := codecColumns2RowChangeColumns(value.Update)
   182  		preCols := codecColumns2RowChangeColumns(value.PreColumns)
   183  		sortColumnArrays(cols)
   184  		sortColumnArrays(preCols)
   185  		indexColumns := model.GetHandleAndUniqueIndexOffsets4Test(cols)
   186  		e.TableInfo = model.BuildTableInfo(key.Schema, key.Table, cols, indexColumns)
   187  		e.Columns = model.Columns2ColumnDatas(cols, e.TableInfo)
   188  		e.PreColumns = model.Columns2ColumnDatas(preCols, e.TableInfo)
   189  	}
   190  
   191  	// TODO: we lost the tableID from kafka message
   192  	if key.Partition != nil {
   193  		e.PhysicalTableID = *key.Partition
   194  		e.TableInfo.TableName.IsPartition = true
   195  	}
   196  
   197  	return e
   198  }
   199  
   200  func rowChangeColumns2CodecColumns(cols []*model.Column, onlyHandleKeyColumns bool) map[string]internal.Column {
   201  	jsonCols := make(map[string]internal.Column, len(cols))
   202  	for _, col := range cols {
   203  		if col == nil {
   204  			continue
   205  		}
   206  		if onlyHandleKeyColumns && !col.Flag.IsHandleKey() {
   207  			continue
   208  		}
   209  		c := internal.Column{}
   210  		c.FromRowChangeColumn(col)
   211  		jsonCols[col.Name] = c
   212  	}
   213  	if len(jsonCols) == 0 {
   214  		return nil
   215  	}
   216  	return jsonCols
   217  }
   218  
   219  func codecColumns2RowChangeColumns(cols map[string]internal.Column) []*model.Column {
   220  	sinkCols := make([]*model.Column, 0, len(cols))
   221  	for name, col := range cols {
   222  		c := col.ToRowChangeColumn(name)
   223  		sinkCols = append(sinkCols, c)
   224  	}
   225  	if len(sinkCols) == 0 {
   226  		return nil
   227  	}
   228  	sort.Slice(sinkCols, func(i, j int) bool {
   229  		return strings.Compare(sinkCols[i].Name, sinkCols[j].Name) > 0
   230  	})
   231  	return sinkCols
   232  }
   233  
   234  func ddlEventToMsg(e *model.DDLEvent) (*internal.MessageKey, *messageDDL) {
   235  	key := &internal.MessageKey{
   236  		Ts:     e.CommitTs,
   237  		Schema: e.TableInfo.TableName.Schema,
   238  		Table:  e.TableInfo.TableName.Table,
   239  		Type:   model.MessageTypeDDL,
   240  	}
   241  	value := &messageDDL{
   242  		Query: e.Query,
   243  		Type:  e.Type,
   244  	}
   245  	return key, value
   246  }
   247  
   248  func msgToDDLEvent(key *internal.MessageKey, value *messageDDL) *model.DDLEvent {
   249  	e := new(model.DDLEvent)
   250  	e.TableInfo = new(model.TableInfo)
   251  	// TODO: we lost the startTs from kafka message
   252  	// startTs-based txn filter is out of work
   253  	e.CommitTs = key.Ts
   254  	e.TableInfo.TableName = model.TableName{
   255  		Schema: key.Schema,
   256  		Table:  key.Table,
   257  	}
   258  	e.Type = value.Type
   259  	e.Query = value.Query
   260  	return e
   261  }