github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/common/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 common
    15  
    16  import (
    17  	"encoding/binary"
    18  	"encoding/json"
    19  	"time"
    20  
    21  	"github.com/pingcap/tiflow/cdc/model"
    22  	"github.com/pingcap/tiflow/pkg/config"
    23  	"github.com/tikv/client-go/v2/oracle"
    24  )
    25  
    26  // MaxRecordOverhead is used to calculate message size by sarama kafka client.
    27  // reference: https://github.com/IBM/sarama/blob/
    28  // 66521126c71c522c15a36663ae9cddc2b024c799/async_producer.go#L233
    29  // For TiCDC, minimum supported kafka version is `0.11.0.2`,
    30  // which will be treated as `version = 2` by sarama producer.
    31  const MaxRecordOverhead = 5*binary.MaxVarintLen32 + binary.MaxVarintLen64 + 1
    32  
    33  // Message represents an message to the sink
    34  type Message struct {
    35  	Key       []byte
    36  	Value     []byte
    37  	Ts        uint64            // reserved for possible output sorting
    38  	Schema    *string           // schema
    39  	Table     *string           // table
    40  	Type      model.MessageType // type
    41  	Protocol  config.Protocol   // protocol
    42  	rowsCount int               // rows in one Message
    43  	Callback  func()            // Callback function will be called when the message is sent to the sink.
    44  
    45  	// PartitionKey for pulsar, route messages to one or different partitions
    46  	PartitionKey *string
    47  }
    48  
    49  // Length returns the expected size of the Kafka message
    50  // We didn't append any `Headers` when send the message, so ignore the calculations related to it.
    51  // If `ProducerMessage` Headers fields used, this method should also adjust.
    52  func (m *Message) Length() int {
    53  	return len(m.Key) + len(m.Value) + MaxRecordOverhead
    54  }
    55  
    56  // PhysicalTime returns physical time part of Ts in time.Time
    57  func (m *Message) PhysicalTime() time.Time {
    58  	return oracle.GetTimeFromTS(m.Ts)
    59  }
    60  
    61  // GetRowsCount returns the number of rows batched in one Message
    62  func (m *Message) GetRowsCount() int {
    63  	return m.rowsCount
    64  }
    65  
    66  // SetRowsCount set the number of rows
    67  func (m *Message) SetRowsCount(cnt int) {
    68  	m.rowsCount = cnt
    69  }
    70  
    71  // IncRowsCount increase the number of rows
    72  func (m *Message) IncRowsCount() {
    73  	m.rowsCount++
    74  }
    75  
    76  // GetSchema returns schema string
    77  func (m *Message) GetSchema() string {
    78  	if m.Schema == nil {
    79  		return ""
    80  	}
    81  	return *m.Schema
    82  }
    83  
    84  // GetTable returns the Table string
    85  func (m *Message) GetTable() string {
    86  	if m.Table == nil {
    87  		return ""
    88  	}
    89  	return *m.Table
    90  }
    91  
    92  // SetPartitionKey sets the PartitionKey for a message
    93  // PartitionKey is used for pulsar producer, route messages to one or different partitions
    94  func (m *Message) SetPartitionKey(key string) {
    95  	m.PartitionKey = &key
    96  }
    97  
    98  // GetPartitionKey returns the GetPartitionKey
    99  func (m *Message) GetPartitionKey() string {
   100  	if m.PartitionKey == nil {
   101  		return ""
   102  	}
   103  	return *m.PartitionKey
   104  }
   105  
   106  // NewDDLMsg creates a DDL message.
   107  func NewDDLMsg(proto config.Protocol, key, value []byte, event *model.DDLEvent) *Message {
   108  	return NewMsg(
   109  		proto,
   110  		key,
   111  		value,
   112  		event.CommitTs,
   113  		model.MessageTypeDDL,
   114  		&event.TableInfo.TableName.Schema,
   115  		&event.TableInfo.TableName.Table,
   116  	)
   117  }
   118  
   119  // NewResolvedMsg creates a resolved ts message.
   120  func NewResolvedMsg(proto config.Protocol, key, value []byte, ts uint64) *Message {
   121  	return NewMsg(proto, key, value, ts, model.MessageTypeResolved, nil, nil)
   122  }
   123  
   124  // NewMsg should be used when creating a Message struct.
   125  // It copies the input byte slices to avoid any surprises in asynchronous MQ writes.
   126  func NewMsg(
   127  	proto config.Protocol,
   128  	key []byte,
   129  	value []byte,
   130  	ts uint64,
   131  	ty model.MessageType,
   132  	schema, table *string,
   133  ) *Message {
   134  	ret := &Message{
   135  		Key:       nil,
   136  		Value:     nil,
   137  		Ts:        ts,
   138  		Schema:    schema,
   139  		Table:     table,
   140  		Type:      ty,
   141  		Protocol:  proto,
   142  		rowsCount: 0,
   143  	}
   144  
   145  	if key != nil {
   146  		ret.Key = make([]byte, len(key))
   147  		copy(ret.Key, key)
   148  	}
   149  
   150  	if value != nil {
   151  		ret.Value = make([]byte, len(value))
   152  		copy(ret.Value, value)
   153  	}
   154  
   155  	return ret
   156  }
   157  
   158  // ClaimCheckMessage is the message sent to the claim-check external storage.
   159  type ClaimCheckMessage struct {
   160  	Key   []byte `json:"key"`
   161  	Value []byte `json:"value"`
   162  }
   163  
   164  // UnmarshalClaimCheckMessage unmarshal bytes to ClaimCheckMessage.
   165  func UnmarshalClaimCheckMessage(data []byte) (*ClaimCheckMessage, error) {
   166  	var m ClaimCheckMessage
   167  	err := json.Unmarshal(data, &m)
   168  	return &m, err
   169  }