Files
tidb/br/pkg/cdclog/decoder.go

300 lines
7.7 KiB
Go

// Copyright 2020 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package cdclog
import (
"bytes"
"encoding/base64"
"encoding/binary"
"encoding/json"
"strconv"
"github.com/pingcap/errors"
"github.com/pingcap/log"
berrors "github.com/pingcap/tidb/br/pkg/errors"
timodel "github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/types"
"go.uber.org/zap"
)
// ColumnFlagType represents the type of Column.
type ColumnFlagType uint64
// ItemType represents the type of SortItem.
type ItemType uint
const (
// RowChanged represents dml type.
RowChanged ItemType = 1 << ItemType(iota)
// DDL represents ddl type.
DDL
)
// TODO let cdc import these flags.
const (
// BatchVersion1 represents the version of batch format.
BatchVersion1 uint64 = 1
)
const (
// BinaryFlag means the Column charset is binary.
BinaryFlag ColumnFlagType = 1 << ColumnFlagType(iota)
// HandleKeyFlag means the Column is selected as the handle key.
HandleKeyFlag
// GeneratedColumnFlag means the Column is a generated Column.
GeneratedColumnFlag
// PrimaryKeyFlag means the Column is primary key.
PrimaryKeyFlag
// UniqueKeyFlag means the Column is unique key.
UniqueKeyFlag
// MultipleKeyFlag means the Column is multiple key.
MultipleKeyFlag
// NullableFlag means the Column is nullable.
NullableFlag
)
// Column represents the column data define by cdc.
type Column struct {
Type byte `json:"t"`
// WhereHandle is deprecated
// WhereHandle is replaced by HandleKey in Flag.
WhereHandle *bool `json:"h,omitempty"`
Flag ColumnFlagType `json:"f"`
Value interface{} `json:"v"`
}
// ToDatum encode Column to Datum.
func (c Column) ToDatum() (types.Datum, error) {
var (
val interface{}
err error
)
switch c.Type {
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeYear:
val, err = c.Value.(json.Number).Int64()
if err != nil {
return types.Datum{}, errors.Trace(err)
}
case mysql.TypeFloat, mysql.TypeDouble:
val, err = c.Value.(json.Number).Float64()
if err != nil {
return types.Datum{}, errors.Trace(err)
}
default:
val = c.Value
}
return types.NewDatum(val), nil
}
func formatColumnVal(c Column) Column {
switch c.Type {
case mysql.TypeVarchar, mysql.TypeString:
if s, ok := c.Value.(string); ok {
// according to open protocol https://docs.pingcap.com/tidb/dev/ticdc-open-protocol
// CHAR/BINARY have the same type: 254
// VARCHAR/VARBINARY have the same type: 15
// we need to process it by its flag.
if c.Flag&BinaryFlag != 0 {
val, err := strconv.Unquote("\"" + s + "\"")
if err != nil {
log.Panic("invalid Column value, please report a bug", zap.Any("col", c), zap.Error(err))
}
c.Value = val
}
}
case mysql.TypeTinyBlob, mysql.TypeMediumBlob,
mysql.TypeLongBlob, mysql.TypeBlob:
if s, ok := c.Value.(string); ok {
var err error
c.Value, err = base64.StdEncoding.DecodeString(s)
if err != nil {
log.Panic("invalid Column value, please report a bug", zap.Any("col", c), zap.Error(err))
}
}
case mysql.TypeBit:
if s, ok := c.Value.(json.Number); ok {
intNum, err := s.Int64()
if err != nil {
log.Panic("invalid Column value, please report a bug", zap.Any("col", c), zap.Error(err))
}
c.Value = uint64(intNum)
}
}
return c
}
type messageKey struct {
TS uint64 `json:"ts"`
Schema string `json:"scm,omitempty"`
Table string `json:"tbl,omitempty"`
RowID int64 `json:"rid,omitempty"`
Partition *int64 `json:"ptn,omitempty"`
}
// Encode the messageKey.
func (m *messageKey) Encode() ([]byte, error) {
return json.Marshal(m)
}
// Decode the messageKey.
func (m *messageKey) Decode(data []byte) error {
return json.Unmarshal(data, m)
}
// MessageDDL represents the ddl changes.
type MessageDDL struct {
Query string `json:"q"`
Type timodel.ActionType `json:"t"`
}
// Encode the DDL message.
func (m *MessageDDL) Encode() ([]byte, error) {
return json.Marshal(m)
}
// Decode the DDL message.
func (m *MessageDDL) Decode(data []byte) error {
return json.Unmarshal(data, m)
}
// MessageRow represents the row changes in same commit ts.
type MessageRow struct {
Update map[string]Column `json:"u,omitempty"`
PreColumns map[string]Column `json:"p,omitempty"`
Delete map[string]Column `json:"d,omitempty"`
}
// Encode the Row message.
func (m *MessageRow) Encode() ([]byte, error) {
return json.Marshal(m)
}
// Decode the Row message.
func (m *MessageRow) Decode(data []byte) error {
decoder := json.NewDecoder(bytes.NewReader(data))
decoder.UseNumber()
err := decoder.Decode(m)
if err != nil {
return errors.Trace(err)
}
for colName, column := range m.Update {
m.Update[colName] = formatColumnVal(column)
}
for colName, column := range m.Delete {
m.Delete[colName] = formatColumnVal(column)
}
for colName, column := range m.PreColumns {
m.PreColumns[colName] = formatColumnVal(column)
}
return nil
}
// SortItem represents a DDL item or Row changed item.
type SortItem struct {
ItemType ItemType
Data interface{}
Schema string
Table string
RowID int64
TS uint64
}
// LessThan return whether it has smaller commit ts than other item.
func (s *SortItem) LessThan(other *SortItem) bool {
if other != nil {
return s.TS < other.TS
}
return true
}
// JSONEventBatchMixedDecoder decodes the byte of a batch into the original messages.
type JSONEventBatchMixedDecoder struct {
mixedBytes []byte
}
func (b *JSONEventBatchMixedDecoder) decodeNextKey() (*messageKey, error) {
keyLen := binary.BigEndian.Uint64(b.mixedBytes[:8])
key := b.mixedBytes[8 : keyLen+8]
// drop value bytes
msgKey := new(messageKey)
err := msgKey.Decode(key)
if err != nil {
return nil, errors.Trace(err)
}
b.mixedBytes = b.mixedBytes[keyLen+8:]
return msgKey, nil
}
// NextEvent return next item depends on type.
func (b *JSONEventBatchMixedDecoder) NextEvent(itemType ItemType) (*SortItem, error) {
if !b.HasNext() {
return nil, nil
}
nextKey, err := b.decodeNextKey()
if err != nil {
return nil, errors.Trace(err)
}
valueLen := binary.BigEndian.Uint64(b.mixedBytes[:8])
value := b.mixedBytes[8 : valueLen+8]
b.mixedBytes = b.mixedBytes[valueLen+8:]
var m interface{}
if itemType == DDL {
m = new(MessageDDL)
if err := m.(*MessageDDL).Decode(value); err != nil {
return nil, errors.Trace(err)
}
} else if itemType == RowChanged {
m = new(MessageRow)
if err := m.(*MessageRow).Decode(value); err != nil {
return nil, errors.Trace(err)
}
}
item := &SortItem{
ItemType: itemType,
Data: m,
Schema: nextKey.Schema,
Table: nextKey.Table,
TS: nextKey.TS,
RowID: nextKey.RowID,
}
return item, nil
}
// HasNext represents whether it has next kv to decode.
func (b *JSONEventBatchMixedDecoder) HasNext() bool {
return len(b.mixedBytes) > 0
}
// NewJSONEventBatchDecoder creates a new JSONEventBatchDecoder.
func NewJSONEventBatchDecoder(data []byte) (*JSONEventBatchMixedDecoder, error) {
if len(data) == 0 {
return nil, nil
}
version := binary.BigEndian.Uint64(data[:8])
data = data[8:]
if version != BatchVersion1 {
return nil, errors.Annotate(berrors.ErrPiTRInvalidCDCLogFormat, "unexpected key format version")
}
return &JSONEventBatchMixedDecoder{
mixedBytes: data,
}, nil
}