github.com/pingcap/br@v5.3.0-alpha.0.20220125034240-ec59c7b6ce30+incompatible/pkg/cdclog/decoder.go (about)

     1  // Copyright 2020 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 cdclog
    15  
    16  import (
    17  	"bytes"
    18  	"encoding/base64"
    19  	"encoding/binary"
    20  	"encoding/json"
    21  	"strconv"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/log"
    25  	timodel "github.com/pingcap/parser/model"
    26  	"github.com/pingcap/parser/mysql"
    27  	"github.com/pingcap/tidb/types"
    28  	"go.uber.org/zap"
    29  
    30  	berrors "github.com/pingcap/br/pkg/errors"
    31  )
    32  
    33  // ColumnFlagType represents the type of Column.
    34  type ColumnFlagType uint64
    35  
    36  // ItemType represents the type of SortItem.
    37  type ItemType uint
    38  
    39  const (
    40  	// RowChanged represents dml type.
    41  	RowChanged ItemType = 1 << ItemType(iota)
    42  	// DDL represents ddl type.
    43  	DDL
    44  )
    45  
    46  // TODO let cdc import these flags.
    47  const (
    48  	// BatchVersion1 represents the version of batch format.
    49  	BatchVersion1 uint64 = 1
    50  )
    51  
    52  const (
    53  	// BinaryFlag means the Column charset is binary.
    54  	BinaryFlag ColumnFlagType = 1 << ColumnFlagType(iota)
    55  	// HandleKeyFlag means the Column is selected as the handle key.
    56  	HandleKeyFlag
    57  	// GeneratedColumnFlag means the Column is a generated Column.
    58  	GeneratedColumnFlag
    59  	// PrimaryKeyFlag means the Column is primary key.
    60  	PrimaryKeyFlag
    61  	// UniqueKeyFlag means the Column is unique key.
    62  	UniqueKeyFlag
    63  	// MultipleKeyFlag means the Column is multiple key.
    64  	MultipleKeyFlag
    65  	// NullableFlag means the Column is nullable.
    66  	NullableFlag
    67  )
    68  
    69  // Column represents the column data define by cdc.
    70  type Column struct {
    71  	Type byte `json:"t"`
    72  
    73  	// WhereHandle is deprecated
    74  	// WhereHandle is replaced by HandleKey in Flag.
    75  	WhereHandle *bool          `json:"h,omitempty"`
    76  	Flag        ColumnFlagType `json:"f"`
    77  	Value       interface{}    `json:"v"`
    78  }
    79  
    80  // ToDatum encode Column to Datum.
    81  func (c Column) ToDatum() (types.Datum, error) {
    82  	var (
    83  		val interface{}
    84  		err error
    85  	)
    86  
    87  	switch c.Type {
    88  	case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeYear:
    89  		val, err = c.Value.(json.Number).Int64()
    90  		if err != nil {
    91  			return types.Datum{}, errors.Trace(err)
    92  		}
    93  	case mysql.TypeFloat, mysql.TypeDouble:
    94  		val, err = c.Value.(json.Number).Float64()
    95  		if err != nil {
    96  			return types.Datum{}, errors.Trace(err)
    97  		}
    98  	default:
    99  		val = c.Value
   100  	}
   101  	return types.NewDatum(val), nil
   102  }
   103  
   104  func formatColumnVal(c Column) Column {
   105  	switch c.Type {
   106  	case mysql.TypeVarchar, mysql.TypeString:
   107  		if s, ok := c.Value.(string); ok {
   108  			// according to open protocol https://docs.pingcap.com/tidb/dev/ticdc-open-protocol
   109  			// CHAR/BINARY have the same type: 254
   110  			// VARCHAR/VARBINARY have the same type: 15
   111  			// we need to process it by its flag.
   112  			if c.Flag&BinaryFlag != 0 {
   113  				val, err := strconv.Unquote("\"" + s + "\"")
   114  				if err != nil {
   115  					log.Panic("invalid Column value, please report a bug", zap.Any("col", c), zap.Error(err))
   116  				}
   117  				c.Value = val
   118  			}
   119  		}
   120  	case mysql.TypeTinyBlob, mysql.TypeMediumBlob,
   121  		mysql.TypeLongBlob, mysql.TypeBlob:
   122  		if s, ok := c.Value.(string); ok {
   123  			var err error
   124  			c.Value, err = base64.StdEncoding.DecodeString(s)
   125  			if err != nil {
   126  				log.Panic("invalid Column value, please report a bug", zap.Any("col", c), zap.Error(err))
   127  			}
   128  		}
   129  	case mysql.TypeBit:
   130  		if s, ok := c.Value.(json.Number); ok {
   131  			intNum, err := s.Int64()
   132  			if err != nil {
   133  				log.Panic("invalid Column value, please report a bug", zap.Any("col", c), zap.Error(err))
   134  			}
   135  			c.Value = uint64(intNum)
   136  		}
   137  	}
   138  	return c
   139  }
   140  
   141  type messageKey struct {
   142  	TS        uint64 `json:"ts"`
   143  	Schema    string `json:"scm,omitempty"`
   144  	Table     string `json:"tbl,omitempty"`
   145  	RowID     int64  `json:"rid,omitempty"`
   146  	Partition *int64 `json:"ptn,omitempty"`
   147  }
   148  
   149  // Encode the messageKey.
   150  func (m *messageKey) Encode() ([]byte, error) {
   151  	return json.Marshal(m)
   152  }
   153  
   154  // Decode the messageKey.
   155  func (m *messageKey) Decode(data []byte) error {
   156  	return json.Unmarshal(data, m)
   157  }
   158  
   159  // MessageDDL represents the ddl changes.
   160  type MessageDDL struct {
   161  	Query string             `json:"q"`
   162  	Type  timodel.ActionType `json:"t"`
   163  }
   164  
   165  // Encode the DDL message.
   166  func (m *MessageDDL) Encode() ([]byte, error) {
   167  	return json.Marshal(m)
   168  }
   169  
   170  // Decode the DDL message.
   171  func (m *MessageDDL) Decode(data []byte) error {
   172  	return json.Unmarshal(data, m)
   173  }
   174  
   175  // MessageRow represents the row changes in same commit ts.
   176  type MessageRow struct {
   177  	Update     map[string]Column `json:"u,omitempty"`
   178  	PreColumns map[string]Column `json:"p,omitempty"`
   179  	Delete     map[string]Column `json:"d,omitempty"`
   180  }
   181  
   182  // Encode the Row message.
   183  func (m *MessageRow) Encode() ([]byte, error) {
   184  	return json.Marshal(m)
   185  }
   186  
   187  // Decode the Row message.
   188  func (m *MessageRow) Decode(data []byte) error {
   189  	decoder := json.NewDecoder(bytes.NewReader(data))
   190  	decoder.UseNumber()
   191  	err := decoder.Decode(m)
   192  	if err != nil {
   193  		return errors.Trace(err)
   194  	}
   195  	for colName, column := range m.Update {
   196  		m.Update[colName] = formatColumnVal(column)
   197  	}
   198  	for colName, column := range m.Delete {
   199  		m.Delete[colName] = formatColumnVal(column)
   200  	}
   201  	for colName, column := range m.PreColumns {
   202  		m.PreColumns[colName] = formatColumnVal(column)
   203  	}
   204  	return nil
   205  }
   206  
   207  // SortItem represents a DDL item or Row changed item.
   208  type SortItem struct {
   209  	ItemType ItemType
   210  	Data     interface{}
   211  	Schema   string
   212  	Table    string
   213  	RowID    int64
   214  	TS       uint64
   215  }
   216  
   217  // LessThan return whether it has smaller commit ts than other item.
   218  func (s *SortItem) LessThan(other *SortItem) bool {
   219  	if other != nil {
   220  		return s.TS < other.TS
   221  	}
   222  	return true
   223  }
   224  
   225  // JSONEventBatchMixedDecoder decodes the byte of a batch into the original messages.
   226  type JSONEventBatchMixedDecoder struct {
   227  	mixedBytes []byte
   228  }
   229  
   230  func (b *JSONEventBatchMixedDecoder) decodeNextKey() (*messageKey, error) {
   231  	keyLen := binary.BigEndian.Uint64(b.mixedBytes[:8])
   232  	key := b.mixedBytes[8 : keyLen+8]
   233  	// drop value bytes
   234  	msgKey := new(messageKey)
   235  	err := msgKey.Decode(key)
   236  	if err != nil {
   237  		return nil, errors.Trace(err)
   238  	}
   239  	b.mixedBytes = b.mixedBytes[keyLen+8:]
   240  	return msgKey, nil
   241  }
   242  
   243  // NextEvent return next item depends on type.
   244  func (b *JSONEventBatchMixedDecoder) NextEvent(itemType ItemType) (*SortItem, error) {
   245  	if !b.HasNext() {
   246  		return nil, nil
   247  	}
   248  	nextKey, err := b.decodeNextKey()
   249  	if err != nil {
   250  		return nil, errors.Trace(err)
   251  	}
   252  
   253  	valueLen := binary.BigEndian.Uint64(b.mixedBytes[:8])
   254  	value := b.mixedBytes[8 : valueLen+8]
   255  	b.mixedBytes = b.mixedBytes[valueLen+8:]
   256  
   257  	var m interface{}
   258  	if itemType == DDL {
   259  		m = new(MessageDDL)
   260  		if err := m.(*MessageDDL).Decode(value); err != nil {
   261  			return nil, errors.Trace(err)
   262  		}
   263  	} else if itemType == RowChanged {
   264  		m = new(MessageRow)
   265  		if err := m.(*MessageRow).Decode(value); err != nil {
   266  			return nil, errors.Trace(err)
   267  		}
   268  	}
   269  
   270  	item := &SortItem{
   271  		ItemType: itemType,
   272  		Data:     m,
   273  		Schema:   nextKey.Schema,
   274  		Table:    nextKey.Table,
   275  		TS:       nextKey.TS,
   276  		RowID:    nextKey.RowID,
   277  	}
   278  	return item, nil
   279  }
   280  
   281  // HasNext represents whether it has next kv to decode.
   282  func (b *JSONEventBatchMixedDecoder) HasNext() bool {
   283  	return len(b.mixedBytes) > 0
   284  }
   285  
   286  // NewJSONEventBatchDecoder creates a new JSONEventBatchDecoder.
   287  func NewJSONEventBatchDecoder(data []byte) (*JSONEventBatchMixedDecoder, error) {
   288  	if len(data) == 0 {
   289  		return nil, nil
   290  	}
   291  	version := binary.BigEndian.Uint64(data[:8])
   292  	data = data[8:]
   293  	if version != BatchVersion1 {
   294  		return nil, errors.Annotate(berrors.ErrPiTRInvalidCDCLogFormat, "unexpected key format version")
   295  	}
   296  	return &JSONEventBatchMixedDecoder{
   297  		mixedBytes: data,
   298  	}, nil
   299  }