github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/simple/decoder.go (about)

     1  // Copyright 2023 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 simple
    15  
    16  import (
    17  	"container/list"
    18  	"context"
    19  	"database/sql"
    20  	"path/filepath"
    21  
    22  	"github.com/pingcap/errors"
    23  	"github.com/pingcap/log"
    24  	"github.com/pingcap/tidb/br/pkg/storage"
    25  	"github.com/pingcap/tidb/pkg/types"
    26  	"github.com/pingcap/tiflow/cdc/model"
    27  	cerror "github.com/pingcap/tiflow/pkg/errors"
    28  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    29  	"github.com/pingcap/tiflow/pkg/util"
    30  	"go.uber.org/zap"
    31  )
    32  
    33  // Decoder implement the RowEventDecoder interface
    34  type Decoder struct {
    35  	config *common.Config
    36  
    37  	marshaller marshaller
    38  
    39  	upstreamTiDB *sql.DB
    40  	storage      storage.ExternalStorage
    41  
    42  	value []byte
    43  	msg   *message
    44  	memo  TableInfoProvider
    45  
    46  	// cachedMessages is used to store the messages which does not have received corresponding table info yet.
    47  	cachedMessages *list.List
    48  	// CachedRowChangedEvents are events just decoded from the cachedMessages
    49  	CachedRowChangedEvents []*model.RowChangedEvent
    50  }
    51  
    52  // NewDecoder returns a new Decoder
    53  func NewDecoder(ctx context.Context, config *common.Config, db *sql.DB) (*Decoder, 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  	m, err := newMarshaller(config)
    72  	return &Decoder{
    73  		config:     config,
    74  		marshaller: m,
    75  
    76  		storage:      externalStorage,
    77  		upstreamTiDB: db,
    78  
    79  		memo:           newMemoryTableInfoProvider(),
    80  		cachedMessages: list.New(),
    81  	}, errors.Trace(err)
    82  }
    83  
    84  // AddKeyValue add the received key and values to the Decoder,
    85  func (d *Decoder) AddKeyValue(_, value []byte) (err error) {
    86  	if d.value != nil {
    87  		return cerror.ErrCodecDecode.GenWithStack(
    88  			"Decoder value already exists, not consumed yet")
    89  	}
    90  	d.value, err = common.Decompress(d.config.LargeMessageHandle.LargeMessageHandleCompression, value)
    91  	return err
    92  }
    93  
    94  // HasNext returns whether there is any event need to be consumed
    95  func (d *Decoder) HasNext() (model.MessageType, bool, error) {
    96  	if d.value == nil {
    97  		return model.MessageTypeUnknown, false, nil
    98  	}
    99  
   100  	m := new(message)
   101  	err := d.marshaller.Unmarshal(d.value, m)
   102  	if err != nil {
   103  		return model.MessageTypeUnknown, false, cerror.WrapError(cerror.ErrDecodeFailed, err)
   104  	}
   105  	d.msg = m
   106  	d.value = nil
   107  
   108  	if d.msg.Data != nil || d.msg.Old != nil {
   109  		return model.MessageTypeRow, true, nil
   110  	}
   111  
   112  	if m.Type == MessageTypeWatermark {
   113  		return model.MessageTypeResolved, true, nil
   114  	}
   115  
   116  	return model.MessageTypeDDL, true, nil
   117  }
   118  
   119  // NextResolvedEvent returns the next resolved event if exists
   120  func (d *Decoder) NextResolvedEvent() (uint64, error) {
   121  	if d.msg.Type != MessageTypeWatermark {
   122  		return 0, cerror.ErrCodecDecode.GenWithStack(
   123  			"not found resolved event message")
   124  	}
   125  
   126  	ts := d.msg.CommitTs
   127  	d.msg = nil
   128  
   129  	return ts, nil
   130  }
   131  
   132  // NextRowChangedEvent returns the next row changed event if exists
   133  func (d *Decoder) NextRowChangedEvent() (*model.RowChangedEvent, error) {
   134  	if d.msg == nil || (d.msg.Data == nil && d.msg.Old == nil) {
   135  		return nil, cerror.ErrCodecDecode.GenWithStack(
   136  			"invalid row changed event message")
   137  	}
   138  
   139  	if d.msg.ClaimCheckLocation != "" {
   140  		return d.assembleClaimCheckRowChangedEvent(d.msg.ClaimCheckLocation)
   141  	}
   142  
   143  	if d.msg.HandleKeyOnly {
   144  		return d.assembleHandleKeyOnlyRowChangedEvent(d.msg)
   145  	}
   146  
   147  	tableInfo := d.memo.Read(d.msg.Schema, d.msg.Table, d.msg.SchemaVersion)
   148  	if tableInfo == nil {
   149  		log.Debug("table info not found for the event, "+
   150  			"the consumer should cache this event temporarily, and update the tableInfo after it's received",
   151  			zap.String("schema", d.msg.Schema),
   152  			zap.String("table", d.msg.Table),
   153  			zap.Uint64("version", d.msg.SchemaVersion))
   154  		d.cachedMessages.PushBack(d.msg)
   155  		d.msg = nil
   156  		return nil, nil
   157  	}
   158  
   159  	event, err := buildRowChangedEvent(d.msg, tableInfo, d.config.EnableRowChecksum)
   160  	d.msg = nil
   161  
   162  	log.Debug("row changed event assembled", zap.Any("event", event))
   163  	return event, err
   164  }
   165  
   166  func (d *Decoder) assembleClaimCheckRowChangedEvent(claimCheckLocation string) (*model.RowChangedEvent, error) {
   167  	_, claimCheckFileName := filepath.Split(claimCheckLocation)
   168  	data, err := d.storage.ReadFile(context.Background(), claimCheckFileName)
   169  	if err != nil {
   170  		return nil, err
   171  	}
   172  	claimCheckM, err := common.UnmarshalClaimCheckMessage(data)
   173  	if err != nil {
   174  		return nil, err
   175  	}
   176  
   177  	value, err := common.Decompress(d.config.LargeMessageHandle.LargeMessageHandleCompression, claimCheckM.Value)
   178  	if err != nil {
   179  		return nil, err
   180  	}
   181  
   182  	m := new(message)
   183  	err = d.marshaller.Unmarshal(value, m)
   184  	if err != nil {
   185  		return nil, err
   186  	}
   187  	d.msg = m
   188  	return d.NextRowChangedEvent()
   189  }
   190  
   191  func (d *Decoder) assembleHandleKeyOnlyRowChangedEvent(m *message) (*model.RowChangedEvent, error) {
   192  	tableInfo := d.memo.Read(m.Schema, m.Table, m.SchemaVersion)
   193  	if tableInfo == nil {
   194  		log.Debug("table info not found for the event, "+
   195  			"the consumer should cache this event temporarily, and update the tableInfo after it's received",
   196  			zap.String("schema", d.msg.Schema),
   197  			zap.String("table", d.msg.Table),
   198  			zap.Uint64("version", d.msg.SchemaVersion))
   199  		d.cachedMessages.PushBack(d.msg)
   200  		d.msg = nil
   201  		return nil, nil
   202  	}
   203  
   204  	fieldTypeMap := make(map[string]*types.FieldType, len(tableInfo.Columns))
   205  	for _, col := range tableInfo.Columns {
   206  		fieldTypeMap[col.Name.O] = &col.FieldType
   207  	}
   208  
   209  	result := &message{
   210  		Version:       defaultVersion,
   211  		Schema:        m.Schema,
   212  		Table:         m.Table,
   213  		TableID:       m.TableID,
   214  		Type:          m.Type,
   215  		CommitTs:      m.CommitTs,
   216  		SchemaVersion: m.SchemaVersion,
   217  	}
   218  
   219  	ctx := context.Background()
   220  	timezone := common.MustQueryTimezone(ctx, d.upstreamTiDB)
   221  	switch m.Type {
   222  	case DMLTypeInsert:
   223  		holder := common.MustSnapshotQuery(ctx, d.upstreamTiDB, m.CommitTs, m.Schema, m.Table, m.Data)
   224  		result.Data = d.buildData(holder, fieldTypeMap, timezone)
   225  	case DMLTypeUpdate:
   226  		holder := common.MustSnapshotQuery(ctx, d.upstreamTiDB, m.CommitTs, m.Schema, m.Table, m.Data)
   227  		result.Data = d.buildData(holder, fieldTypeMap, timezone)
   228  
   229  		holder = common.MustSnapshotQuery(ctx, d.upstreamTiDB, m.CommitTs-1, m.Schema, m.Table, m.Old)
   230  		result.Old = d.buildData(holder, fieldTypeMap, timezone)
   231  	case DMLTypeDelete:
   232  		holder := common.MustSnapshotQuery(ctx, d.upstreamTiDB, m.CommitTs-1, m.Schema, m.Table, m.Old)
   233  		result.Old = d.buildData(holder, fieldTypeMap, timezone)
   234  	}
   235  
   236  	d.msg = result
   237  	return d.NextRowChangedEvent()
   238  }
   239  
   240  func (d *Decoder) buildData(
   241  	holder *common.ColumnsHolder, fieldTypeMap map[string]*types.FieldType, timezone string,
   242  ) map[string]interface{} {
   243  	columnsCount := holder.Length()
   244  	result := make(map[string]interface{}, columnsCount)
   245  
   246  	for i := 0; i < columnsCount; i++ {
   247  		col := holder.Types[i]
   248  		value := holder.Values[i]
   249  
   250  		fieldType := fieldTypeMap[col.Name()]
   251  		result[col.Name()] = encodeValue(value, fieldType, timezone)
   252  	}
   253  	return result
   254  }
   255  
   256  // NextDDLEvent returns the next DDL event if exists
   257  func (d *Decoder) NextDDLEvent() (*model.DDLEvent, error) {
   258  	if d.msg == nil {
   259  		return nil, cerror.ErrCodecDecode.GenWithStack(
   260  			"no message found when decode DDL event")
   261  	}
   262  	ddl := newDDLEvent(d.msg)
   263  	d.msg = nil
   264  
   265  	d.memo.Write(ddl.TableInfo)
   266  	d.memo.Write(ddl.PreTableInfo)
   267  
   268  	for ele := d.cachedMessages.Front(); ele != nil; {
   269  		d.msg = ele.Value.(*message)
   270  		event, err := d.NextRowChangedEvent()
   271  		if err != nil {
   272  			return nil, err
   273  		}
   274  		d.CachedRowChangedEvents = append(d.CachedRowChangedEvents, event)
   275  
   276  		next := ele.Next()
   277  		d.cachedMessages.Remove(ele)
   278  		ele = next
   279  	}
   280  	return ddl, nil
   281  }
   282  
   283  // GetCachedEvents returns the cached events
   284  func (d *Decoder) GetCachedEvents() []*model.RowChangedEvent {
   285  	result := d.CachedRowChangedEvents
   286  	d.CachedRowChangedEvents = nil
   287  	return result
   288  }
   289  
   290  // TableInfoProvider is used to store and read table info
   291  // It works like a schema cache when consuming simple protocol messages
   292  // It will store multiple versions of table info for a table
   293  // The table info which has the exact (schema, table, version) will be returned when reading
   294  type TableInfoProvider interface {
   295  	Write(info *model.TableInfo)
   296  	Read(schema, table string, version uint64) *model.TableInfo
   297  }
   298  
   299  type memoryTableInfoProvider struct {
   300  	memo map[tableSchemaKey]*model.TableInfo
   301  }
   302  
   303  func newMemoryTableInfoProvider() *memoryTableInfoProvider {
   304  	return &memoryTableInfoProvider{
   305  		memo: make(map[tableSchemaKey]*model.TableInfo),
   306  	}
   307  }
   308  
   309  func (m *memoryTableInfoProvider) Write(info *model.TableInfo) {
   310  	if info == nil || info.TableName.Schema == "" || info.TableName.Table == "" {
   311  		return
   312  	}
   313  	key := tableSchemaKey{
   314  		schema:  info.TableName.Schema,
   315  		table:   info.TableName.Table,
   316  		version: info.UpdateTS,
   317  	}
   318  
   319  	_, ok := m.memo[key]
   320  	if ok {
   321  		log.Warn("table info not stored, since it already exists",
   322  			zap.String("schema", info.TableName.Schema),
   323  			zap.String("table", info.TableName.Table),
   324  			zap.Uint64("version", info.UpdateTS))
   325  		return
   326  	}
   327  
   328  	m.memo[key] = info
   329  	log.Info("table info stored",
   330  		zap.String("schema", info.TableName.Schema),
   331  		zap.String("table", info.TableName.Table),
   332  		zap.Uint64("version", info.UpdateTS))
   333  }
   334  
   335  // Read returns the table info with the exact (schema, table, version)
   336  // Note: It's a blocking call, it will wait until the table info is stored
   337  func (m *memoryTableInfoProvider) Read(schema, table string, version uint64) *model.TableInfo {
   338  	key := tableSchemaKey{
   339  		schema:  schema,
   340  		table:   table,
   341  		version: version,
   342  	}
   343  	return m.memo[key]
   344  }
   345  
   346  type tableSchemaKey struct {
   347  	schema  string
   348  	table   string
   349  	version uint64
   350  }