github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/open/open_protocol_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.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  	"context"
    18  	"database/sql"
    19  	"encoding/binary"
    20  	"path/filepath"
    21  	"strings"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/tidb/br/pkg/storage"
    26  	"github.com/pingcap/tidb/pkg/parser/mysql"
    27  	"github.com/pingcap/tidb/pkg/parser/types"
    28  	"github.com/pingcap/tiflow/cdc/model"
    29  	cerror "github.com/pingcap/tiflow/pkg/errors"
    30  	"github.com/pingcap/tiflow/pkg/sink/codec"
    31  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    32  	"github.com/pingcap/tiflow/pkg/sink/codec/internal"
    33  	"github.com/pingcap/tiflow/pkg/util"
    34  	"go.uber.org/zap"
    35  )
    36  
    37  // BatchDecoder decodes the byte of a batch into the original messages.
    38  type BatchDecoder struct {
    39  	keyBytes   []byte
    40  	valueBytes []byte
    41  
    42  	nextKey   *internal.MessageKey
    43  	nextEvent *model.RowChangedEvent
    44  
    45  	storage storage.ExternalStorage
    46  
    47  	config *common.Config
    48  
    49  	upstreamTiDB *sql.DB
    50  }
    51  
    52  // NewBatchDecoder creates a new BatchDecoder.
    53  func NewBatchDecoder(ctx context.Context, config *common.Config, db *sql.DB) (codec.RowEventDecoder, error) {
    54  	var (
    55  		externalStorage storage.ExternalStorage
    56  		err             error
    57  	)
    58  	if config.LargeMessageHandle.EnableClaimCheck() {
    59  		storageURI := config.LargeMessageHandle.ClaimCheckStorageURI
    60  		externalStorage, err = util.GetExternalStorageFromURI(ctx, storageURI)
    61  		if err != nil {
    62  			return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err)
    63  		}
    64  	}
    65  
    66  	if config.LargeMessageHandle.HandleKeyOnly() && db == nil {
    67  		return nil, cerror.ErrCodecDecode.
    68  			GenWithStack("handle-key-only is enabled, but upstream TiDB is not provided")
    69  	}
    70  
    71  	return &BatchDecoder{
    72  		config:       config,
    73  		storage:      externalStorage,
    74  		upstreamTiDB: db,
    75  	}, nil
    76  }
    77  
    78  // AddKeyValue implements the RowEventDecoder interface
    79  func (b *BatchDecoder) AddKeyValue(key, value []byte) error {
    80  	if len(b.keyBytes) != 0 || len(b.valueBytes) != 0 {
    81  		return cerror.ErrOpenProtocolCodecInvalidData.
    82  			GenWithStack("decoder key and value not nil")
    83  	}
    84  	version := binary.BigEndian.Uint64(key[:8])
    85  	key = key[8:]
    86  	if version != codec.BatchVersion1 {
    87  		return cerror.ErrOpenProtocolCodecInvalidData.
    88  			GenWithStack("unexpected key format version")
    89  	}
    90  
    91  	b.keyBytes = key
    92  	b.valueBytes = value
    93  
    94  	return nil
    95  }
    96  
    97  func (b *BatchDecoder) hasNext() bool {
    98  	keyLen := len(b.keyBytes)
    99  	valueLen := len(b.valueBytes)
   100  
   101  	if keyLen > 0 && valueLen > 0 {
   102  		return true
   103  	}
   104  
   105  	if keyLen == 0 && valueLen != 0 || keyLen != 0 && valueLen == 0 {
   106  		log.Panic("open-protocol meet invalid data",
   107  			zap.Int("keyLen", keyLen), zap.Int("valueLen", valueLen))
   108  	}
   109  
   110  	return false
   111  }
   112  
   113  func (b *BatchDecoder) decodeNextKey() error {
   114  	keyLen := binary.BigEndian.Uint64(b.keyBytes[:8])
   115  	key := b.keyBytes[8 : keyLen+8]
   116  	msgKey := new(internal.MessageKey)
   117  	err := msgKey.Decode(key)
   118  	if err != nil {
   119  		return errors.Trace(err)
   120  	}
   121  	b.nextKey = msgKey
   122  
   123  	b.keyBytes = b.keyBytes[keyLen+8:]
   124  	return nil
   125  }
   126  
   127  // HasNext implements the RowEventDecoder interface
   128  func (b *BatchDecoder) HasNext() (model.MessageType, bool, error) {
   129  	if !b.hasNext() {
   130  		return 0, false, nil
   131  	}
   132  	if err := b.decodeNextKey(); err != nil {
   133  		return 0, false, err
   134  	}
   135  
   136  	if b.nextKey.Type == model.MessageTypeRow {
   137  		valueLen := binary.BigEndian.Uint64(b.valueBytes[:8])
   138  		value := b.valueBytes[8 : valueLen+8]
   139  		b.valueBytes = b.valueBytes[valueLen+8:]
   140  
   141  		rowMsg := new(messageRow)
   142  
   143  		value, err := common.Decompress(b.config.LargeMessageHandle.LargeMessageHandleCompression, value)
   144  		if err != nil {
   145  			return model.MessageTypeUnknown, false, cerror.ErrOpenProtocolCodecInvalidData.
   146  				GenWithStack("decompress data failed")
   147  		}
   148  
   149  		if err := rowMsg.decode(value); err != nil {
   150  			return b.nextKey.Type, false, errors.Trace(err)
   151  		}
   152  		b.nextEvent = msgToRowChange(b.nextKey, rowMsg)
   153  	}
   154  
   155  	return b.nextKey.Type, true, nil
   156  }
   157  
   158  // NextResolvedEvent implements the RowEventDecoder interface
   159  func (b *BatchDecoder) NextResolvedEvent() (uint64, error) {
   160  	if b.nextKey.Type != model.MessageTypeResolved {
   161  		return 0, cerror.ErrOpenProtocolCodecInvalidData.GenWithStack("not found resolved event message")
   162  	}
   163  	resolvedTs := b.nextKey.Ts
   164  	b.nextKey = nil
   165  	// resolved ts event's value part is empty, can be ignored.
   166  	b.valueBytes = nil
   167  	return resolvedTs, nil
   168  }
   169  
   170  // NextDDLEvent implements the RowEventDecoder interface
   171  func (b *BatchDecoder) NextDDLEvent() (*model.DDLEvent, error) {
   172  	if b.nextKey.Type != model.MessageTypeDDL {
   173  		return nil, cerror.ErrOpenProtocolCodecInvalidData.GenWithStack("not found ddl event message")
   174  	}
   175  
   176  	valueLen := binary.BigEndian.Uint64(b.valueBytes[:8])
   177  	value := b.valueBytes[8 : valueLen+8]
   178  
   179  	value, err := common.Decompress(b.config.LargeMessageHandle.LargeMessageHandleCompression, value)
   180  	if err != nil {
   181  		return nil, cerror.ErrOpenProtocolCodecInvalidData.
   182  			GenWithStack("decompress DDL event failed")
   183  	}
   184  
   185  	ddlMsg := new(messageDDL)
   186  	if err := ddlMsg.decode(value); err != nil {
   187  		return nil, errors.Trace(err)
   188  	}
   189  	ddlEvent := msgToDDLEvent(b.nextKey, ddlMsg)
   190  
   191  	b.nextKey = nil
   192  	b.valueBytes = nil
   193  	return ddlEvent, nil
   194  }
   195  
   196  // NextRowChangedEvent implements the RowEventDecoder interface
   197  func (b *BatchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) {
   198  	if b.nextKey.Type != model.MessageTypeRow {
   199  		return nil, cerror.ErrOpenProtocolCodecInvalidData.GenWithStack("not found row event message")
   200  	}
   201  
   202  	ctx := context.Background()
   203  	// claim-check message found
   204  	if b.nextKey.ClaimCheckLocation != "" {
   205  		return b.assembleEventFromClaimCheckStorage(ctx)
   206  	}
   207  
   208  	event := b.nextEvent
   209  	if b.nextKey.OnlyHandleKey {
   210  		event = b.assembleHandleKeyOnlyEvent(ctx, event)
   211  	}
   212  
   213  	b.nextKey = nil
   214  	return event, nil
   215  }
   216  
   217  func (b *BatchDecoder) buildColumns(
   218  	holder *common.ColumnsHolder, handleKeyColumns map[string]interface{},
   219  ) []*model.Column {
   220  	columnsCount := holder.Length()
   221  	columns := make([]*model.Column, 0, columnsCount)
   222  	for i := 0; i < columnsCount; i++ {
   223  		columnType := holder.Types[i]
   224  		name := columnType.Name()
   225  		mysqlType := types.StrToType(strings.ToLower(columnType.DatabaseTypeName()))
   226  
   227  		var value interface{}
   228  		value = holder.Values[i].([]uint8)
   229  
   230  		switch mysqlType {
   231  		case mysql.TypeJSON:
   232  			value = string(value.([]uint8))
   233  		case mysql.TypeBit:
   234  			value = common.MustBinaryLiteralToInt(value.([]uint8))
   235  		}
   236  
   237  		column := &model.Column{
   238  			Name:  name,
   239  			Type:  mysqlType,
   240  			Value: value,
   241  		}
   242  
   243  		if _, ok := handleKeyColumns[name]; ok {
   244  			column.Flag = model.PrimaryKeyFlag | model.HandleKeyFlag
   245  		}
   246  		columns = append(columns, column)
   247  	}
   248  	return columns
   249  }
   250  
   251  func (b *BatchDecoder) assembleHandleKeyOnlyEvent(
   252  	ctx context.Context, handleKeyOnlyEvent *model.RowChangedEvent,
   253  ) *model.RowChangedEvent {
   254  	var (
   255  		schema   = handleKeyOnlyEvent.TableInfo.GetSchemaName()
   256  		table    = handleKeyOnlyEvent.TableInfo.GetTableName()
   257  		commitTs = handleKeyOnlyEvent.CommitTs
   258  	)
   259  
   260  	tableInfo := handleKeyOnlyEvent.TableInfo
   261  	if handleKeyOnlyEvent.IsInsert() {
   262  		conditions := make(map[string]interface{}, len(handleKeyOnlyEvent.Columns))
   263  		for _, col := range handleKeyOnlyEvent.Columns {
   264  			colName := tableInfo.ForceGetColumnName(col.ColumnID)
   265  			conditions[colName] = col.Value
   266  		}
   267  		holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs, schema, table, conditions)
   268  		columns := b.buildColumns(holder, conditions)
   269  		indexColumns := model.GetHandleAndUniqueIndexOffsets4Test(columns)
   270  		handleKeyOnlyEvent.TableInfo = model.BuildTableInfo(schema, table, columns, indexColumns)
   271  		handleKeyOnlyEvent.Columns = model.Columns2ColumnDatas(columns, handleKeyOnlyEvent.TableInfo)
   272  	} else if handleKeyOnlyEvent.IsDelete() {
   273  		conditions := make(map[string]interface{}, len(handleKeyOnlyEvent.PreColumns))
   274  		for _, col := range handleKeyOnlyEvent.PreColumns {
   275  			colName := tableInfo.ForceGetColumnName(col.ColumnID)
   276  			conditions[colName] = col.Value
   277  		}
   278  		holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs-1, schema, table, conditions)
   279  		preColumns := b.buildColumns(holder, conditions)
   280  		indexColumns := model.GetHandleAndUniqueIndexOffsets4Test(preColumns)
   281  		handleKeyOnlyEvent.TableInfo = model.BuildTableInfo(schema, table, preColumns, indexColumns)
   282  		handleKeyOnlyEvent.PreColumns = model.Columns2ColumnDatas(preColumns, handleKeyOnlyEvent.TableInfo)
   283  	} else if handleKeyOnlyEvent.IsUpdate() {
   284  		conditions := make(map[string]interface{}, len(handleKeyOnlyEvent.Columns))
   285  		for _, col := range handleKeyOnlyEvent.Columns {
   286  			colName := tableInfo.ForceGetColumnName(col.ColumnID)
   287  			conditions[colName] = col.Value
   288  		}
   289  		holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs, schema, table, conditions)
   290  		columns := b.buildColumns(holder, conditions)
   291  		indexColumns := model.GetHandleAndUniqueIndexOffsets4Test(columns)
   292  		handleKeyOnlyEvent.TableInfo = model.BuildTableInfo(schema, table, columns, indexColumns)
   293  		handleKeyOnlyEvent.Columns = model.Columns2ColumnDatas(columns, handleKeyOnlyEvent.TableInfo)
   294  
   295  		conditions = make(map[string]interface{}, len(handleKeyOnlyEvent.PreColumns))
   296  		for _, col := range handleKeyOnlyEvent.PreColumns {
   297  			colName := tableInfo.ForceGetColumnName(col.ColumnID)
   298  			conditions[colName] = col.Value
   299  		}
   300  		holder = common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs-1, schema, table, conditions)
   301  		preColumns := b.buildColumns(holder, conditions)
   302  		handleKeyOnlyEvent.PreColumns = model.Columns2ColumnDatas(preColumns, handleKeyOnlyEvent.TableInfo)
   303  	}
   304  
   305  	return handleKeyOnlyEvent
   306  }
   307  
   308  func (b *BatchDecoder) assembleEventFromClaimCheckStorage(ctx context.Context) (*model.RowChangedEvent, error) {
   309  	_, claimCheckFileName := filepath.Split(b.nextKey.ClaimCheckLocation)
   310  	b.nextKey = nil
   311  	data, err := b.storage.ReadFile(ctx, claimCheckFileName)
   312  	if err != nil {
   313  		return nil, errors.Trace(err)
   314  	}
   315  	claimCheckM, err := common.UnmarshalClaimCheckMessage(data)
   316  	if err != nil {
   317  		return nil, errors.Trace(err)
   318  	}
   319  
   320  	version := binary.BigEndian.Uint64(claimCheckM.Key[:8])
   321  	if version != codec.BatchVersion1 {
   322  		return nil, cerror.ErrOpenProtocolCodecInvalidData.
   323  			GenWithStack("unexpected key format version")
   324  	}
   325  
   326  	key := claimCheckM.Key[8:]
   327  	keyLen := binary.BigEndian.Uint64(key[:8])
   328  	key = key[8 : keyLen+8]
   329  	msgKey := new(internal.MessageKey)
   330  	if err := msgKey.Decode(key); err != nil {
   331  		return nil, errors.Trace(err)
   332  	}
   333  
   334  	valueLen := binary.BigEndian.Uint64(claimCheckM.Value[:8])
   335  	value := claimCheckM.Value[8 : valueLen+8]
   336  	value, err = common.Decompress(b.config.LargeMessageHandle.LargeMessageHandleCompression, value)
   337  	if err != nil {
   338  		return nil, cerror.WrapError(cerror.ErrOpenProtocolCodecInvalidData, err)
   339  	}
   340  
   341  	rowMsg := new(messageRow)
   342  	if err := rowMsg.decode(value); err != nil {
   343  		return nil, errors.Trace(err)
   344  	}
   345  
   346  	event := msgToRowChange(msgKey, rowMsg)
   347  
   348  	return event, nil
   349  }